From 1d761775107b3b4c73d2b968c352b805225284eb Mon Sep 17 00:00:00 2001 From: Colin Goodheart-Smithe Date: Tue, 17 May 2016 14:16:20 +0100 Subject: [PATCH] Adds aggregation profiling (not including reduce phase) Add Aggregation profiling initially only be for the shard phases (i.e. the reduce phase will not be profiled in this change) This change refactors the query profiling class to extract abstract classes where it is useful for other profiler types to share code. --- .../action/search/SearchResponse.java | 3 +- .../aggregations/AggregatorFactories.java | 16 +- .../aggregations/AggregatorFactory.java | 243 ++++++----- .../cardinality/CardinalityAggregator.java | 1 - .../controller/SearchPhaseController.java | 5 +- .../search/internal/ContextIndexSearcher.java | 2 +- .../internal/InternalSearchResponse.java | 5 +- .../profile/AbstractInternalProfileTree.java | 209 +++++++++ .../search/profile/AbstractProfiler.java | 54 +++ .../search/profile/ProfileShardResult.java | 75 +--- .../search/profile/Profilers.java | 10 +- .../profile/SearchProfileShardResults.java | 72 ++-- .../AggregationProfileBreakdown.java | 30 ++ .../AggregationProfileShardResult.java | 79 ++++ .../aggregation/AggregationProfiler.java | 57 +++ .../aggregation/AggregationTimingType.java | 34 ++ .../InternalAggregationProfileTree.java | 46 ++ .../aggregation/ProfilingAggregator.java | 103 +++++ .../ProfilingLeafBucketCollector.java | 43 ++ .../query/InternalQueryProfileTree.java | 186 +------- .../query/QueryProfileShardResult.java | 104 +++++ .../search/profile/query/QueryProfiler.java | 41 +- .../search/query/QueryPhase.java | 8 +- .../search/query/QuerySearchResult.java | 18 +- .../aggregation/AggregationProfilerIT.java | 403 ++++++++++++++++++ .../search/profile/query/QueryProfilerIT.java | 45 +- ...fileTests.java => QueryProfilerTests.java} | 10 +- 27 files changed, 1430 insertions(+), 472 deletions(-) create mode 100644 core/src/main/java/org/elasticsearch/search/profile/AbstractInternalProfileTree.java create mode 100644 core/src/main/java/org/elasticsearch/search/profile/AbstractProfiler.java create mode 100644 core/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationProfileBreakdown.java create mode 100644 core/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationProfileShardResult.java create mode 100644 core/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationProfiler.java create mode 100644 core/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationTimingType.java create mode 100644 core/src/main/java/org/elasticsearch/search/profile/aggregation/InternalAggregationProfileTree.java create mode 100644 core/src/main/java/org/elasticsearch/search/profile/aggregation/ProfilingAggregator.java create mode 100644 core/src/main/java/org/elasticsearch/search/profile/aggregation/ProfilingLeafBucketCollector.java create mode 100644 core/src/main/java/org/elasticsearch/search/profile/query/QueryProfileShardResult.java create mode 100644 core/src/test/java/org/elasticsearch/search/profile/aggregation/AggregationProfilerIT.java rename core/src/test/java/org/elasticsearch/search/profile/query/{ProfileTests.java => QueryProfilerTests.java} (96%) 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 b76e640fa95..8e144537c84 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchResponse.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchResponse.java @@ -36,7 +36,6 @@ 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; @@ -169,7 +168,7 @@ public class SearchResponse extends ActionResponse implements StatusToXContent { * * @return The profile results or an empty map */ - public @Nullable Map> getProfileResults() { + public @Nullable Map getProfileResults() { return internalResponse.profile(); } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java b/core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java index 38fb0e6a431..2d8ea318fd6 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java @@ -27,6 +27,8 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationPath; import org.elasticsearch.search.aggregations.support.AggregationPath.PathElement; +import org.elasticsearch.search.profile.Profilers; +import org.elasticsearch.search.profile.aggregation.ProfilingAggregator; import java.io.IOException; import java.util.ArrayList; @@ -81,7 +83,12 @@ public class AggregatorFactories { // propagate the fact that only bucket 0 will be collected with single-bucket // aggs final boolean collectsFromSingleBucket = false; - aggregators[i] = factories[i].create(parent, collectsFromSingleBucket); + Aggregator factory = factories[i].create(parent, collectsFromSingleBucket); + Profilers profilers = factory.context().searchContext().getProfilers(); + if (profilers != null) { + factory = new ProfilingAggregator(factory, profilers.getAggregationProfiler()); + } + aggregators[i] = factory; } return aggregators; } @@ -92,7 +99,12 @@ public class AggregatorFactories { for (int i = 0; i < factories.length; i++) { // top-level aggs only get called with bucket 0 final boolean collectsFromSingleBucket = true; - aggregators[i] = factories[i].create(null, collectsFromSingleBucket); + Aggregator factory = factories[i].create(null, collectsFromSingleBucket); + Profilers profilers = factory.context().searchContext().getProfilers(); + if (profilers != null) { + factory = new ProfilingAggregator(factory, profilers.getAggregationProfiler()); + } + aggregators[i] = factory; } return aggregators; } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java b/core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java index 3fced89a014..854838b7441 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java @@ -28,13 +28,139 @@ import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.internal.SearchContext.Lifetime; - import java.io.IOException; import java.util.List; import java.util.Map; public abstract class AggregatorFactory> { + public static final class MultiBucketAggregatorWrapper extends Aggregator { + private final BigArrays bigArrays; + private final Aggregator parent; + private final AggregatorFactory factory; + private final Aggregator first; + ObjectArray aggregators; + ObjectArray collectors; + + MultiBucketAggregatorWrapper(BigArrays bigArrays, AggregationContext context, Aggregator parent, AggregatorFactory factory, + Aggregator first) { + this.bigArrays = bigArrays; + this.parent = parent; + this.factory = factory; + this.first = first; + context.searchContext().addReleasable(this, Lifetime.PHASE); + aggregators = bigArrays.newObjectArray(1); + aggregators.set(0, first); + collectors = bigArrays.newObjectArray(1); + } + + public Class getWrappedClass() { + return first.getClass(); + } + + @Override + public String name() { + return first.name(); + } + + @Override + public AggregationContext context() { + return first.context(); + } + + @Override + public Aggregator parent() { + return first.parent(); + } + + @Override + public boolean needsScores() { + return first.needsScores(); + } + + @Override + public Aggregator subAggregator(String name) { + throw new UnsupportedOperationException(); + } + + @Override + public void preCollection() throws IOException { + for (long i = 0; i < aggregators.size(); ++i) { + final Aggregator aggregator = aggregators.get(i); + if (aggregator != null) { + aggregator.preCollection(); + } + } + } + + @Override + public void postCollection() throws IOException { + for (long i = 0; i < aggregators.size(); ++i) { + final Aggregator aggregator = aggregators.get(i); + if (aggregator != null) { + aggregator.postCollection(); + } + } + } + + @Override + public LeafBucketCollector getLeafCollector(final LeafReaderContext ctx) { + for (long i = 0; i < collectors.size(); ++i) { + collectors.set(i, null); + } + return new LeafBucketCollector() { + Scorer scorer; + + @Override + public void setScorer(Scorer scorer) throws IOException { + this.scorer = scorer; + } + + @Override + public void collect(int doc, long bucket) throws IOException { + collectors = bigArrays.grow(collectors, bucket + 1); + + LeafBucketCollector collector = collectors.get(bucket); + if (collector == null) { + aggregators = bigArrays.grow(aggregators, bucket + 1); + Aggregator aggregator = aggregators.get(bucket); + if (aggregator == null) { + aggregator = factory.create(parent, true); + aggregator.preCollection(); + aggregators.set(bucket, aggregator); + } + collector = aggregator.getLeafCollector(ctx); + collector.setScorer(scorer); + collectors.set(bucket, collector); + } + collector.collect(doc, 0); + } + + }; + } + + @Override + public InternalAggregation buildAggregation(long bucket) throws IOException { + if (bucket < aggregators.size()) { + Aggregator aggregator = aggregators.get(bucket); + if (aggregator != null) { + return aggregator.buildAggregation(0); + } + } + return buildEmptyAggregation(); + } + + @Override + public InternalAggregation buildEmptyAggregation() { + return first.buildEmptyAggregation(); + } + + @Override + public void close() { + Releasables.close(aggregators, collectors); + } + } + protected final String name; protected final Type type; protected final AggregatorFactory parent; @@ -112,120 +238,7 @@ public abstract class AggregatorFactory> { final Aggregator parent) throws IOException { final Aggregator first = factory.create(parent, true); final BigArrays bigArrays = context.bigArrays(); - return new Aggregator() { - - ObjectArray aggregators; - ObjectArray collectors; - - { - context.searchContext().addReleasable(this, Lifetime.PHASE); - aggregators = bigArrays.newObjectArray(1); - aggregators.set(0, first); - collectors = bigArrays.newObjectArray(1); - } - - @Override - public String name() { - return first.name(); - } - - @Override - public AggregationContext context() { - return first.context(); - } - - @Override - public Aggregator parent() { - return first.parent(); - } - - @Override - public boolean needsScores() { - return first.needsScores(); - } - - @Override - public Aggregator subAggregator(String name) { - throw new UnsupportedOperationException(); - } - - @Override - public void preCollection() throws IOException { - for (long i = 0; i < aggregators.size(); ++i) { - final Aggregator aggregator = aggregators.get(i); - if (aggregator != null) { - aggregator.preCollection(); - } - } - } - - @Override - public void postCollection() throws IOException { - for (long i = 0; i < aggregators.size(); ++i) { - final Aggregator aggregator = aggregators.get(i); - if (aggregator != null) { - aggregator.postCollection(); - } - } - } - - @Override - public LeafBucketCollector getLeafCollector(final LeafReaderContext ctx) { - for (long i = 0; i < collectors.size(); ++i) { - collectors.set(i, null); - } - return new LeafBucketCollector() { - Scorer scorer; - - @Override - public void setScorer(Scorer scorer) throws IOException { - this.scorer = scorer; - } - - @Override - public void collect(int doc, long bucket) throws IOException { - aggregators = bigArrays.grow(aggregators, bucket + 1); - collectors = bigArrays.grow(collectors, bucket + 1); - - LeafBucketCollector collector = collectors.get(bucket); - if (collector == null) { - Aggregator aggregator = aggregators.get(bucket); - if (aggregator == null) { - aggregator = factory.create(parent, true); - aggregator.preCollection(); - aggregators.set(bucket, aggregator); - } - collector = aggregator.getLeafCollector(ctx); - collector.setScorer(scorer); - collectors.set(bucket, collector); - } - collector.collect(doc, 0); - } - - }; - } - - @Override - public InternalAggregation buildAggregation(long bucket) throws IOException { - if (bucket < aggregators.size()) { - Aggregator aggregator = aggregators.get(bucket); - if (aggregator != null) { - return aggregator.buildAggregation(0); - } - } - return buildEmptyAggregation(); - } - - @Override - public InternalAggregation buildEmptyAggregation() { - return first.buildEmptyAggregation(); - } - - @Override - public void close() { - Releasables.close(aggregators, collectors); - } - }; + return new MultiBucketAggregatorWrapper(bigArrays, context, parent, factory, first); } } \ No newline at end of file diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregator.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregator.java index b5ed80a0022..0793bacf722 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregator.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregator.java @@ -35,7 +35,6 @@ import org.elasticsearch.common.util.LongArray; import org.elasticsearch.common.util.ObjectArray; import org.elasticsearch.index.fielddata.SortedBinaryDocValues; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; -import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollector; 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 36fe562a568..b2ce044e4fc 100644 --- a/core/src/main/java/org/elasticsearch/search/controller/SearchPhaseController.java +++ b/core/src/main/java/org/elasticsearch/search/controller/SearchPhaseController.java @@ -51,8 +51,9 @@ 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.SearchProfileShardResults; import org.elasticsearch.search.profile.ProfileShardResult; +import org.elasticsearch.search.profile.SearchProfileShardResults; +import org.elasticsearch.search.profile.query.QueryProfileShardResult; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.query.QuerySearchResultProvider; import org.elasticsearch.search.suggest.Suggest; @@ -407,7 +408,7 @@ public class SearchPhaseController extends AbstractComponent { //Collect profile results SearchProfileShardResults shardResults = null; if (!queryResults.isEmpty() && firstResult.profileResults() != null) { - Map> profileResults = new HashMap<>(queryResults.size()); + 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()); 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 df1007ebc71..50e91e082cd 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java +++ b/core/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java @@ -122,7 +122,7 @@ public class ContextIndexSearcher extends IndexSearcher implements Releasable { weight = super.createWeight(query, needsScores); } finally { profile.stopAndRecordTime(); - profiler.pollLastQuery(); + profiler.pollLastElement(); } return new ProfileWeight(query, weight, profile); } else { 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 9c33889dc9c..26410cc9680 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/InternalSearchResponse.java +++ b/core/src/main/java/org/elasticsearch/search/internal/InternalSearchResponse.java @@ -28,13 +28,12 @@ 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.SearchProfileShardResults; import org.elasticsearch.search.profile.ProfileShardResult; +import org.elasticsearch.search.profile.SearchProfileShardResults; 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; @@ -99,7 +98,7 @@ public class InternalSearchResponse implements Streamable, ToXContent { * * @return Profile results */ - public Map> profile() { + public Map profile() { if (profileResults == null) { return Collections.emptyMap(); } diff --git a/core/src/main/java/org/elasticsearch/search/profile/AbstractInternalProfileTree.java b/core/src/main/java/org/elasticsearch/search/profile/AbstractInternalProfileTree.java new file mode 100644 index 00000000000..31cb3c21237 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/profile/AbstractInternalProfileTree.java @@ -0,0 +1,209 @@ +/* + * 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.profile.query.QueryProfileBreakdown; + +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Deque; +import java.util.List; +import java.util.Map; + +public abstract class AbstractInternalProfileTree, E> { + + protected ArrayList timings; + /** Maps the Query to it's list of children. This is basically the dependency tree */ + protected ArrayList> tree; + /** A list of the original queries, keyed by index position */ + protected ArrayList elements; + /** A list of top-level "roots". Each root can have its own tree of profiles */ + protected ArrayList roots; + /** A temporary stack used to record where we are in the dependency tree. */ + protected Deque stack; + private int currentToken = 0; + + public AbstractInternalProfileTree() { + timings = new ArrayList<>(10); + stack = new ArrayDeque<>(10); + tree = new ArrayList<>(10); + elements = new ArrayList<>(10); + roots = new ArrayList<>(10); + } + + /** + * Returns a {@link QueryProfileBreakdown} 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 PB getProfileBreakdown(E 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); + } + + /** + * 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 QueryProfileBreakdown} to track the timings of + * this query + * + * @param element + * The element to profile + * @param token + * The assigned token for this element + * @return A ProfileBreakdown to profile this element + */ + private PB addDependencyNode(E element, int token) { + + // Add a new slot in the dependency tree + tree.add(new ArrayList<>(5)); + + // Save our query for lookup later + elements.add(element); + + PB queryTimings = createProfileBreakdown(); + timings.add(token, queryTimings); + return queryTimings; + } + + protected abstract PB createProfileBreakdown(); + + /** + * 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 getTree() { + ArrayList results = new ArrayList<>(5); + for (Integer root : roots) { + results.add(doGetTree(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 doGetTree(int token) { + E element = elements.get(token); + PB 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 = doGetTree(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 type = getTypeFromElement(element); + String description = getDescriptionFromElement(element); + return new ProfileResult(type, description, timings, childrenProfileResults, nodeTime); + } + + protected abstract String getTypeFromElement(E element); + + protected abstract String getDescriptionFromElement(E element); + + /** + * 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; + } + +} \ No newline at end of file diff --git a/core/src/main/java/org/elasticsearch/search/profile/AbstractProfiler.java b/core/src/main/java/org/elasticsearch/search/profile/AbstractProfiler.java new file mode 100644 index 00000000000..a7ccb72785e --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/profile/AbstractProfiler.java @@ -0,0 +1,54 @@ +/* + * 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.List; + +public class AbstractProfiler, E> { + + protected final AbstractInternalProfileTree profileTree; + + public AbstractProfiler(AbstractInternalProfileTree profileTree) { + this.profileTree = profileTree; + } + + /** + * Get the {@link AbstractProfileBreakdown} for the given element in the + * tree, potentially creating it if it did not exist. + */ + public PB getQueryBreakdown(E query) { + return profileTree.getProfileBreakdown(query); + } + + /** + * Removes the last (e.g. most recent) element on the stack. + */ + public void pollLastElement() { + profileTree.pollLast(); + } + + /** + * @return a hierarchical representation of the profiled tree + */ + public List getTree() { + return profileTree.getTree(); + } + +} \ No newline at end of file diff --git a/core/src/main/java/org/elasticsearch/search/profile/ProfileShardResult.java b/core/src/main/java/org/elasticsearch/search/profile/ProfileShardResult.java index 9def3db7582..2a1fb0ba9b1 100644 --- a/core/src/main/java/org/elasticsearch/search/profile/ProfileShardResult.java +++ b/core/src/main/java/org/elasticsearch/search/profile/ProfileShardResult.java @@ -22,83 +22,50 @@ 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 org.elasticsearch.search.profile.query.CollectorResult; +import org.elasticsearch.search.profile.aggregation.AggregationProfileShardResult; +import org.elasticsearch.search.profile.query.QueryProfileShardResult; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; -/** - * 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 { +public class ProfileShardResult implements Writeable { - private final List queryProfileResults; + private final List queryProfileResults; - private final CollectorResult profileCollector; + private final AggregationProfileShardResult aggProfileShardResult; - private final long rewriteTime; - - public ProfileShardResult(List queryProfileResults, long rewriteTime, - CollectorResult profileCollector) { - assert(profileCollector != null); - this.queryProfileResults = queryProfileResults; - this.profileCollector = profileCollector; - this.rewriteTime = rewriteTime; + public ProfileShardResult(List queryProfileResults, AggregationProfileShardResult aggProfileShardResult) { + this.aggProfileShardResult = aggProfileShardResult; + this.queryProfileResults = Collections.unmodifiableList(queryProfileResults); } - /** - * Read from a stream. - */ public ProfileShardResult(StreamInput in) throws IOException { int profileSize = in.readVInt(); - queryProfileResults = new ArrayList<>(profileSize); - for (int j = 0; j < profileSize; j++) { - queryProfileResults.add(new ProfileResult(in)); + List queryProfileResults = new ArrayList<>(profileSize); + for (int i = 0; i < profileSize; i++) { + QueryProfileShardResult result = new QueryProfileShardResult(in); + queryProfileResults.add(result); } - - profileCollector = new CollectorResult(in); - rewriteTime = in.readLong(); + this.queryProfileResults = Collections.unmodifiableList(queryProfileResults); + this.aggProfileShardResult = new AggregationProfileShardResult(in); } @Override public void writeTo(StreamOutput out) throws IOException { out.writeVInt(queryProfileResults.size()); - for (ProfileResult p : queryProfileResults) { - p.writeTo(out); + for (QueryProfileShardResult queryShardResult : queryProfileResults) { + queryShardResult.writeTo(out); } - profileCollector.writeTo(out); - out.writeLong(rewriteTime); + aggProfileShardResult.writeTo(out); } - - public List getQueryResults() { - return Collections.unmodifiableList(queryProfileResults); + public List getQueryProfileResults() { + return queryProfileResults; } - 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 : queryProfileResults) { - p.toXContent(builder, params); - } - builder.endArray(); - builder.field("rewrite_time", rewriteTime); - builder.startArray("collector"); - profileCollector.toXContent(builder, params); - builder.endArray(); - return builder; + public AggregationProfileShardResult getAggregationProfileResults() { + return aggProfileShardResult; } } diff --git a/core/src/main/java/org/elasticsearch/search/profile/Profilers.java b/core/src/main/java/org/elasticsearch/search/profile/Profilers.java index e9e6d88db18..d754be41f6d 100644 --- a/core/src/main/java/org/elasticsearch/search/profile/Profilers.java +++ b/core/src/main/java/org/elasticsearch/search/profile/Profilers.java @@ -20,22 +20,25 @@ package org.elasticsearch.search.profile; import org.elasticsearch.search.internal.ContextIndexSearcher; +import org.elasticsearch.search.profile.aggregation.AggregationProfiler; import org.elasticsearch.search.profile.query.QueryProfiler; import java.util.ArrayList; import java.util.Collections; import java.util.List; -/** Wrapper around several {@link QueryProfiler}s that makes management easier. */ +/** Wrapper around all the profilers that makes management easier. */ public final class Profilers { private final ContextIndexSearcher searcher; private final List queryProfilers; + private final AggregationProfiler aggProfiler; /** Sole constructor. This {@link Profilers} instance will initially wrap one {@link QueryProfiler}. */ public Profilers(ContextIndexSearcher searcher) { this.searcher = searcher; this.queryProfilers = new ArrayList<>(); + this.aggProfiler = new AggregationProfiler(); addQueryProfiler(); } @@ -57,4 +60,9 @@ public final class Profilers { return Collections.unmodifiableList(queryProfilers); } + /** Return the {@link AggregationProfiler}. */ + public AggregationProfiler getAggregationProfiler() { + return aggProfiler; + } + } diff --git a/core/src/main/java/org/elasticsearch/search/profile/SearchProfileShardResults.java b/core/src/main/java/org/elasticsearch/search/profile/SearchProfileShardResults.java index bf265dd9a7e..6794aa49399 100644 --- a/core/src/main/java/org/elasticsearch/search/profile/SearchProfileShardResults.java +++ b/core/src/main/java/org/elasticsearch/search/profile/SearchProfileShardResults.java @@ -24,6 +24,9 @@ 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 org.elasticsearch.search.profile.aggregation.AggregationProfileShardResult; +import org.elasticsearch.search.profile.aggregation.AggregationProfiler; +import org.elasticsearch.search.profile.query.QueryProfileShardResult; import org.elasticsearch.search.profile.query.QueryProfiler; import java.io.IOException; @@ -32,7 +35,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; /** * A container class to hold all the profile results across all shards. Internally @@ -40,17 +42,10 @@ import java.util.stream.Collectors; */ public final class SearchProfileShardResults implements Writeable, ToXContent{ - private Map> shardResults; + private Map shardResults; - public SearchProfileShardResults(Map> shardResults) { - Map> transformed = - shardResults.entrySet() - .stream() - .collect(Collectors.toMap( - Map.Entry::getKey, - e -> Collections.unmodifiableList(e.getValue())) - ); - this.shardResults = Collections.unmodifiableMap(transformed); + public SearchProfileShardResults(Map shardResults) { + this.shardResults = Collections.unmodifiableMap(shardResults); } public SearchProfileShardResults(StreamInput in) throws IOException { @@ -59,33 +54,22 @@ public final class SearchProfileShardResults implements Writeable, ToXContent{ 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, Collections.unmodifiableList(shardResult)); + ProfileShardResult shardResult = new ProfileShardResult(in); + shardResults.put(key, shardResult); } shardResults = Collections.unmodifiableMap(shardResults); } - public Map> getShardResults() { + public Map getShardResults() { return this.shardResults; } @Override public void writeTo(StreamOutput out) throws IOException { out.writeInt(shardResults.size()); - for (Map.Entry> entry : shardResults.entrySet()) { + for (Map.Entry entry : shardResults.entrySet()) { out.writeString(entry.getKey()); - out.writeInt(entry.getValue().size()); - - for (ProfileShardResult result : entry.getValue()) { - result.writeTo(out); - } + entry.getValue().writeTo(out); } } @@ -93,14 +77,18 @@ public final class SearchProfileShardResults implements Writeable, ToXContent{ 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()) { + for (Map.Entry entry : shardResults.entrySet()) { + builder.startObject(); + builder.field("id", entry.getKey()); + builder.startArray("searches"); + for (QueryProfileShardResult result : entry.getValue().getQueryProfileResults()) { builder.startObject(); result.toXContent(builder, params); builder.endObject(); } - builder.endArray().endObject(); + builder.endArray(); + entry.getValue().getAggregationProfileResults().toXContent(builder, params); + builder.endObject(); } builder.endArray().endObject(); @@ -112,16 +100,20 @@ public final class SearchProfileShardResults implements Writeable, ToXContent{ * 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 + * The {@link Profilers} to convert into results + * @return A {@link ProfileShardResult} representing the results for this + * shard */ - public static List buildShardResults(List profilers) { - List results = new ArrayList<>(profilers.size()); - for (QueryProfiler profiler : profilers) { - ProfileShardResult result = new ProfileShardResult(profiler.getQueryTree(), profiler.getRewriteTime(), profiler.getCollector()); - results.add(result); + public static ProfileShardResult buildShardResults(Profilers profilers) { + List queryProfilers = profilers.getQueryProfilers(); + AggregationProfiler aggProfiler = profilers.getAggregationProfiler(); + List queryResults = new ArrayList<>(queryProfilers.size()); + for (QueryProfiler queryProfiler : queryProfilers) { + QueryProfileShardResult result = new QueryProfileShardResult(queryProfiler.getTree(), queryProfiler.getRewriteTime(), + queryProfiler.getCollector()); + queryResults.add(result); } - return results; + AggregationProfileShardResult aggResults = new AggregationProfileShardResult(aggProfiler.getTree()); + return new ProfileShardResult(queryResults, aggResults); } } diff --git a/core/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationProfileBreakdown.java b/core/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationProfileBreakdown.java new file mode 100644 index 00000000000..b4cb1efe5d3 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationProfileBreakdown.java @@ -0,0 +1,30 @@ +/* + * 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.aggregation; + +import org.elasticsearch.search.profile.AbstractProfileBreakdown; + +public class AggregationProfileBreakdown extends AbstractProfileBreakdown { + + public AggregationProfileBreakdown() { + super(AggregationTimingType.values()); + } + +} diff --git a/core/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationProfileShardResult.java b/core/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationProfileShardResult.java new file mode 100644 index 00000000000..df55c5592d6 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationProfileShardResult.java @@ -0,0 +1,79 @@ +/* + * 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.aggregation; + +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 org.elasticsearch.search.profile.ProfileResult; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * 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 AggregationProfileShardResult implements Writeable, ToXContent { + + private final List aggProfileResults; + + public AggregationProfileShardResult(List aggProfileResults) { + this.aggProfileResults = aggProfileResults; + } + + /** + * Read from a stream. + */ + public AggregationProfileShardResult(StreamInput in) throws IOException { + int profileSize = in.readVInt(); + aggProfileResults = new ArrayList<>(profileSize); + for (int j = 0; j < profileSize; j++) { + aggProfileResults.add(new ProfileResult(in)); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(aggProfileResults.size()); + for (ProfileResult p : aggProfileResults) { + p.writeTo(out); + } + } + + + public List getProfileResults() { + return Collections.unmodifiableList(aggProfileResults); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startArray("aggregations"); + for (ProfileResult p : aggProfileResults) { + p.toXContent(builder, params); + } + builder.endArray(); + return builder; + } +} diff --git a/core/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationProfiler.java b/core/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationProfiler.java new file mode 100644 index 00000000000..45d401ccbdc --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationProfiler.java @@ -0,0 +1,57 @@ +/* + * 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.aggregation; + +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.profile.AbstractProfiler; + +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +public class AggregationProfiler extends AbstractProfiler { + + private final Map, AggregationProfileBreakdown> profileBrakdownLookup = new HashMap<>(); + + public AggregationProfiler() { + super(new InternalAggregationProfileTree()); + } + + @Override + public AggregationProfileBreakdown getQueryBreakdown(Aggregator agg) { + List path = getAggregatorPath(agg); + AggregationProfileBreakdown aggregationProfileBreakdown = profileBrakdownLookup.get(path); + if (aggregationProfileBreakdown == null) { + aggregationProfileBreakdown = super.getQueryBreakdown(agg); + profileBrakdownLookup.put(path, aggregationProfileBreakdown); + } + return aggregationProfileBreakdown; + } + + public static List getAggregatorPath(Aggregator agg) { + LinkedList path = new LinkedList<>(); + while (agg != null) { + path.addFirst(agg.name()); + agg = agg.parent(); + } + return path; + } +} diff --git a/core/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationTimingType.java b/core/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationTimingType.java new file mode 100644 index 00000000000..d1c5d3dd538 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationTimingType.java @@ -0,0 +1,34 @@ +/* + * 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.aggregation; + +import java.util.Locale; + +public enum AggregationTimingType { + INITIALIZE, + COLLECT, + BUILD_AGGREGATION, + REDUCE; + + @Override + public String toString() { + return name().toLowerCase(Locale.ROOT); + } +} diff --git a/core/src/main/java/org/elasticsearch/search/profile/aggregation/InternalAggregationProfileTree.java b/core/src/main/java/org/elasticsearch/search/profile/aggregation/InternalAggregationProfileTree.java new file mode 100644 index 00000000000..f367595c84c --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/profile/aggregation/InternalAggregationProfileTree.java @@ -0,0 +1,46 @@ +/* + * 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.aggregation; + +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.AggregatorFactory.MultiBucketAggregatorWrapper; +import org.elasticsearch.search.profile.AbstractInternalProfileTree; + +public class InternalAggregationProfileTree extends AbstractInternalProfileTree { + + @Override + protected AggregationProfileBreakdown createProfileBreakdown() { + return new AggregationProfileBreakdown(); + } + + @Override + protected String getTypeFromElement(Aggregator element) { + if (element instanceof MultiBucketAggregatorWrapper) { + return ((MultiBucketAggregatorWrapper) element).getWrappedClass().getName(); + } + return element.getClass().getName(); + } + + @Override + protected String getDescriptionFromElement(Aggregator element) { + return element.name(); + } + +} diff --git a/core/src/main/java/org/elasticsearch/search/profile/aggregation/ProfilingAggregator.java b/core/src/main/java/org/elasticsearch/search/profile/aggregation/ProfilingAggregator.java new file mode 100644 index 00000000000..2883c2903e8 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/profile/aggregation/ProfilingAggregator.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.aggregation; + +import org.apache.lucene.index.LeafReaderContext; +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.LeafBucketCollector; +import org.elasticsearch.search.aggregations.support.AggregationContext; + +import java.io.IOException; + +public class ProfilingAggregator extends Aggregator { + + private final Aggregator delegate; + private final AggregationProfiler profiler; + private AggregationProfileBreakdown profileBreakdown; + + public ProfilingAggregator(Aggregator delegate, AggregationProfiler profiler) throws IOException { + this.profiler = profiler; + this.delegate = delegate; + } + + @Override + public void close() { + delegate.close(); + } + + @Override + public boolean needsScores() { + return delegate.needsScores(); + } + + @Override + public String name() { + return delegate.name(); + } + + @Override + public AggregationContext context() { + return delegate.context(); + } + + @Override + public Aggregator parent() { + return delegate.parent(); + } + + @Override + public Aggregator subAggregator(String name) { + return delegate.subAggregator(name); + } + + @Override + public InternalAggregation buildAggregation(long bucket) throws IOException { + profileBreakdown.startTime(AggregationTimingType.BUILD_AGGREGATION); + InternalAggregation result = delegate.buildAggregation(bucket); + profileBreakdown.stopAndRecordTime(); + return result; + } + + @Override + public InternalAggregation buildEmptyAggregation() { + return delegate.buildEmptyAggregation(); + } + + @Override + public LeafBucketCollector getLeafCollector(LeafReaderContext ctx) throws IOException { + return new ProfilingLeafBucketCollector(delegate.getLeafCollector(ctx), profileBreakdown); + } + + @Override + public void preCollection() throws IOException { + this.profileBreakdown = profiler.getQueryBreakdown(delegate); + profileBreakdown.startTime(AggregationTimingType.INITIALIZE); + delegate.preCollection(); + profileBreakdown.stopAndRecordTime(); + profiler.pollLastElement(); + } + + @Override + public void postCollection() throws IOException { + delegate.postCollection(); + } + +} diff --git a/core/src/main/java/org/elasticsearch/search/profile/aggregation/ProfilingLeafBucketCollector.java b/core/src/main/java/org/elasticsearch/search/profile/aggregation/ProfilingLeafBucketCollector.java new file mode 100644 index 00000000000..75c90ded709 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/profile/aggregation/ProfilingLeafBucketCollector.java @@ -0,0 +1,43 @@ +/* + * 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.aggregation; + +import org.elasticsearch.search.aggregations.LeafBucketCollector; + +import java.io.IOException; + +public class ProfilingLeafBucketCollector extends LeafBucketCollector { + + private LeafBucketCollector delegate; + private AggregationProfileBreakdown profileBreakdown; + + public ProfilingLeafBucketCollector(LeafBucketCollector delegate, AggregationProfileBreakdown profileBreakdown) { + this.delegate = delegate; + this.profileBreakdown = profileBreakdown; + } + + @Override + public void collect(int doc, long bucket) throws IOException { + profileBreakdown.startTime(AggregationTimingType.COLLECT); + delegate.collect(doc, bucket); + profileBreakdown.stopAndRecordTime(); + } + +} diff --git a/core/src/main/java/org/elasticsearch/search/profile/query/InternalQueryProfileTree.java b/core/src/main/java/org/elasticsearch/search/profile/query/InternalQueryProfileTree.java index 5b92ef8b2a9..013b7d3a506 100644 --- a/core/src/main/java/org/elasticsearch/search/profile/query/InternalQueryProfileTree.java +++ b/core/src/main/java/org/elasticsearch/search/profile/query/InternalQueryProfileTree.java @@ -20,89 +20,33 @@ package org.elasticsearch.search.profile.query; import org.apache.lucene.search.Query; +import org.elasticsearch.search.profile.AbstractInternalProfileTree; import org.elasticsearch.search.profile.ProfileResult; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Deque; -import java.util.List; -import java.util.Map; -import java.util.concurrent.LinkedBlockingDeque; - /** * This class tracks the dependency tree for queries (scoring and rewriting) and * generates {@link QueryProfileBreakdown} 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 InternalQueryProfileTree { - - 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; +final class InternalQueryProfileTree extends AbstractInternalProfileTree { /** 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 InternalQueryProfileTree() { - timings = new ArrayList<>(10); - stack = new LinkedBlockingDeque<>(10); - tree = new ArrayList<>(10); - queries = new ArrayList<>(10); - roots = new ArrayList<>(10); + @Override + protected QueryProfileBreakdown createProfileBreakdown() { + return new QueryProfileBreakdown(); } - /** - * Returns a {@link QueryProfileBreakdown} 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 QueryProfileBreakdown getQueryBreakdown(Query query) { - int token = currentToken; + @Override + protected String getTypeFromElement(Query query) { + return query.getClass().getSimpleName(); + } - 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); + @Override + protected String getDescriptionFromElement(Query query) { + return query.toString(); } /** @@ -128,113 +72,7 @@ final class InternalQueryProfileTree { 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 QueryProfileBreakdown} 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 QueryProfileBreakdown 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); - - QueryProfileBreakdown queryTimings = new QueryProfileBreakdown(); - 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); - QueryProfileBreakdown 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/query/QueryProfileShardResult.java b/core/src/main/java/org/elasticsearch/search/profile/query/QueryProfileShardResult.java new file mode 100644 index 00000000000..d5e00aca336 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/profile/query/QueryProfileShardResult.java @@ -0,0 +1,104 @@ +/* + * 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.query; + +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 org.elasticsearch.search.profile.ProfileResult; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * 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 QueryProfileShardResult implements Writeable, ToXContent { + + private final List queryProfileResults; + + private final CollectorResult profileCollector; + + private final long rewriteTime; + + public QueryProfileShardResult(List queryProfileResults, long rewriteTime, + CollectorResult profileCollector) { + assert(profileCollector != null); + this.queryProfileResults = queryProfileResults; + this.profileCollector = profileCollector; + this.rewriteTime = rewriteTime; + } + + /** + * Read from a stream. + */ + public QueryProfileShardResult(StreamInput in) throws IOException { + int profileSize = in.readVInt(); + queryProfileResults = new ArrayList<>(profileSize); + for (int j = 0; j < profileSize; j++) { + queryProfileResults.add(new ProfileResult(in)); + } + + profileCollector = new CollectorResult(in); + rewriteTime = in.readLong(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(queryProfileResults.size()); + for (ProfileResult p : queryProfileResults) { + p.writeTo(out); + } + profileCollector.writeTo(out); + out.writeLong(rewriteTime); + } + + + public List getQueryResults() { + return Collections.unmodifiableList(queryProfileResults); + } + + 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 : queryProfileResults) { + p.toXContent(builder, params); + } + builder.endArray(); + builder.field("rewrite_time", rewriteTime); + builder.startArray("collector"); + profileCollector.toXContent(builder, params); + builder.endArray(); + return builder; + } +} diff --git a/core/src/main/java/org/elasticsearch/search/profile/query/QueryProfiler.java b/core/src/main/java/org/elasticsearch/search/profile/query/QueryProfiler.java index 57341ee132f..0051356e35a 100644 --- a/core/src/main/java/org/elasticsearch/search/profile/query/QueryProfiler.java +++ b/core/src/main/java/org/elasticsearch/search/profile/query/QueryProfiler.java @@ -20,9 +20,8 @@ package org.elasticsearch.search.profile.query; import org.apache.lucene.search.Query; -import org.elasticsearch.search.profile.ProfileResult; +import org.elasticsearch.search.profile.AbstractProfiler; -import java.util.List; import java.util.Objects; /** @@ -36,16 +35,16 @@ import java.util.Objects; * request may execute two searches (query + global agg). A Profiler just * represents one of those */ -public final class QueryProfiler { - - private final InternalQueryProfileTree queryTree = new InternalQueryProfileTree(); +public final class QueryProfiler extends AbstractProfiler { /** * The root Collector used in the search */ private InternalProfileCollector collector; - public QueryProfiler() {} + public QueryProfiler() { + super(new InternalQueryProfileTree()); + } /** Set the collector that is associated with this profiler. */ public void setCollector(InternalProfileCollector collector) { @@ -55,21 +54,12 @@ public final class QueryProfiler { this.collector = Objects.requireNonNull(collector); } - /** - * Get the {@link QueryProfileBreakdown} 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 QueryProfileBreakdown 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(); + ((InternalQueryProfileTree) profileTree).startRewriteTime(); } /** @@ -79,29 +69,14 @@ public final class QueryProfiler { * @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 ((InternalQueryProfileTree) profileTree).stopAndAddRewriteTime(); } /** * @return total time taken to rewrite all queries in this profile */ public long getRewriteTime() { - return queryTree.getRewriteTime(); + return ((InternalQueryProfileTree) profileTree).getRewriteTime(); } /** 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 df68064f617..174a337f3d2 100644 --- a/core/src/main/java/org/elasticsearch/search/query/QueryPhase.java +++ b/core/src/main/java/org/elasticsearch/search/query/QueryPhase.java @@ -112,8 +112,8 @@ public class QueryPhase implements SearchPhase { aggregationPhase.execute(searchContext); if (searchContext.getProfilers() != null) { - List shardResults = SearchProfileShardResults - .buildShardResults(searchContext.getProfilers().getQueryProfilers()); + ProfileShardResult shardResults = SearchProfileShardResults + .buildShardResults(searchContext.getProfilers()); searchContext.queryResult().profileResults(shardResults); } } @@ -385,8 +385,8 @@ public class QueryPhase implements SearchPhase { queryResult.topDocs(topDocsCallable.call(), sortValueFormats); if (searchContext.getProfilers() != null) { - List shardResults = SearchProfileShardResults - .buildShardResults(searchContext.getProfilers().getQueryProfilers()); + ProfileShardResult shardResults = SearchProfileShardResults + .buildShardResults(searchContext.getProfilers()); searchContext.queryResult().profileResults(shardResults); } 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 1408ebe8359..be8c895eecd 100644 --- a/core/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java +++ b/core/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java @@ -59,7 +59,7 @@ public class QuerySearchResult extends QuerySearchResultProvider { private Suggest suggest; private boolean searchTimedOut; private Boolean terminatedEarly = null; - private List profileShardResults; + private ProfileShardResult profileShardResults; public QuerySearchResult() { @@ -143,7 +143,7 @@ public class QuerySearchResult extends QuerySearchResultProvider { * Returns the profiled results for this search, or potentially null if result was empty * @return The profiled results, or null */ - public @Nullable List profileResults() { + public @Nullable ProfileShardResult profileResults() { return profileShardResults; } @@ -151,7 +151,7 @@ public class QuerySearchResult extends QuerySearchResultProvider { * Sets the finalized profiling results for this query * @param shardResults The finalized profile */ - public void profileResults(List shardResults) { + public void profileResults(ProfileShardResult shardResults) { this.profileShardResults = shardResults; } @@ -237,12 +237,7 @@ public class QuerySearchResult extends QuerySearchResultProvider { 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); - } + profileShardResults = new ProfileShardResult(in); } } @@ -296,10 +291,7 @@ public class QuerySearchResult extends QuerySearchResultProvider { out.writeBoolean(false); } else { out.writeBoolean(true); - out.writeVInt(profileShardResults.size()); - for (ProfileShardResult shardResult : profileShardResults) { - shardResult.writeTo(out); - } + profileShardResults.writeTo(out); } } } diff --git a/core/src/test/java/org/elasticsearch/search/profile/aggregation/AggregationProfilerIT.java b/core/src/test/java/org/elasticsearch/search/profile/aggregation/AggregationProfilerIT.java new file mode 100644 index 00000000000..848b230b3fa --- /dev/null +++ b/core/src/test/java/org/elasticsearch/search/profile/aggregation/AggregationProfilerIT.java @@ -0,0 +1,403 @@ +/* + * 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.aggregation; + +import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregator; +import org.elasticsearch.search.aggregations.bucket.terms.GlobalOrdinalsStringTermsAggregator; +import org.elasticsearch.search.aggregations.metrics.avg.AvgAggregator; +import org.elasticsearch.search.aggregations.metrics.max.MaxAggregator; +import org.elasticsearch.search.profile.ProfileResult; +import org.elasticsearch.search.profile.ProfileShardResult; +import org.elasticsearch.search.profile.aggregation.AggregationProfileShardResult; +import org.elasticsearch.search.profile.aggregation.AggregationTimingType; +import org.elasticsearch.test.ESIntegTestCase; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse; +import static org.elasticsearch.search.aggregations.AggregationBuilders.avg; +import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram; +import static org.elasticsearch.search.aggregations.AggregationBuilders.max; +import static org.elasticsearch.search.aggregations.AggregationBuilders.terms; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.notNullValue; + +@ESIntegTestCase.SuiteScopeTestCase +public class AggregationProfilerIT extends ESIntegTestCase { + + + private static final String NUMBER_FIELD = "number"; + private static final String TAG_FIELD = "tag"; + private static final String STRING_FIELD = "string_field"; + + @Override + protected int numberOfShards() { + return 1; + } + + @Override + protected void setupSuiteScopeCluster() throws Exception { + assertAcked(client().admin().indices().prepareCreate("idx") + .addMapping("type", STRING_FIELD, "type=keyword", NUMBER_FIELD, "type=integer", TAG_FIELD, "type=keyword").get()); + List builders = new ArrayList<>(); + + String[] randomStrings = new String[randomIntBetween(2, 10)]; + for (int i = 0; i < randomStrings.length; i++) { + randomStrings[i] = randomAsciiOfLength(10); + } + + for (int i = 0; i < 5; i++) { + builders.add(client().prepareIndex("idx", "type").setSource( + jsonBuilder().startObject() + .field(STRING_FIELD, randomFrom(randomStrings)) + .field(NUMBER_FIELD, randomIntBetween(0, 9)) + .field(TAG_FIELD, randomBoolean() ? "more" : "less") + .endObject())); + } + + indexRandom(true, builders); + createIndex("idx_unmapped"); + ensureSearchable(); + } + + public void testSimpleProfile() { + SearchResponse response = client().prepareSearch("idx").setProfile(true) + .addAggregation(histogram("histo").field(NUMBER_FIELD).interval(1L)).get(); + assertSearchResponse(response); + Map profileResults = response.getProfileResults(); + assertThat(profileResults, notNullValue()); + assertThat(profileResults.size(), equalTo(getNumShards("idx").numPrimaries)); + for (ProfileShardResult profileShardResult : profileResults.values()) { + assertThat(profileShardResult, notNullValue()); + AggregationProfileShardResult aggProfileResults = profileShardResult.getAggregationProfileResults(); + assertThat(aggProfileResults, notNullValue()); + List aggProfileResultsList = aggProfileResults.getProfileResults(); + assertThat(aggProfileResultsList, notNullValue()); + assertThat(aggProfileResultsList.size(), equalTo(1)); + ProfileResult histoAggResult = aggProfileResultsList.get(0); + assertThat(histoAggResult, notNullValue()); + assertThat(histoAggResult.getQueryName(), equalTo(HistogramAggregator.class.getName())); + assertThat(histoAggResult.getLuceneDescription(), equalTo("histo")); + assertThat(histoAggResult.getProfiledChildren().size(), equalTo(0)); + assertThat(histoAggResult.getTime(), greaterThan(0L)); + Map breakdown = histoAggResult.getTimeBreakdown(); + assertThat(breakdown, notNullValue()); + assertThat(breakdown.get(AggregationTimingType.INITIALIZE.toString()), notNullValue()); + assertThat(breakdown.get(AggregationTimingType.INITIALIZE.toString()), greaterThan(0L)); + assertThat(breakdown.get(AggregationTimingType.COLLECT.toString()), notNullValue()); + assertThat(breakdown.get(AggregationTimingType.COLLECT.toString()), greaterThan(0L)); + assertThat(breakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), notNullValue()); + assertThat(breakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), greaterThan(0L)); + assertThat(breakdown.get(AggregationTimingType.REDUCE.toString()), notNullValue()); + assertThat(breakdown.get(AggregationTimingType.REDUCE.toString()), equalTo(0L)); + + } + } + + public void testMultiLevelProfile() { + SearchResponse response = client().prepareSearch("idx").setProfile(true) + .addAggregation(histogram("histo").field(NUMBER_FIELD).interval(1L) + .subAggregation(terms("terms").field(TAG_FIELD) + .subAggregation(avg("avg").field(NUMBER_FIELD)))).get(); + assertSearchResponse(response); + Map profileResults = response.getProfileResults(); + assertThat(profileResults, notNullValue()); + assertThat(profileResults.size(), equalTo(getNumShards("idx").numPrimaries)); + for (ProfileShardResult profileShardResult : profileResults.values()) { + assertThat(profileShardResult, notNullValue()); + AggregationProfileShardResult aggProfileResults = profileShardResult.getAggregationProfileResults(); + assertThat(aggProfileResults, notNullValue()); + List aggProfileResultsList = aggProfileResults.getProfileResults(); + assertThat(aggProfileResultsList, notNullValue()); + assertThat(aggProfileResultsList.size(), equalTo(1)); + ProfileResult histoAggResult = aggProfileResultsList.get(0); + assertThat(histoAggResult, notNullValue()); + assertThat(histoAggResult.getQueryName(), equalTo(HistogramAggregator.class.getName())); + assertThat(histoAggResult.getLuceneDescription(), equalTo("histo")); + assertThat(histoAggResult.getTime(), greaterThan(0L)); + Map histoBreakdown = histoAggResult.getTimeBreakdown(); + assertThat(histoBreakdown, notNullValue()); + assertThat(histoBreakdown.get(AggregationTimingType.INITIALIZE.toString()), notNullValue()); + assertThat(histoBreakdown.get(AggregationTimingType.INITIALIZE.toString()), greaterThan(0L)); + assertThat(histoBreakdown.get(AggregationTimingType.COLLECT.toString()), notNullValue()); + assertThat(histoBreakdown.get(AggregationTimingType.COLLECT.toString()), greaterThan(0L)); + assertThat(histoBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), notNullValue()); + assertThat(histoBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), greaterThan(0L)); + assertThat(histoBreakdown.get(AggregationTimingType.REDUCE.toString()), notNullValue()); + assertThat(histoBreakdown.get(AggregationTimingType.REDUCE.toString()), equalTo(0L)); + assertThat(histoAggResult.getProfiledChildren().size(), equalTo(1)); + + ProfileResult termsAggResult = histoAggResult.getProfiledChildren().get(0); + assertThat(termsAggResult, notNullValue()); + assertThat(termsAggResult.getQueryName(), equalTo(GlobalOrdinalsStringTermsAggregator.WithHash.class.getName())); + assertThat(termsAggResult.getLuceneDescription(), equalTo("terms")); + assertThat(termsAggResult.getTime(), greaterThan(0L)); + Map termsBreakdown = termsAggResult.getTimeBreakdown(); + assertThat(termsBreakdown, notNullValue()); + assertThat(termsBreakdown.get(AggregationTimingType.INITIALIZE.toString()), notNullValue()); + assertThat(termsBreakdown.get(AggregationTimingType.INITIALIZE.toString()), greaterThan(0L)); + assertThat(termsBreakdown.get(AggregationTimingType.COLLECT.toString()), notNullValue()); + assertThat(termsBreakdown.get(AggregationTimingType.COLLECT.toString()), greaterThan(0L)); + assertThat(termsBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), notNullValue()); + assertThat(termsBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), greaterThan(0L)); + assertThat(termsBreakdown.get(AggregationTimingType.REDUCE.toString()), notNullValue()); + assertThat(termsBreakdown.get(AggregationTimingType.REDUCE.toString()), equalTo(0L)); + assertThat(termsAggResult.getProfiledChildren().size(), equalTo(1)); + + ProfileResult avgAggResult = termsAggResult.getProfiledChildren().get(0); + assertThat(avgAggResult, notNullValue()); + assertThat(avgAggResult.getQueryName(), equalTo(AvgAggregator.class.getName())); + assertThat(avgAggResult.getLuceneDescription(), equalTo("avg")); + assertThat(avgAggResult.getTime(), greaterThan(0L)); + Map avgBreakdown = termsAggResult.getTimeBreakdown(); + assertThat(avgBreakdown, notNullValue()); + assertThat(avgBreakdown.get(AggregationTimingType.INITIALIZE.toString()), notNullValue()); + assertThat(avgBreakdown.get(AggregationTimingType.INITIALIZE.toString()), greaterThan(0L)); + assertThat(avgBreakdown.get(AggregationTimingType.COLLECT.toString()), notNullValue()); + assertThat(avgBreakdown.get(AggregationTimingType.COLLECT.toString()), greaterThan(0L)); + assertThat(avgBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), notNullValue()); + assertThat(avgBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), greaterThan(0L)); + assertThat(avgBreakdown.get(AggregationTimingType.REDUCE.toString()), notNullValue()); + assertThat(avgBreakdown.get(AggregationTimingType.REDUCE.toString()), equalTo(0L)); + assertThat(avgAggResult.getProfiledChildren().size(), equalTo(0)); + } + } + + public void testComplexProfile() { + SearchResponse response = client().prepareSearch("idx").setProfile(true) + .addAggregation(histogram("histo").field(NUMBER_FIELD).interval(1L) + .subAggregation(terms("tags").field(TAG_FIELD) + .subAggregation(avg("avg").field(NUMBER_FIELD)) + .subAggregation(max("max").field(NUMBER_FIELD))) + .subAggregation(terms("strings").field(STRING_FIELD) + .subAggregation(avg("avg").field(NUMBER_FIELD)) + .subAggregation(max("max").field(NUMBER_FIELD)) + .subAggregation(terms("tags").field(TAG_FIELD) + .subAggregation(avg("avg").field(NUMBER_FIELD)) + .subAggregation(max("max").field(NUMBER_FIELD))))) + .get(); + assertSearchResponse(response); + Map profileResults = response.getProfileResults(); + assertThat(profileResults, notNullValue()); + assertThat(profileResults.size(), equalTo(getNumShards("idx").numPrimaries)); + for (ProfileShardResult profileShardResult : profileResults.values()) { + assertThat(profileShardResult, notNullValue()); + AggregationProfileShardResult aggProfileResults = profileShardResult.getAggregationProfileResults(); + assertThat(aggProfileResults, notNullValue()); + List aggProfileResultsList = aggProfileResults.getProfileResults(); + assertThat(aggProfileResultsList, notNullValue()); + assertThat(aggProfileResultsList.size(), equalTo(1)); + ProfileResult histoAggResult = aggProfileResultsList.get(0); + assertThat(histoAggResult, notNullValue()); + assertThat(histoAggResult.getQueryName(), equalTo(HistogramAggregator.class.getName())); + assertThat(histoAggResult.getLuceneDescription(), equalTo("histo")); + assertThat(histoAggResult.getTime(), greaterThan(0L)); + Map histoBreakdown = histoAggResult.getTimeBreakdown(); + assertThat(histoBreakdown, notNullValue()); + assertThat(histoBreakdown.get(AggregationTimingType.INITIALIZE.toString()), notNullValue()); + assertThat(histoBreakdown.get(AggregationTimingType.INITIALIZE.toString()), greaterThan(0L)); + assertThat(histoBreakdown.get(AggregationTimingType.COLLECT.toString()), notNullValue()); + assertThat(histoBreakdown.get(AggregationTimingType.COLLECT.toString()), greaterThan(0L)); + assertThat(histoBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), notNullValue()); + assertThat(histoBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), greaterThan(0L)); + assertThat(histoBreakdown.get(AggregationTimingType.REDUCE.toString()), notNullValue()); + assertThat(histoBreakdown.get(AggregationTimingType.REDUCE.toString()), equalTo(0L)); + assertThat(histoAggResult.getProfiledChildren().size(), equalTo(2)); + + ProfileResult tagsAggResult = histoAggResult.getProfiledChildren().get(0); + assertThat(tagsAggResult, notNullValue()); + assertThat(tagsAggResult.getQueryName(), equalTo(GlobalOrdinalsStringTermsAggregator.WithHash.class.getName())); + assertThat(tagsAggResult.getLuceneDescription(), equalTo("tags")); + assertThat(tagsAggResult.getTime(), greaterThan(0L)); + Map tagsBreakdown = tagsAggResult.getTimeBreakdown(); + assertThat(tagsBreakdown, notNullValue()); + assertThat(tagsBreakdown.get(AggregationTimingType.INITIALIZE.toString()), notNullValue()); + assertThat(tagsBreakdown.get(AggregationTimingType.INITIALIZE.toString()), greaterThan(0L)); + assertThat(tagsBreakdown.get(AggregationTimingType.COLLECT.toString()), notNullValue()); + assertThat(tagsBreakdown.get(AggregationTimingType.COLLECT.toString()), greaterThan(0L)); + assertThat(tagsBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), notNullValue()); + assertThat(tagsBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), greaterThan(0L)); + assertThat(tagsBreakdown.get(AggregationTimingType.REDUCE.toString()), notNullValue()); + assertThat(tagsBreakdown.get(AggregationTimingType.REDUCE.toString()), equalTo(0L)); + assertThat(tagsAggResult.getProfiledChildren().size(), equalTo(2)); + + ProfileResult avgAggResult = tagsAggResult.getProfiledChildren().get(0); + assertThat(avgAggResult, notNullValue()); + assertThat(avgAggResult.getQueryName(), equalTo(AvgAggregator.class.getName())); + assertThat(avgAggResult.getLuceneDescription(), equalTo("avg")); + assertThat(avgAggResult.getTime(), greaterThan(0L)); + Map avgBreakdown = tagsAggResult.getTimeBreakdown(); + assertThat(avgBreakdown, notNullValue()); + assertThat(avgBreakdown.get(AggregationTimingType.INITIALIZE.toString()), notNullValue()); + assertThat(avgBreakdown.get(AggregationTimingType.INITIALIZE.toString()), greaterThan(0L)); + assertThat(avgBreakdown.get(AggregationTimingType.COLLECT.toString()), notNullValue()); + assertThat(avgBreakdown.get(AggregationTimingType.COLLECT.toString()), greaterThan(0L)); + assertThat(avgBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), notNullValue()); + assertThat(avgBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), greaterThan(0L)); + assertThat(avgBreakdown.get(AggregationTimingType.REDUCE.toString()), notNullValue()); + assertThat(avgBreakdown.get(AggregationTimingType.REDUCE.toString()), equalTo(0L)); + assertThat(avgAggResult.getProfiledChildren().size(), equalTo(0)); + + ProfileResult maxAggResult = tagsAggResult.getProfiledChildren().get(1); + assertThat(maxAggResult, notNullValue()); + assertThat(maxAggResult.getQueryName(), equalTo(MaxAggregator.class.getName())); + assertThat(maxAggResult.getLuceneDescription(), equalTo("max")); + assertThat(maxAggResult.getTime(), greaterThan(0L)); + Map maxBreakdown = tagsAggResult.getTimeBreakdown(); + assertThat(maxBreakdown, notNullValue()); + assertThat(maxBreakdown.get(AggregationTimingType.INITIALIZE.toString()), notNullValue()); + assertThat(maxBreakdown.get(AggregationTimingType.INITIALIZE.toString()), greaterThan(0L)); + assertThat(maxBreakdown.get(AggregationTimingType.COLLECT.toString()), notNullValue()); + assertThat(maxBreakdown.get(AggregationTimingType.COLLECT.toString()), greaterThan(0L)); + assertThat(maxBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), notNullValue()); + assertThat(maxBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), greaterThan(0L)); + assertThat(maxBreakdown.get(AggregationTimingType.REDUCE.toString()), notNullValue()); + assertThat(maxBreakdown.get(AggregationTimingType.REDUCE.toString()), equalTo(0L)); + assertThat(maxAggResult.getProfiledChildren().size(), equalTo(0)); + + ProfileResult stringsAggResult = histoAggResult.getProfiledChildren().get(1); + assertThat(stringsAggResult, notNullValue()); + assertThat(stringsAggResult.getQueryName(), equalTo(GlobalOrdinalsStringTermsAggregator.WithHash.class.getName())); + assertThat(stringsAggResult.getLuceneDescription(), equalTo("strings")); + assertThat(stringsAggResult.getTime(), greaterThan(0L)); + Map stringsBreakdown = stringsAggResult.getTimeBreakdown(); + assertThat(stringsBreakdown, notNullValue()); + assertThat(stringsBreakdown.get(AggregationTimingType.INITIALIZE.toString()), notNullValue()); + assertThat(stringsBreakdown.get(AggregationTimingType.INITIALIZE.toString()), greaterThan(0L)); + assertThat(stringsBreakdown.get(AggregationTimingType.COLLECT.toString()), notNullValue()); + assertThat(stringsBreakdown.get(AggregationTimingType.COLLECT.toString()), greaterThan(0L)); + assertThat(stringsBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), notNullValue()); + assertThat(stringsBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), greaterThan(0L)); + assertThat(stringsBreakdown.get(AggregationTimingType.REDUCE.toString()), notNullValue()); + assertThat(stringsBreakdown.get(AggregationTimingType.REDUCE.toString()), equalTo(0L)); + assertThat(stringsAggResult.getProfiledChildren().size(), equalTo(3)); + + avgAggResult = stringsAggResult.getProfiledChildren().get(0); + assertThat(avgAggResult, notNullValue()); + assertThat(avgAggResult.getQueryName(), equalTo(AvgAggregator.class.getName())); + assertThat(avgAggResult.getLuceneDescription(), equalTo("avg")); + assertThat(avgAggResult.getTime(), greaterThan(0L)); + avgBreakdown = stringsAggResult.getTimeBreakdown(); + assertThat(avgBreakdown, notNullValue()); + assertThat(avgBreakdown.get(AggregationTimingType.INITIALIZE.toString()), notNullValue()); + assertThat(avgBreakdown.get(AggregationTimingType.INITIALIZE.toString()), greaterThan(0L)); + assertThat(avgBreakdown.get(AggregationTimingType.COLLECT.toString()), notNullValue()); + assertThat(avgBreakdown.get(AggregationTimingType.COLLECT.toString()), greaterThan(0L)); + assertThat(avgBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), notNullValue()); + assertThat(avgBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), greaterThan(0L)); + assertThat(avgBreakdown.get(AggregationTimingType.REDUCE.toString()), notNullValue()); + assertThat(avgBreakdown.get(AggregationTimingType.REDUCE.toString()), equalTo(0L)); + assertThat(avgAggResult.getProfiledChildren().size(), equalTo(0)); + + maxAggResult = stringsAggResult.getProfiledChildren().get(1); + assertThat(maxAggResult, notNullValue()); + assertThat(maxAggResult.getQueryName(), equalTo(MaxAggregator.class.getName())); + assertThat(maxAggResult.getLuceneDescription(), equalTo("max")); + assertThat(maxAggResult.getTime(), greaterThan(0L)); + maxBreakdown = stringsAggResult.getTimeBreakdown(); + assertThat(maxBreakdown, notNullValue()); + assertThat(maxBreakdown.get(AggregationTimingType.INITIALIZE.toString()), notNullValue()); + assertThat(maxBreakdown.get(AggregationTimingType.INITIALIZE.toString()), greaterThan(0L)); + assertThat(maxBreakdown.get(AggregationTimingType.COLLECT.toString()), notNullValue()); + assertThat(maxBreakdown.get(AggregationTimingType.COLLECT.toString()), greaterThan(0L)); + assertThat(maxBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), notNullValue()); + assertThat(maxBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), greaterThan(0L)); + assertThat(maxBreakdown.get(AggregationTimingType.REDUCE.toString()), notNullValue()); + assertThat(maxBreakdown.get(AggregationTimingType.REDUCE.toString()), equalTo(0L)); + assertThat(maxAggResult.getProfiledChildren().size(), equalTo(0)); + + tagsAggResult = stringsAggResult.getProfiledChildren().get(2); + assertThat(tagsAggResult, notNullValue()); + assertThat(tagsAggResult.getQueryName(), equalTo(GlobalOrdinalsStringTermsAggregator.WithHash.class.getName())); + assertThat(tagsAggResult.getLuceneDescription(), equalTo("tags")); + assertThat(tagsAggResult.getTime(), greaterThan(0L)); + tagsBreakdown = tagsAggResult.getTimeBreakdown(); + assertThat(tagsBreakdown, notNullValue()); + assertThat(tagsBreakdown.get(AggregationTimingType.INITIALIZE.toString()), notNullValue()); + assertThat(tagsBreakdown.get(AggregationTimingType.INITIALIZE.toString()), greaterThan(0L)); + assertThat(tagsBreakdown.get(AggregationTimingType.COLLECT.toString()), notNullValue()); + assertThat(tagsBreakdown.get(AggregationTimingType.COLLECT.toString()), greaterThan(0L)); + assertThat(tagsBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), notNullValue()); + assertThat(tagsBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), greaterThan(0L)); + assertThat(tagsBreakdown.get(AggregationTimingType.REDUCE.toString()), notNullValue()); + assertThat(tagsBreakdown.get(AggregationTimingType.REDUCE.toString()), equalTo(0L)); + assertThat(tagsAggResult.getProfiledChildren().size(), equalTo(2)); + + avgAggResult = tagsAggResult.getProfiledChildren().get(0); + assertThat(avgAggResult, notNullValue()); + assertThat(avgAggResult.getQueryName(), equalTo(AvgAggregator.class.getName())); + assertThat(avgAggResult.getLuceneDescription(), equalTo("avg")); + assertThat(avgAggResult.getTime(), greaterThan(0L)); + avgBreakdown = tagsAggResult.getTimeBreakdown(); + assertThat(avgBreakdown, notNullValue()); + assertThat(avgBreakdown.get(AggregationTimingType.INITIALIZE.toString()), notNullValue()); + assertThat(avgBreakdown.get(AggregationTimingType.INITIALIZE.toString()), greaterThan(0L)); + assertThat(avgBreakdown.get(AggregationTimingType.COLLECT.toString()), notNullValue()); + assertThat(avgBreakdown.get(AggregationTimingType.COLLECT.toString()), greaterThan(0L)); + assertThat(avgBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), notNullValue()); + assertThat(avgBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), greaterThan(0L)); + assertThat(avgBreakdown.get(AggregationTimingType.REDUCE.toString()), notNullValue()); + assertThat(avgBreakdown.get(AggregationTimingType.REDUCE.toString()), equalTo(0L)); + assertThat(avgAggResult.getProfiledChildren().size(), equalTo(0)); + + maxAggResult = tagsAggResult.getProfiledChildren().get(1); + assertThat(maxAggResult, notNullValue()); + assertThat(maxAggResult.getQueryName(), equalTo(MaxAggregator.class.getName())); + assertThat(maxAggResult.getLuceneDescription(), equalTo("max")); + assertThat(maxAggResult.getTime(), greaterThan(0L)); + maxBreakdown = tagsAggResult.getTimeBreakdown(); + assertThat(maxBreakdown, notNullValue()); + assertThat(maxBreakdown.get(AggregationTimingType.INITIALIZE.toString()), notNullValue()); + assertThat(maxBreakdown.get(AggregationTimingType.INITIALIZE.toString()), greaterThan(0L)); + assertThat(maxBreakdown.get(AggregationTimingType.COLLECT.toString()), notNullValue()); + assertThat(maxBreakdown.get(AggregationTimingType.COLLECT.toString()), greaterThan(0L)); + assertThat(maxBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), notNullValue()); + assertThat(maxBreakdown.get(AggregationTimingType.BUILD_AGGREGATION.toString()), greaterThan(0L)); + assertThat(maxBreakdown.get(AggregationTimingType.REDUCE.toString()), notNullValue()); + assertThat(maxBreakdown.get(AggregationTimingType.REDUCE.toString()), equalTo(0L)); + assertThat(maxAggResult.getProfiledChildren().size(), equalTo(0)); + } + } + + public void testNoProfile() { + SearchResponse response = client().prepareSearch("idx").setProfile(false) + .addAggregation(histogram("histo").field(NUMBER_FIELD).interval(1L) + .subAggregation(terms("tags").field(TAG_FIELD) + .subAggregation(avg("avg").field(NUMBER_FIELD)) + .subAggregation(max("max").field(NUMBER_FIELD))) + .subAggregation(terms("strings").field(STRING_FIELD) + .subAggregation(avg("avg").field(NUMBER_FIELD)) + .subAggregation(max("max").field(NUMBER_FIELD)) + .subAggregation(terms("tags").field(TAG_FIELD) + .subAggregation(avg("avg").field(NUMBER_FIELD)) + .subAggregation(max("max").field(NUMBER_FIELD))))) + .get(); + assertSearchResponse(response); + Map profileResults = response.getProfileResults(); + assertThat(profileResults, notNullValue()); + assertThat(profileResults.size(), equalTo(0)); + } +} diff --git a/core/src/test/java/org/elasticsearch/search/profile/query/QueryProfilerIT.java b/core/src/test/java/org/elasticsearch/search/profile/query/QueryProfilerIT.java index 371aaadd3a7..b6935f021d4 100644 --- a/core/src/test/java/org/elasticsearch/search/profile/query/QueryProfilerIT.java +++ b/core/src/test/java/org/elasticsearch/search/profile/query/QueryProfilerIT.java @@ -85,8 +85,8 @@ public class QueryProfilerIT extends ESIntegTestCase { assertNotNull("Profile response element should not be null", resp.getProfileResults()); assertThat("Profile response should not be an empty array", resp.getProfileResults().size(), not(0)); - for (Map.Entry> shard : resp.getProfileResults().entrySet()) { - for (ProfileShardResult searchProfiles : shard.getValue()) { + for (Map.Entry shard : resp.getProfileResults().entrySet()) { + for (QueryProfileShardResult searchProfiles : shard.getValue().getQueryProfileResults()) { for (ProfileResult result : searchProfiles.getQueryResults()) { assertNotNull(result.getQueryName()); assertNotNull(result.getLuceneDescription()); @@ -163,8 +163,9 @@ public class QueryProfilerIT extends ESIntegTestCase { nearlyEqual(vanillaMaxScore, profileMaxScore, 0.001)); } - assertThat("Profile totalHits of [" + profileResponse.getHits().totalHits() + "] is not close to Vanilla totalHits [" - + vanillaResponse.getHits().totalHits() + "]", + 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(); @@ -203,12 +204,12 @@ public class QueryProfilerIT extends ESIntegTestCase { .setSearchType(SearchType.QUERY_THEN_FETCH) .execute().actionGet(); - Map> p = resp.getProfileResults(); + Map p = resp.getProfileResults(); assertNotNull(p); assertThat("Profile response should not be an empty array", resp.getProfileResults().size(), not(0)); - for (Map.Entry> shardResult : resp.getProfileResults().entrySet()) { - for (ProfileShardResult searchProfiles : shardResult.getValue()) { + for (Map.Entry shardResult : resp.getProfileResults().entrySet()) { + for (QueryProfileShardResult searchProfiles : shardResult.getValue().getQueryProfileResults()) { for (ProfileResult result : searchProfiles.getQueryResults()) { assertEquals(result.getQueryName(), "TermQuery"); assertEquals(result.getLuceneDescription(), "field1:one"); @@ -250,12 +251,12 @@ public class QueryProfilerIT extends ESIntegTestCase { .setSearchType(SearchType.QUERY_THEN_FETCH) .execute().actionGet(); - Map> p = resp.getProfileResults(); + Map p = resp.getProfileResults(); assertNotNull(p); assertThat("Profile response should not be an empty array", resp.getProfileResults().size(), not(0)); - for (Map.Entry> shardResult : resp.getProfileResults().entrySet()) { - for (ProfileShardResult searchProfiles : shardResult.getValue()) { + for (Map.Entry shardResult : resp.getProfileResults().entrySet()) { + for (QueryProfileShardResult searchProfiles : shardResult.getValue().getQueryProfileResults()) { for (ProfileResult result : searchProfiles.getQueryResults()) { assertEquals(result.getQueryName(), "BooleanQuery"); assertEquals(result.getLuceneDescription(), "+field1:one +field1:two"); @@ -322,8 +323,8 @@ public class QueryProfilerIT extends ESIntegTestCase { assertNotNull("Profile response element should not be null", resp.getProfileResults()); assertThat("Profile response should not be an empty array", resp.getProfileResults().size(), not(0)); - for (Map.Entry> shardResult : resp.getProfileResults().entrySet()) { - for (ProfileShardResult searchProfiles : shardResult.getValue()) { + for (Map.Entry shardResult : resp.getProfileResults().entrySet()) { + for (QueryProfileShardResult searchProfiles : shardResult.getValue().getQueryProfileResults()) { for (ProfileResult result : searchProfiles.getQueryResults()) { assertNotNull(result.getQueryName()); assertNotNull(result.getLuceneDescription()); @@ -374,8 +375,8 @@ public class QueryProfilerIT extends ESIntegTestCase { assertNotNull("Profile response element should not be null", resp.getProfileResults()); assertThat("Profile response should not be an empty array", resp.getProfileResults().size(), not(0)); - for (Map.Entry> shardResult : resp.getProfileResults().entrySet()) { - for (ProfileShardResult searchProfiles : shardResult.getValue()) { + for (Map.Entry shardResult : resp.getProfileResults().entrySet()) { + for (QueryProfileShardResult searchProfiles : shardResult.getValue().getQueryProfileResults()) { for (ProfileResult result : searchProfiles.getQueryResults()) { assertNotNull(result.getQueryName()); assertNotNull(result.getLuceneDescription()); @@ -421,8 +422,8 @@ public class QueryProfilerIT extends ESIntegTestCase { assertNotNull("Profile response element should not be null", resp.getProfileResults()); assertThat("Profile response should not be an empty array", resp.getProfileResults().size(), not(0)); - for (Map.Entry> shardResult : resp.getProfileResults().entrySet()) { - for (ProfileShardResult searchProfiles : shardResult.getValue()) { + for (Map.Entry shardResult : resp.getProfileResults().entrySet()) { + for (QueryProfileShardResult searchProfiles : shardResult.getValue().getQueryProfileResults()) { for (ProfileResult result : searchProfiles.getQueryResults()) { assertNotNull(result.getQueryName()); assertNotNull(result.getLuceneDescription()); @@ -468,8 +469,8 @@ public class QueryProfilerIT extends ESIntegTestCase { assertNotNull("Profile response element should not be null", resp.getProfileResults()); assertThat("Profile response should not be an empty array", resp.getProfileResults().size(), not(0)); - for (Map.Entry> shardResult : resp.getProfileResults().entrySet()) { - for (ProfileShardResult searchProfiles : shardResult.getValue()) { + for (Map.Entry shardResult : resp.getProfileResults().entrySet()) { + for (QueryProfileShardResult searchProfiles : shardResult.getValue().getQueryProfileResults()) { for (ProfileResult result : searchProfiles.getQueryResults()) { assertNotNull(result.getQueryName()); assertNotNull(result.getLuceneDescription()); @@ -514,8 +515,8 @@ public class QueryProfilerIT extends ESIntegTestCase { assertNotNull("Profile response element should not be null", resp.getProfileResults()); assertThat("Profile response should not be an empty array", resp.getProfileResults().size(), not(0)); - for (Map.Entry> shardResult : resp.getProfileResults().entrySet()) { - for (ProfileShardResult searchProfiles : shardResult.getValue()) { + for (Map.Entry shardResult : resp.getProfileResults().entrySet()) { + for (QueryProfileShardResult searchProfiles : shardResult.getValue().getQueryProfileResults()) { for (ProfileResult result : searchProfiles.getQueryResults()) { assertNotNull(result.getQueryName()); assertNotNull(result.getLuceneDescription()); @@ -569,8 +570,8 @@ public class QueryProfilerIT extends ESIntegTestCase { assertNotNull("Profile response element should not be null", resp.getProfileResults()); assertThat("Profile response should not be an empty array", resp.getProfileResults().size(), not(0)); - for (Map.Entry> shardResult : resp.getProfileResults().entrySet()) { - for (ProfileShardResult searchProfiles : shardResult.getValue()) { + for (Map.Entry shardResult : resp.getProfileResults().entrySet()) { + for (QueryProfileShardResult searchProfiles : shardResult.getValue().getQueryProfileResults()) { for (ProfileResult result : searchProfiles.getQueryResults()) { assertNotNull(result.getQueryName()); assertNotNull(result.getLuceneDescription()); diff --git a/core/src/test/java/org/elasticsearch/search/profile/query/ProfileTests.java b/core/src/test/java/org/elasticsearch/search/profile/query/QueryProfilerTests.java similarity index 96% rename from core/src/test/java/org/elasticsearch/search/profile/query/ProfileTests.java rename to core/src/test/java/org/elasticsearch/search/profile/query/QueryProfilerTests.java index 7488dbceee7..ffad39bc3f2 100644 --- a/core/src/test/java/org/elasticsearch/search/profile/query/ProfileTests.java +++ b/core/src/test/java/org/elasticsearch/search/profile/query/QueryProfilerTests.java @@ -51,7 +51,7 @@ import java.util.Map; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; -public class ProfileTests extends ESTestCase { +public class QueryProfilerTests extends ESTestCase { static Directory dir; static IndexReader reader; @@ -90,7 +90,7 @@ public class ProfileTests extends ESTestCase { searcher.setProfiler(profiler); Query query = new TermQuery(new Term("foo", "bar")); searcher.search(query, 1); - List results = profiler.getQueryTree(); + List results = profiler.getTree(); assertEquals(1, results.size()); Map breakdown = results.get(0).getTimeBreakdown(); assertThat(breakdown.get(QueryTimingType.CREATE_WEIGHT.toString()).longValue(), greaterThan(0L)); @@ -109,7 +109,7 @@ public class ProfileTests extends ESTestCase { 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(); + List results = profiler.getTree(); assertEquals(1, results.size()); Map breakdown = results.get(0).getTimeBreakdown(); assertThat(breakdown.get(QueryTimingType.CREATE_WEIGHT.toString()).longValue(), greaterThan(0L)); @@ -128,7 +128,7 @@ public class ProfileTests extends ESTestCase { searcher.setProfiler(profiler); Query query = new TermQuery(new Term("foo", "bar")); searcher.count(query); // will use index stats - List results = profiler.getQueryTree(); + List results = profiler.getTree(); assertEquals(0, results.size()); long rewriteTime = profiler.getRewriteTime(); @@ -144,7 +144,7 @@ public class ProfileTests extends ESTestCase { searcher.setProfiler(profiler); Query query = new RandomApproximationQuery(new TermQuery(new Term("foo", "bar")), random()); searcher.count(query); - List results = profiler.getQueryTree(); + List results = profiler.getTree(); assertEquals(1, results.size()); Map breakdown = results.get(0).getTimeBreakdown(); assertThat(breakdown.get(QueryTimingType.CREATE_WEIGHT.toString()).longValue(), greaterThan(0L));