From 45d04b2cd00c5798a24162ef4913d9dccb6ebe8c Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 24 Apr 2020 16:50:58 -0400 Subject: [PATCH 01/14] Save memory when numeric terms agg is not top Right now all implementations of the `terms` agg allocate a new `Aggregator` per bucket. This uses a bunch of memory. Exactly how much isn't clear but each `Aggregator` ends up making its own objects to read doc values which have non-trivial buffers. And it forces all of it sub-aggregations to do the same. We allocate a new `Aggregator` per bucket for two reasons: 1. We didn't have an appropriate data structure to track the sub-ordinals of each parent bucket. 2. You can only make a single call to `runDeferredCollections(long...)` per `Aggregator` which was the only way to delay collection of sub-aggregations. This change adds a way to return "deferred" aggregations from any bucket aggregation and undefers them as part of regular collections. This mechanism allows you to defer without `runDeferredCollections(long...)`. This change also adds a fairly simplistic data structure to track the sub-ordinals for `long`-keyed buckets. It uses both of those to power numeric `terms` aggregations and removes the per-bucket allocation of their `Aggregator`. This fairly substantially reduces memory consumption of numeric `terms` aggregations that are not the "top level", especially when those aggregations contain many sub-aggregations. I picked numeric `terms` aggregations because those have the simplest implementation. At least, I could kind of fit it in my head. And I haven't fully understood the "bytes"-based terms aggregations, but I imagine I'll be able to make similar optimizations to them in follow up changes. --- docs/reference/search/profile.asciidoc | 36 ++- .../search/aggregations/AggregationPhase.java | 3 +- .../search/aggregations/Aggregator.java | 42 ++- .../search/aggregations/AggregatorBase.java | 9 + .../aggregations/AggregatorFactory.java | 12 + .../aggregations/InternalAggregation.java | 36 +++ .../InternalMultiBucketAggregation.java | 24 +- .../aggregations/LeafBucketCollector.java | 29 +- .../bucket/BestBucketsDeferringCollector.java | 11 +- .../bucket/BucketsAggregator.java | 3 + .../bucket/DeferableBucketAggregator.java | 166 ++++++++++- .../bucket/DeferringBucketCollector.java | 23 +- .../InternalSingleBucketAggregation.java | 4 + .../MergingBucketsDeferringCollector.java | 2 +- .../bucket/composite/CompositeAggregator.java | 7 +- .../sampler/BestDocsDeferringCollector.java | 3 +- .../SignificantLongTermsAggregator.java | 19 +- .../bucket/terms/DoubleTermsAggregator.java | 11 +- .../bucket/terms/LongKeyedBucketOrds.java | 266 ++++++++++++++++++ .../bucket/terms/LongTermsAggregator.java | 64 +++-- .../bucket/terms/TermsAggregatorFactory.java | 29 +- .../bucket/terms/TermsAggregatorSupplier.java | 3 + .../aggregation/AggregationTimingType.java | 1 + .../aggregation/ProfilingAggregator.java | 11 + .../terms/LongKeyedBucketOrdsTests.java | 175 ++++++++++++ .../bucket/terms/TermsAggregatorTests.java | 31 ++ .../metrics/MaxAggregatorTests.java | 6 +- .../aggregations/AggregatorTestCase.java | 8 +- 28 files changed, 936 insertions(+), 98 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongKeyedBucketOrds.java create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/LongKeyedBucketOrdsTests.java diff --git a/docs/reference/search/profile.asciidoc b/docs/reference/search/profile.asciidoc index 561ed30a8cc74..57b8f4b437b4a 100644 --- a/docs/reference/search/profile.asciidoc +++ b/docs/reference/search/profile.asciidoc @@ -784,46 +784,52 @@ This yields the following aggregation profile output: { "type" : "LongTermsAggregator", "description" : "my_scoped_agg", - "time_in_nanos" : 195386, + "time_in_nanos" : 195421, "breakdown" : { "reduce" : 0, - "build_aggregation" : 81171, + "build_aggregation" : 80927, "build_aggregation_count" : 1, - "initialize" : 22753, + "initialize" : 3619, "initialize_count" : 1, "reduce_count" : 0, - "collect" : 91456, - "collect_count" : 4 + "collect" : 109778, + "collect_count" : 4, + "run_deferred_collections" : 1090, + "run_deferred_collections_count" : 1 } }, { "type" : "GlobalAggregator", "description" : "my_global_agg", - "time_in_nanos" : 190430, + "time_in_nanos" : 231761, "breakdown" : { "reduce" : 0, - "build_aggregation" : 59990, + "build_aggregation" : 64334, "build_aggregation_count" : 1, - "initialize" : 29619, + "initialize" : 17229, "initialize_count" : 1, "reduce_count" : 0, - "collect" : 100815, - "collect_count" : 4 + "collect" : 147490, + "collect_count" : 4, + "run_deferred_collections" : 2701, + "run_deferred_collections_count": 1 }, "children" : [ { "type" : "LongTermsAggregator", "description" : "my_level_agg", - "time_in_nanos" : 160329, + "time_in_nanos" : 198881, "breakdown" : { "reduce" : 0, - "build_aggregation" : 55712, + "build_aggregation" : 58959, "build_aggregation_count" : 1, - "initialize" : 10559, + "initialize" : 1801, "initialize_count" : 1, "reduce_count" : 0, - "collect" : 94052, - "collect_count" : 4 + "collect" : 137302, + "collect_count" : 4, + "run_deferred_collections" : 812, + "run_deferred_collections_count" : 1 } } ] diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java b/server/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java index 9f91dedf30703..168880ee77241 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java @@ -125,7 +125,7 @@ public void execute(SearchContext context) { for (Aggregator aggregator : context.aggregations().aggregators()) { try { aggregator.postCollection(); - aggregations.add(aggregator.buildAggregation(0)); + aggregations.add(aggregator.buildTopLevel()); } catch (IOException e) { throw new AggregationExecutionException("Failed to build aggregation [" + aggregator.name() + "]", e); } @@ -136,5 +136,4 @@ public void execute(SearchContext context) { context.aggregations(null); context.queryCollectors().remove(AggregationPhase.class); } - } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java index 41058ee206656..c6514226955df 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java @@ -152,15 +152,53 @@ public interface BucketComparator { } /** - * Build an aggregation for data that has been collected into {@code bucket}. + * Build an aggregation for data that has been collected into + * {@code owningBucketOrd}. + *

+ * Bucketing aggregations sometimes delay collecting their results if they + * are selective (like {@code terms}) or if they aren't sure which buckets + * their documents will ultimately fall into + * (like {@code auto_date_histogram}). If they do so then anything they + * return from this method can't be trusted. To turn them into "real" + * results you have to make all of the calls to this method that you + * will ever make, then call {@link #runDeferredCollections()}. If that + * returns {@code true} then every aggregation will need to be rewritten + * with {@code InternalAggregation#undefer()}. This entire dance is + * generally accomplished by calling {@link #buildTopLevel()} on each top + * level aggregator. */ - public abstract InternalAggregation buildAggregation(long bucket) throws IOException; + public abstract InternalAggregation buildAggregation(long owningBucketOrd) throws IOException; + + /** + * Build the result of this aggregation if it is on top level of the + * aggregation tree, properly handling deferred collections. It is only + * correct to call this on aggregations that are at the + * top level of the aggregation tree. Calling it for other aggregations + * will trip assertions or return the wrong result. Those aggregtions will + * have {@link #buildAggregation(long)}, {@link #runDeferredCollections()} + * and {@link InternalAggregation#undefer()} called by their parent + * aggregations. + */ + public final InternalAggregation buildTopLevel() throws IOException { + assert parent() == null; + InternalAggregation result = buildAggregation(0); + if (runDeferredCollections()) { + return result.undefer(); + } + return result; + } /** * Build an empty aggregation. */ public abstract InternalAggregation buildEmptyAggregation(); + /** + * Run any deferred collections that are required to + * {@link InternalAggregation#undefer()} this aggregations's results. + */ + public abstract boolean runDeferredCollections() throws IOException; + /** Aggregation mode for sub aggregations. */ public enum SubAggCollectionMode implements Writeable { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorBase.java b/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorBase.java index 11ec88a8ce8bd..3265cf396f3fb 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorBase.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorBase.java @@ -254,6 +254,15 @@ protected void doClose() {} protected void doPostCollection() throws IOException { } + @Override + public boolean runDeferredCollections() throws IOException { + boolean ran = false; + for (Aggregator sub : subAggregators) { + ran |= sub.runDeferredCollections(); + } + return ran; + } + protected final InternalAggregations buildEmptySubAggregations() { List aggs = new ArrayList<>(); for (Aggregator aggregator : subAggregators) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java index b308de0327e52..536547eabf2fc 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java @@ -159,6 +159,18 @@ public InternalAggregation buildEmptyAggregation() { return first.buildEmptyAggregation(); } + @Override + public boolean runDeferredCollections() throws IOException { + boolean ran = false; + for (long i = 0; i < aggregators.size(); ++i) { + final Aggregator aggregator = aggregators.get(i); + if (aggregator != null) { + ran |= aggregator.runDeferredCollections(); + } + } + return ran; + } + @Override public void close() { Releasables.close(aggregators, collectors); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/InternalAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/InternalAggregation.java index df3971e07a305..573d25b03dd17 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/InternalAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/InternalAggregation.java @@ -31,6 +31,7 @@ import org.elasticsearch.search.aggregations.support.AggregationPath; import java.io.IOException; +import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -333,4 +334,39 @@ public double sortValue(AggregationPath.PathElement head, Iterator { + List results = null; + for (int a = 0; a < ia.aggregations.size(); a++) { + InternalAggregation orig = ((InternalAggregation) ia.aggregations.get(a)); + InternalAggregation undeferred = orig.undefer(); + if (undeferred == orig) { + if (results != null) { + results.add(orig); + } + continue; + } + if (results == null) { + results = new ArrayList<>(ia.aggregations.size()); + for (int fillIn = 0; fillIn < a; fillIn++) { + results.add((InternalAggregation) ia.aggregations.get(fillIn)); + } + } + results.add(undeferred); + } + return results == null ? null : new InternalAggregations(results); + }); + } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/InternalMultiBucketAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/InternalMultiBucketAggregation.java index a1e1efd1f5152..d85a407a25e5e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/InternalMultiBucketAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/InternalMultiBucketAggregation.java @@ -158,19 +158,27 @@ public final InternalAggregation reducePipelines( @Override public InternalAggregation copyWithRewritenBuckets(Function rewriter) { - boolean modified = false; - List newBuckets = new ArrayList<>(); - for (B bucket : getBuckets()) { + List orig = getBuckets(); + List newBuckets = null; + for (int b = 0; b < orig.size(); b++) { + B bucket = orig.get(b); InternalAggregations rewritten = rewriter.apply((InternalAggregations) bucket.getAggregations()); if (rewritten == null) { - newBuckets.add(bucket); + if (newBuckets != null) { + newBuckets.add(bucket); + } continue; } - modified = true; + if (newBuckets == null) { + newBuckets = new ArrayList<>(orig.size()); + for (int fillIn = 0; fillIn < b; fillIn++) { + newBuckets.add(orig.get(fillIn)); + } + } B newBucket = createBucket(rewritten, bucket); newBuckets.add(newBucket); } - return modified ? create(newBuckets) : this; + return newBuckets == null ? this : create(newBuckets); } @Override @@ -193,6 +201,10 @@ private List reducePipelineBuckets(ReduceContext reduceContext, PipelineTree return reducedBuckets; } + public final InternalAggregation undefer() { + return undeferBuckets(); + } + public abstract static class InternalBucket implements Bucket, Writeable { public Object getProperty(String containingAggName, List path) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/LeafBucketCollector.java b/server/src/main/java/org/elasticsearch/search/aggregations/LeafBucketCollector.java index 367e1cce0608d..da5009371f986 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/LeafBucketCollector.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/LeafBucketCollector.java @@ -21,6 +21,7 @@ import org.apache.lucene.search.LeafCollector; import org.apache.lucene.search.Scorable; +import org.elasticsearch.search.aggregations.bucket.terms.LongKeyedBucketOrds; import java.io.IOException; import java.util.stream.Stream; @@ -73,9 +74,33 @@ public void collect(int doc, long bucket) throws IOException { } /** - * Collect the given doc in the given bucket. + * Collect the given {@code doc} in the bucket owned by + * {@code owningBucketOrd}. + *

+ * The implementation of this method metric aggregations is generally + * something along the lines of + *

{@code
+     * array[owningBucketOrd] += loadValueFromDoc(doc)
+     * }
+ *

Bucket aggregations have more trouble because their job is to + * make new ordinals. So their implementation generally + * looks kind of like + *

{@code
+     * long myBucketOrd = mapOwningBucketAndValueToMyOrd(owningBucketOrd, loadValueFromDoc(doc));
+     * collectBucket(doc, myBucketOrd);
+     * }
+ *

+ * Some bucket aggregations "know" how many ordinals each owning ordinal + * needs so they can map "densely". The {@code range} aggregation, for + * example, can perform this mapping with something like: + *

{@code
+     * return rangeCount * owningBucketOrd + matchingRange(value);
+     * }
+ * Other aggregations don't know how many buckets will fall into any + * particular owning bucket. The {@code terms} aggregation, for example, + * uses {@link LongKeyedBucketOrds} which amounts to a hash lookup. */ - public abstract void collect(int doc, long bucket) throws IOException; + public abstract void collect(int doc, long owningBucketOrd) throws IOException; @Override public final void collect(int doc) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java index 9a0b0a83d4961..79d8496ee4934 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java @@ -29,7 +29,6 @@ import org.apache.lucene.search.Weight; import org.apache.lucene.util.packed.PackedInts; import org.apache.lucene.util.packed.PackedLongValues; -import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.LongHash; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.BucketCollector; @@ -146,7 +145,7 @@ public void postCollection() throws IOException { * Replay the wrapped collector, but only on a selection of buckets. */ @Override - public void prepareSelectedBuckets(long... selectedBuckets) throws IOException { + public void prepareSelectedBuckets(LongHash selectedBuckets) throws IOException { if (finished == false) { throw new IllegalStateException("Cannot replay yet, collection is not finished: postCollect() has not been called"); } @@ -154,11 +153,7 @@ public void prepareSelectedBuckets(long... selectedBuckets) throws IOException { throw new IllegalStateException("Already been replayed"); } - final LongHash hash = new LongHash(selectedBuckets.length, BigArrays.NON_RECYCLING_INSTANCE); - for (long bucket : selectedBuckets) { - hash.add(bucket); - } - this.selectedBuckets = hash; + this.selectedBuckets = selectedBuckets; boolean needsScores = scoreMode().needsScores(); Weight weight = null; @@ -185,7 +180,7 @@ public void prepareSelectedBuckets(long... selectedBuckets) throws IOException { for (long i = 0, end = entry.docDeltas.size(); i < end; ++i) { doc += docDeltaIterator.next(); final long bucket = buckets.next(); - final long rebasedBucket = hash.find(bucket); + final long rebasedBucket = selectedBuckets.find(bucket); if (rebasedBucket != -1) { if (needsScores) { if (scoreIt.docID() < doc) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java index 81aa90edac337..da33eeebddf14 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java @@ -174,6 +174,9 @@ public Aggregator resolveSortPath(AggregationPath.PathElement next, Iterator order.reverseMul() * Integer.compare(bucketDocCount(lhs), bucketDocCount(rhs)); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferableBucketAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferableBucketAggregator.java index f8713b665365d..325bfa5c7931e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferableBucketAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferableBucketAggregator.java @@ -19,9 +19,16 @@ package org.elasticsearch.search.aggregations.bucket; +import org.apache.lucene.util.IOSupplier; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.lease.Releasables; +import org.elasticsearch.common.util.LongHash; +import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.AggregatorFactory.MultiBucketAggregatorWrapper; import org.elasticsearch.search.aggregations.BucketCollector; +import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.MultiBucketCollector; import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregator; import org.elasticsearch.search.internal.SearchContext; @@ -32,8 +39,28 @@ import java.util.Map; public abstract class DeferableBucketAggregator extends BucketsAggregator { - + /** + * Wrapper that records collections. Non-null if any aggregations have + * been deferred. + */ private DeferringBucketCollector recordingWrapper; + /** + * Hash of surviving sub-aggregation ordinals. Non-null if there are any + * surviving ordinals. + */ + private LongHash survivingOrds; + /** + * List of deferred results to build when running deferred collections. + * Non-null if any results have been deferred. + *

+ * Its kind of a shame to keep a {@linkplain List} around to keep the + * reference to these but we must build the results after + * running our deferred collections and before running our + * sub-aggregation's deferred collections. This is because it is building + * our results that marks which gives our sub-aggregators the chance to + * mark any of their buckets as surviving. + */ + private List deferred; protected DeferableBucketAggregator(String name, AggregatorFactories factories, SearchContext context, Aggregator parent, Map metadata) throws IOException { @@ -95,6 +122,15 @@ protected boolean shouldDefer(Aggregator aggregator) { return false; } + /** + * Collect sub aggregations for a list of bucket ordinals. This may + * only be called once so any aggregation that calls this must be + * wrapped in {@link MultiBucketAggregatorWrapper}. + * @deprecated prefer delaying construction of the result with many calls + * to {@link #recordSurvingOrd(long)} and returning + * {@link #deferred(IOSupplier)}. + */ + @Deprecated protected final void runDeferredCollections(long... bucketOrds) throws IOException { // Being lenient here - ignore calls where there are no deferred // collections to playback @@ -103,4 +139,132 @@ protected final void runDeferredCollections(long... bucketOrds) throws IOExcepti } } + @Override + public final boolean runDeferredCollections() throws IOException { + if (survivingOrds == null) { + /* + * If there aren't any ords that need to be replayed we just + * function as a regular aggregator, giving our sub-aggregation's + * a chance to run deferred collections if they need to. + */ + return super.runDeferredCollections(); + } + /* + * If we have any deferred collections we reply them and then + * build our results. Building our results will have the side + * effect of marking any of the our children's surviving buckets + * for replay. So, after that, we can finally run our children's + * deferred collections by calling super.runDeferredCollections. + */ + recordingWrapper.prepareSelectedBuckets(survivingOrds); + for (DeferredInternalAggregation d : deferred) { + d.buildRealResult(); + } + super.runDeferredCollections(); + return true; + } + + /** + * Record an ordinal to be replayed before suppliers passed to + * {@link #deferred(IOSupplier)} are called. + */ + protected final void recordSurvingOrd(long ord) { + if (recordingWrapper == null) { + /* + * recording wrapper is null if no child aggregations have + * been delayed. + */ + return; + } + if (survivingOrds == null) { + survivingOrds = new LongHash(1, context.bigArrays()); + } + survivingOrds.add(ord); + } + + /** + * Build a "deferred" aggregation result. The provided supplier is called + * after all ordinals recorded with {@link #recordSurvingOrd(long)} are + * replayed on child aggregations. + */ + protected final InternalAggregation deferred(IOSupplier buildRealResult) throws IOException { + if (survivingOrds == null) { + /* + * No aggregations have been delayed so we are safe to finish + * building the result right away. + */ + return buildRealResult.get(); + } + if (deferred == null) { + deferred = new ArrayList<>(); + } + DeferredInternalAggregation d = new DeferredInternalAggregation(name, buildRealResult); + deferred.add(d); + return d; + } + + @Override + protected void doClose() { + super.doClose(); + Releasables.close(survivingOrds); + } + + private static class DeferredInternalAggregation extends InternalAggregation { + private final IOSupplier buildRealResult; + private InternalAggregation realResult; + + DeferredInternalAggregation(String name, IOSupplier buildRealResult) { + super(name, null); + this.buildRealResult = buildRealResult; + } + + @Override + public String toString() { + if (realResult == null) { + return "deferred [" + name + "/" + buildRealResult + "]"; + } else { + return realResult.toString(); + } + } + + private void buildRealResult() throws IOException { + realResult = buildRealResult.get(); + if (realResult == null) { + throw new IllegalStateException("undeferring bucket didn't build anything"); + } + } + + @Override + public InternalAggregation undefer() { + if (realResult == null) { + throw new IllegalStateException("deferred collections not replayed!"); + } + return realResult.undefer(); + } + + @Override + public String getWriteableName() { + throw new IllegalStateException(); + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + throw new IllegalStateException(); + } + + @Override + public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) { + throw new IllegalStateException(); + } + + @Override + public Object getProperty(List path) { + throw new IllegalStateException(); + } + + @Override + public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + throw new IllegalStateException(); + } + } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java index 10dd8ee1bf7b9..96dabbc1cf86a 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java @@ -21,6 +21,9 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.ScoreMode; +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.LongHash; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.BucketCollector; import org.elasticsearch.search.aggregations.InternalAggregation; @@ -45,10 +48,21 @@ public DeferringBucketCollector() {} public abstract void setDeferredCollector(Iterable deferredCollectors); public final void replay(long... selectedBuckets) throws IOException { - prepareSelectedBuckets(selectedBuckets); + LongHash hash = new LongHash(selectedBuckets.length, BigArrays.NON_RECYCLING_INSTANCE); + for (long bucket : selectedBuckets) { + hash.add(bucket); + } + prepareSelectedBuckets(hash); } - public abstract void prepareSelectedBuckets(long... selectedBuckets) throws IOException; + /** + * Replay some selected buckets. + *

+ * The collector might retain a reference to the provided + * buckets but it doesn't take responsibility for + * {@link Releasable#close() closing} it. + */ + public abstract void prepareSelectedBuckets(LongHash selectedBuckets) throws IOException; /** * Wrap the provided aggregator so that it behaves (almost) as if it had @@ -105,6 +119,11 @@ public InternalAggregation buildEmptyAggregation() { return in.buildEmptyAggregation(); } + @Override + public boolean runDeferredCollections() throws IOException { + return in.runDeferredCollections(); + } + @Override public LeafBucketCollector getLeafCollector(LeafReaderContext ctx) throws IOException { throw new IllegalStateException( diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregation.java index a05c317a9908d..2a265c2c30c07 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregation.java @@ -186,6 +186,10 @@ public void forEachBucket(Consumer consumer) { consumer.accept(aggregations); } + public final InternalAggregation undefer() { + return undeferBuckets(); + } + @Override public boolean equals(Object obj) { if (this == obj) return true; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java index bff5015846951..2fd2be6613351 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java @@ -37,7 +37,7 @@ public MergingBucketsDeferringCollector(SearchContext context, boolean isGlobal) super(context, isGlobal); } -/** + /** * Merges/prunes the existing bucket ordinals and docDeltas according to the provided mergeMap. * * The mergeMap is an array where the index position represents the current bucket ordinal, and diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregator.java index dd5924e8b472a..439c4df3c15a7 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregator.java @@ -138,7 +138,7 @@ public InternalAggregation buildAggregation(long zeroBucket) throws IOException consumeBucketsAndMaybeBreak(queue.size()); if (deferredCollectors != NO_OP_COLLECTOR) { // Replay all documents that contain at least one top bucket (collected during the first pass). - runDeferredCollections(); + runCompositeDeferredCollections(); } int num = Math.min(size, queue.size()); @@ -352,8 +352,11 @@ public void collect(int doc, long bucket) throws IOException { /** * Replay the documents that might contain a top bucket and pass top buckets to * the {@link #deferredCollectors}. + *

+ * Note: this is distinct from {@link #runDeferredCollections()} because it + * predates it and works differently. */ - private void runDeferredCollections() throws IOException { + private void runCompositeDeferredCollections() throws IOException { final boolean needsScores = scoreMode().needsScores(); Weight weight = null; if (needsScores) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/BestDocsDeferringCollector.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/BestDocsDeferringCollector.java index 47174089cd5e8..82ba5dbb1912e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/BestDocsDeferringCollector.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/BestDocsDeferringCollector.java @@ -32,6 +32,7 @@ import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.LongHash; import org.elasticsearch.common.util.ObjectArray; import org.elasticsearch.search.aggregations.BucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollector; @@ -131,7 +132,7 @@ public void postCollection() throws IOException { @Override - public void prepareSelectedBuckets(long... selectedBuckets) throws IOException { + public void prepareSelectedBuckets(LongHash selectedBuckets) throws IOException { // no-op - deferred aggs processed in postCollection call } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTermsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTermsAggregator.java index cee8d16d472e2..ec628b18f9929 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTermsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTermsAggregator.java @@ -28,6 +28,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic; import org.elasticsearch.search.aggregations.bucket.terms.IncludeExclude; +import org.elasticsearch.search.aggregations.bucket.terms.LongKeyedBucketOrds.BucketOrdsEnum; import org.elasticsearch.search.aggregations.bucket.terms.LongTermsAggregator; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.internal.ContextIndexSearcher; @@ -47,7 +48,7 @@ public SignificantLongTermsAggregator(String name, AggregatorFactories factories IncludeExclude.LongFilter includeExclude, Map metadata) throws IOException { super(name, factories, valuesSource, format, null, bucketCountThresholds, context, parent, - SubAggCollectionMode.BREADTH_FIRST, false, includeExclude, metadata); + SubAggCollectionMode.BREADTH_FIRST, false, includeExclude, false, metadata); this.significanceHeuristic = significanceHeuristic; this.termsAggFactory = termsAggFactory; } @@ -69,25 +70,27 @@ public void collect(int doc, long bucket) throws IOException { } @Override - public SignificantLongTerms buildAggregation(long owningBucketOrdinal) throws IOException { - assert owningBucketOrdinal == 0; + public SignificantLongTerms buildAggregation(long owningBucketOrd) throws IOException { + assert owningBucketOrd == 0; - final int size = (int) Math.min(bucketOrds.size(), bucketCountThresholds.getShardSize()); + long bucketsInOrd = bucketOrds.bucketsInOrd(owningBucketOrd); + final int size = (int) Math.min(bucketsInOrd, bucketCountThresholds.getShardSize()); long supersetSize = termsAggFactory.getSupersetNumDocs(); long subsetSize = numCollectedDocs; BucketSignificancePriorityQueue ordered = new BucketSignificancePriorityQueue<>(size); SignificantLongTerms.Bucket spare = null; - for (long i = 0; i < bucketOrds.size(); i++) { - final int docCount = bucketDocCount(i); + BucketOrdsEnum ordsEnum = bucketOrds.ordsEnum(owningBucketOrd); + while (ordsEnum.next()) { + final int docCount = bucketDocCount(ordsEnum.ord()); if (docCount < bucketCountThresholds.getShardMinDocCount()) { continue; } if (spare == null) { spare = new SignificantLongTerms.Bucket(0, 0, 0, 0, 0, null, format, 0); } - spare.term = bucketOrds.get(i); + spare.term = ordsEnum.value(); spare.subsetDf = docCount; spare.subsetSize = subsetSize; spare.supersetDf = termsAggFactory.getBackgroundFrequency(spare.term); @@ -96,7 +99,7 @@ public SignificantLongTerms buildAggregation(long owningBucketOrdinal) throws IO // Back at the central reducer these properties will be updated with global stats spare.updateScore(significanceHeuristic); - spare.bucketOrd = i; + spare.bucketOrd = ordsEnum.ord(); spare = ordered.insertWithOverflow(spare); if (spare == null) { consumeBucketsAndMaybeBreak(1); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsAggregator.java index 827b40444abef..6f42cbcf6b75d 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsAggregator.java @@ -26,6 +26,8 @@ import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.BucketOrder; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.bucket.terms.LongTerms.Bucket; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; import org.elasticsearch.search.internal.SearchContext; @@ -40,9 +42,9 @@ public class DoubleTermsAggregator extends LongTermsAggregator { DoubleTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, DocValueFormat format, BucketOrder order, BucketCountThresholds bucketCountThresholds, SearchContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, IncludeExclude.LongFilter longFilter, - Map metadata) throws IOException { + boolean collectsFromSingleBucket, Map metadata) throws IOException { super(name, factories, valuesSource, format, order, bucketCountThresholds, aggregationContext, parent, collectionMode, - showTermDocCountError, longFilter, metadata); + showTermDocCountError, longFilter, collectsFromSingleBucket, metadata); } @Override @@ -51,9 +53,8 @@ protected SortedNumericDocValues getValues(Numeric valuesSource, LeafReaderConte } @Override - public DoubleTerms buildAggregation(long owningBucketOrdinal) throws IOException { - final LongTerms terms = (LongTerms) super.buildAggregation(owningBucketOrdinal); - return convertToDouble(terms); + protected InternalAggregation buildResult(long otherDocCount, List buckets) { + return convertToDouble((LongTerms) super.buildResult(otherDocCount, buckets)); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongKeyedBucketOrds.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongKeyedBucketOrds.java new file mode 100644 index 0000000000000..c0dca23b51b53 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongKeyedBucketOrds.java @@ -0,0 +1,266 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.bucket.terms; + +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.lease.Releasables; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.LongArray; +import org.elasticsearch.common.util.LongHash; +import org.elasticsearch.common.util.ObjectArray; + +/** + * Maps long bucket keys to bucket ordinals. + */ +public abstract class LongKeyedBucketOrds implements Releasable { + /** + * Build a {@link LongKeyedBucketOrds}. + */ + public static LongKeyedBucketOrds build(BigArrays bigArrays, boolean collectsFromSingleBucket) { + return collectsFromSingleBucket ? new FromSingle(bigArrays) : new FromMany(bigArrays); + } + + private LongKeyedBucketOrds() {} + + /** + * Add the {@code owningBucketOrd, term} pair. Return the ord for + * their bucket if they have yet to be added, or {@code -1-ord} + * if they were already present. + */ + public abstract long add(long owningBucketOrd, long value); + + /** + * Count the buckets in {@code owningBucketOrd}. + */ + public abstract long bucketsInOrd(long owningBucketOrd); + + /** + * Build an iterator for buckets inside {@code owningBucketOrd}. + *

+ * When this is first returns it is "unpositioned" and you must call + * {@link BucketOrdsEnum#next()} to move it to the first value. + */ + public abstract BucketOrdsEnum ordsEnum(long owningBucketOrd); + /** + * An iterator for buckets inside a particular {@code owningBucketOrd}. + */ + public interface BucketOrdsEnum { + /** + * Advance to the next value. + * @return {@code true} if there *is* a next value, + * {@code false} if there isn't + */ + boolean next(); + /** + * The ordinal of the current value. + */ + long ord(); + /** + * The current value. + */ + long value(); + + /** + * An {@linkplain BucketOrdsEnum} that is empty. + */ + BucketOrdsEnum EMPTY = new BucketOrdsEnum() { + @Override + public boolean next() { return false; } + @Override + public long ord() { return 0; } + @Override + public long value() { return 0; } + }; + } + + + /** + * Implementation that only works if it is collecting from a single bucket. + */ + private static class FromSingle extends LongKeyedBucketOrds { + private final LongHash ords; + + FromSingle(BigArrays bigArrays) { + ords = new LongHash(1, bigArrays); + } + + @Override + public long add(long owningBucketOrd, long value) { + assert owningBucketOrd == 0; + return ords.add(value); + } + + @Override + public long bucketsInOrd(long owningBucketOrd) { + assert owningBucketOrd == 0; + return ords.size(); + } + + @Override + public BucketOrdsEnum ordsEnum(long owningBucketOrd) { + assert owningBucketOrd == 0; + return new BucketOrdsEnum() { + private long ord = -1; + private long value; + + @Override + public boolean next() { + ord++; + if (ord >= ords.size()) { + return false; + } + value = ords.get(ord); + return true; + } + + @Override + public long value() { + return value; + } + + @Override + public long ord() { + return ord; + } + }; + } + + @Override + public void close() { + ords.close(); + } + } + + /** + * Implementation that works properly when collecting from many buckets. + */ + private static class FromMany extends LongKeyedBucketOrds { + // TODO we can almost certainly do better here by building something fit for purpose rather than trying to lego together stuff + private static class Buckets implements Releasable { + private final LongHash valueToThisBucketOrd; + private LongArray thisBucketOrdToGlobalOrd; + + Buckets(BigArrays bigArrays) { + valueToThisBucketOrd = new LongHash(1, bigArrays); + thisBucketOrdToGlobalOrd = bigArrays.newLongArray(1, false); + } + + @Override + public void close() { + Releasables.close(valueToThisBucketOrd, thisBucketOrdToGlobalOrd); + } + } + private final BigArrays bigArrays; + private ObjectArray owningOrdToBuckets; + private long lastGlobalOrd = -1; + + FromMany(BigArrays bigArrays) { + this.bigArrays = bigArrays; + owningOrdToBuckets = bigArrays.newObjectArray(1); + } + + @Override + public long add(long owningBucketOrd, long value) { + Buckets buckets = bucketsForOrd(owningBucketOrd); + long thisBucketOrd = buckets.valueToThisBucketOrd.add(value); + if (thisBucketOrd < 0) { + // Already in the hash + thisBucketOrd = -1 - thisBucketOrd; + return -1 - buckets.thisBucketOrdToGlobalOrd.get(thisBucketOrd); + } + buckets.thisBucketOrdToGlobalOrd = bigArrays.grow(buckets.thisBucketOrdToGlobalOrd, thisBucketOrd + 1); + lastGlobalOrd++; + buckets.thisBucketOrdToGlobalOrd.set(thisBucketOrd, lastGlobalOrd); + return lastGlobalOrd; + } + + private Buckets bucketsForOrd(long owningBucketOrd) { + if (owningOrdToBuckets.size() <= owningBucketOrd) { + owningOrdToBuckets = bigArrays.grow(owningOrdToBuckets, owningBucketOrd + 1); + Buckets buckets = new Buckets(bigArrays); + owningOrdToBuckets.set(owningBucketOrd, buckets); + return buckets; + } + Buckets buckets = owningOrdToBuckets.get(owningBucketOrd); + if (buckets == null) { + buckets = new Buckets(bigArrays); + owningOrdToBuckets.set(owningBucketOrd, buckets); + } + return buckets; + } + + @Override + public long bucketsInOrd(long owningBucketOrd) { + Buckets buckets = owningOrdToBuckets.get(owningBucketOrd); + if (buckets == null) { + return 0; + } + return buckets.valueToThisBucketOrd.size(); + } + + @Override + public BucketOrdsEnum ordsEnum(long owningBucketOrd) { + if (owningBucketOrd >= owningOrdToBuckets.size()) { + return BucketOrdsEnum.EMPTY; + } + Buckets buckets = owningOrdToBuckets.get(owningBucketOrd); + if (buckets == null) { + return BucketOrdsEnum.EMPTY; + } + return new BucketOrdsEnum() { + private long thisBucketOrd = -1; + private long value; + private long ord; + + @Override + public boolean next() { + thisBucketOrd++; + if (thisBucketOrd >= buckets.valueToThisBucketOrd.size()) { + return false; + } + value = buckets.valueToThisBucketOrd.get(thisBucketOrd); + ord = buckets.thisBucketOrdToGlobalOrd.get(thisBucketOrd); + return true; + } + + @Override + public long value() { + return value; + } + + @Override + public long ord() { + return ord; + } + }; + } + + @Override + public void close() { + for (long owningBucketOrd = 0; owningBucketOrd < owningOrdToBuckets.size(); owningBucketOrd++) { + Buckets buckets = owningOrdToBuckets.get(owningBucketOrd); + if (buckets != null) { + buckets.close(); + } + } + owningOrdToBuckets.close(); + } + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java index 8d187b79c83d3..6f2bf359b8f7f 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java @@ -22,7 +22,6 @@ import org.apache.lucene.index.SortedNumericDocValues; import org.apache.lucene.search.ScoreMode; import org.elasticsearch.common.lease.Releasables; -import org.elasticsearch.common.util.LongHash; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; @@ -32,11 +31,14 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.bucket.terms.IncludeExclude.LongFilter; +import org.elasticsearch.search.aggregations.bucket.terms.LongKeyedBucketOrds.BucketOrdsEnum; +import org.elasticsearch.search.aggregations.bucket.terms.LongTerms.Bucket; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; import java.util.Arrays; +import java.util.List; import java.util.Map; import static java.util.Collections.emptyList; @@ -44,19 +46,19 @@ public class LongTermsAggregator extends TermsAggregator { protected final ValuesSource.Numeric valuesSource; - protected final LongHash bucketOrds; + protected final LongKeyedBucketOrds bucketOrds; private boolean showTermDocCountError; private LongFilter longFilter; public LongTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, DocValueFormat format, BucketOrder order, BucketCountThresholds bucketCountThresholds, SearchContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, IncludeExclude.LongFilter longFilter, - Map metadata) throws IOException { + boolean collectsFromSingleBucket, Map metadata) throws IOException { super(name, factories, aggregationContext, parent, bucketCountThresholds, order, format, subAggCollectMode, metadata); this.valuesSource = valuesSource; this.showTermDocCountError = showTermDocCountError; this.longFilter = longFilter; - bucketOrds = new LongHash(1, aggregationContext.bigArrays()); + bucketOrds = LongKeyedBucketOrds.build(context.bigArrays(), collectsFromSingleBucket); } @Override @@ -77,8 +79,7 @@ public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final SortedNumericDocValues values = getValues(valuesSource, ctx); return new LeafBucketCollectorBase(sub, values) { @Override - public void collect(int doc, long owningBucketOrdinal) throws IOException { - assert owningBucketOrdinal == 0; + public void collect(int doc, long owningBucketOrd) throws IOException { if (values.advanceExact(doc)) { final int valuesCount = values.docValueCount(); @@ -87,7 +88,7 @@ public void collect(int doc, long owningBucketOrdinal) throws IOException { final long val = values.nextValue(); if (previous != val || i == 0) { if ((longFilter == null) || (longFilter.accept(val))) { - long bucketOrdinal = bucketOrds.add(val); + long bucketOrdinal = bucketOrds.add(owningBucketOrd, val); if (bucketOrdinal < 0) { // already seen bucketOrdinal = -1 - bucketOrdinal; collectExistingBucket(sub, doc, bucketOrdinal); @@ -105,11 +106,10 @@ public void collect(int doc, long owningBucketOrdinal) throws IOException { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - assert owningBucketOrdinal == 0; - + public InternalAggregation buildAggregation(long owningBucketOrd) throws IOException { + long bucketsInOrd = bucketOrds.bucketsInOrd(owningBucketOrd); if (bucketCountThresholds.getMinDocCount() == 0 && (InternalOrder.isCountDesc(order) == false || - bucketOrds.size() < bucketCountThresholds.getRequiredSize())) { + bucketsInOrd < bucketCountThresholds.getRequiredSize())) { // we need to fill-in the blanks for (LeafReaderContext ctx : context.searcher().getTopReaderContext().leaves()) { final SortedNumericDocValues values = getValues(valuesSource, ctx); @@ -119,26 +119,28 @@ public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOE for (int i = 0; i < valueCount; ++i) { long value = values.nextValue(); if (longFilter == null || longFilter.accept(value)) { - bucketOrds.add(value); + bucketOrds.add(owningBucketOrd, value); } } } } } + bucketsInOrd = bucketOrds.bucketsInOrd(owningBucketOrd); } - final int size = (int) Math.min(bucketOrds.size(), bucketCountThresholds.getShardSize()); + final int size = (int) Math.min(bucketsInOrd, bucketCountThresholds.getShardSize()); long otherDocCount = 0; BucketPriorityQueue ordered = new BucketPriorityQueue<>(size, partiallyBuiltBucketComparator); LongTerms.Bucket spare = null; - for (long i = 0; i < bucketOrds.size(); i++) { + BucketOrdsEnum ordsEnum = bucketOrds.ordsEnum(owningBucketOrd); + while (ordsEnum.next()) { if (spare == null) { spare = new LongTerms.Bucket(0, 0, null, showTermDocCountError, 0, format); } - spare.term = bucketOrds.get(i); - spare.docCount = bucketDocCount(i); + spare.term = ordsEnum.value(); + spare.docCount = bucketDocCount(ordsEnum.ord()); otherDocCount += spare.docCount; - spare.bucketOrd = i; + spare.bucketOrd = ordsEnum.ord(); if (bucketCountThresholds.getShardMinDocCount() <= spare.docCount) { spare = ordered.insertWithOverflow(spare); if (spare == null) { @@ -149,25 +151,29 @@ public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOE // Get the top buckets final LongTerms.Bucket[] list = new LongTerms.Bucket[ordered.size()]; - long survivingBucketOrds[] = new long[ordered.size()]; for (int i = ordered.size() - 1; i >= 0; --i) { - final LongTerms.Bucket bucket = (LongTerms.Bucket) ordered.pop(); - survivingBucketOrds[i] = bucket.bucketOrd; + final LongTerms.Bucket bucket = ordered.pop(); + recordSurvingOrd(bucket.bucketOrd); list[i] = bucket; otherDocCount -= bucket.docCount; } - runDeferredCollections(survivingBucketOrds); + // Return a deferred agg that will build the buckets when undeferred + final long finalOtherDocCount = otherDocCount; + return deferred(() -> { + for (int i = 0; i < list.length; i++) { + list[i].aggregations = bucketAggregations(list[i].bucketOrd); + list[i].docCountError = 0; + } - // Now build the aggs - for (int i = 0; i < list.length; i++) { - list[i].aggregations = bucketAggregations(list[i].bucketOrd); - list[i].docCountError = 0; - } + return buildResult(finalOtherDocCount, Arrays.asList(list)); + }); + } + protected InternalAggregation buildResult(long otherDocCount, List buckets) { return new LongTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), - metadata(), format, bucketCountThresholds.getShardSize(), showTermDocCountError, otherDocCount, - Arrays.asList(list), 0); + metadata(), format, bucketCountThresholds.getShardSize(), showTermDocCountError, otherDocCount, + buckets, 0); } @Override @@ -178,7 +184,7 @@ public InternalAggregation buildEmptyAggregation() { @Override public void doClose() { + super.doClose(); Releasables.close(bucketOrds); } - } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java index c427650ed5cb2..da14b32053a69 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java @@ -90,7 +90,9 @@ public Aggregator build(String name, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, + boolean collectsFromSingleBucket, Map metadata) throws IOException { + assert collectsFromSingleBucket; ExecutionMode execution = null; if (executionHint != null) { @@ -124,6 +126,11 @@ public Aggregator build(String name, context, parent, subAggCollectMode, showTermDocCountError, metadata); } + + @Override + public boolean needsToCollectFromSingleBucket() { + return true; + } }; } @@ -146,6 +153,7 @@ public Aggregator build(String name, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, + boolean collectsFromSingleBucket, Map metadata) throws IOException { if ((includeExclude != null) && (includeExclude.isRegexBased())) { @@ -168,13 +176,20 @@ public Aggregator build(String name, longFilter = includeExclude.convertToDoubleFilter(); } return new DoubleTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, format, order, - bucketCountThresholds, context, parent, subAggCollectMode, showTermDocCountError, longFilter, metadata); + bucketCountThresholds, context, parent, subAggCollectMode, showTermDocCountError, longFilter, + collectsFromSingleBucket, metadata); } if (includeExclude != null) { longFilter = includeExclude.convertToLongFilter(format); } return new LongTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, format, order, - bucketCountThresholds, context, parent, subAggCollectMode, showTermDocCountError, longFilter, metadata); + bucketCountThresholds, context, parent, subAggCollectMode, showTermDocCountError, longFilter, + collectsFromSingleBucket, metadata); + } + + @Override + public boolean needsToCollectFromSingleBucket() { + return false; } }; } @@ -234,10 +249,6 @@ protected Aggregator doCreateInternal(ValuesSource valuesSource, Aggregator parent, boolean collectsFromSingleBucket, Map metadata) throws IOException { - if (collectsFromSingleBucket == false) { - return asMultiBucketAggregator(this, searchContext, parent); - } - AggregatorSupplier aggregatorSupplier = queryShardContext.getValuesSourceRegistry().getAggregator(config.valueSourceType(), TermsAggregationBuilder.NAME); if (aggregatorSupplier instanceof TermsAggregatorSupplier == false) { @@ -246,6 +257,10 @@ protected Aggregator doCreateInternal(ValuesSource valuesSource, } TermsAggregatorSupplier termsAggregatorSupplier = (TermsAggregatorSupplier) aggregatorSupplier; + if (collectsFromSingleBucket == false && termsAggregatorSupplier.needsToCollectFromSingleBucket()) { + return asMultiBucketAggregator(this, searchContext, parent); + } + BucketCountThresholds bucketCountThresholds = new BucketCountThresholds(this.bucketCountThresholds); if (InternalOrder.isKeyOrder(order) == false && bucketCountThresholds.getShardSize() == TermsAggregationBuilder.DEFAULT_BUCKET_COUNT_THRESHOLDS.getShardSize()) { @@ -258,7 +273,7 @@ protected Aggregator doCreateInternal(ValuesSource valuesSource, return termsAggregatorSupplier.build(name, factories, valuesSource, order, config.format(), bucketCountThresholds, includeExclude, executionHint, searchContext, parent, collectMode, - showTermDocCountError, metadata); + showTermDocCountError, collectsFromSingleBucket, metadata); } // return the SubAggCollectionMode that this aggregation should use based on the expected size diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorSupplier.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorSupplier.java index 240b451454ca9..e862e8f4fd53d 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorSupplier.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorSupplier.java @@ -42,5 +42,8 @@ Aggregator build(String name, Aggregator parent, Aggregator.SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, + boolean collectsFromSingleBucket, Map metadata) throws IOException; + + boolean needsToCollectFromSingleBucket(); } diff --git a/server/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationTimingType.java b/server/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationTimingType.java index d1c5d3dd53859..d7039e90d7973 100644 --- a/server/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationTimingType.java +++ b/server/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationTimingType.java @@ -25,6 +25,7 @@ public enum AggregationTimingType { INITIALIZE, COLLECT, BUILD_AGGREGATION, + RUN_DEFERRED_COLLECTIONS, REDUCE; @Override diff --git a/server/src/main/java/org/elasticsearch/search/profile/aggregation/ProfilingAggregator.java b/server/src/main/java/org/elasticsearch/search/profile/aggregation/ProfilingAggregator.java index 0a770c2baf56d..032f89c303607 100644 --- a/server/src/main/java/org/elasticsearch/search/profile/aggregation/ProfilingAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/profile/aggregation/ProfilingAggregator.java @@ -101,6 +101,17 @@ public InternalAggregation buildEmptyAggregation() { return delegate.buildEmptyAggregation(); } + @Override + public boolean runDeferredCollections() throws IOException { + Timer timer = profileBreakdown.getTimer(AggregationTimingType.RUN_DEFERRED_COLLECTIONS); + timer.start(); + try { + return delegate.runDeferredCollections(); + } finally { + timer.stop(); + } + } + @Override public LeafBucketCollector getLeafCollector(LeafReaderContext ctx) throws IOException { return new ProfilingLeafBucketCollector(delegate.getLeafCollector(ctx), profileBreakdown); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/LongKeyedBucketOrdsTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/LongKeyedBucketOrdsTests.java new file mode 100644 index 0000000000000..137c9abf98414 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/LongKeyedBucketOrdsTests.java @@ -0,0 +1,175 @@ +/* + * 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.aggregations.bucket.terms; + +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.MockBigArrays; +import org.elasticsearch.common.util.MockPageCacheRecycler; +import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; +import org.elasticsearch.test.ESTestCase; + +import java.util.HashSet; +import java.util.Objects; +import java.util.Set; + +import static org.hamcrest.Matchers.equalTo; + +public class LongKeyedBucketOrdsTests extends ESTestCase { + private final MockBigArrays bigArrays = new MockBigArrays(new MockPageCacheRecycler(Settings.EMPTY), new NoneCircuitBreakerService()); + + public void testExplicitCollectsFromSingleBucket() { + collectsFromSingleBucketCase(LongKeyedBucketOrds.build(bigArrays, true)); + } + + public void testSurpriseCollectsFromSingleBucket() { + collectsFromSingleBucketCase(LongKeyedBucketOrds.build(bigArrays, false)); + } + + private void collectsFromSingleBucketCase(LongKeyedBucketOrds ords) { + try { + // Test a few explicit values + assertThat(ords.add(0, 0), equalTo(0L)); + assertThat(ords.add(0, 1000), equalTo(1L)); + assertThat(ords.add(0, 0), equalTo(-1L)); + assertThat(ords.add(0, 1000), equalTo(-2L)); + + // And some random values + Set seen = new HashSet<>(); + seen.add(0L); + seen.add(1000L); + long[] values = new long[scaledRandomIntBetween(1, 10000)]; + for (int i = 0; i < values.length; i++) { + values[i] = randomValueOtherThanMany(seen::contains, ESTestCase::randomLong); + seen.add(values[i]); + } + for (int i = 0; i < values.length; i++) { + assertThat(ords.add(0, values[i]), equalTo(i + 2L)); + if (randomBoolean()) { + assertThat(ords.add(0, 0), equalTo(-1L)); + } + } + for (int i = 0; i < values.length; i++) { + assertThat(ords.add(0, values[i]), equalTo(-1 - (i + 2L))); + } + + // And the explicit values are still ok + assertThat(ords.add(0, 0), equalTo(-1L)); + assertThat(ords.add(0, 1000), equalTo(-2L)); + + LongKeyedBucketOrds.BucketOrdsEnum ordEnum = ords.ordsEnum(0); + assertTrue(ordEnum.next()); + assertThat(ordEnum.ord(), equalTo(0L)); + assertThat(ordEnum.value(), equalTo(0L)); + assertTrue(ordEnum.next()); + assertThat(ordEnum.ord(), equalTo(1L)); + assertThat(ordEnum.value(), equalTo(1000L)); + for (int i = 0; i < values.length; i++) { + assertTrue(ordEnum.next()); + assertThat(ordEnum.ord(), equalTo(i + 2L)); + assertThat(ordEnum.value(), equalTo(values[i])); + } + assertFalse(ordEnum.next()); + } finally { + ords.close(); + } + } + + public void testCollectsFromManyBuckets() { + try (LongKeyedBucketOrds ords = LongKeyedBucketOrds.build(bigArrays, false)) { + // Test a few explicit values + assertThat(ords.add(0, 0), equalTo(0L)); + assertThat(ords.add(1, 0), equalTo(1L)); + assertThat(ords.add(0, 0), equalTo(-1L)); + assertThat(ords.add(1, 0), equalTo(-2L)); + + // And some random values + Set seen = new HashSet<>(); + seen.add(new OwningBucketOrdAndValue(0, 0)); + seen.add(new OwningBucketOrdAndValue(1, 0)); + OwningBucketOrdAndValue[] values = new OwningBucketOrdAndValue[scaledRandomIntBetween(1, 10000)]; + long maxOwningBucketOrd = scaledRandomIntBetween(0, values.length); + for (int i = 0; i < values.length; i++) { + values[i] = randomValueOtherThanMany(seen::contains, () -> + new OwningBucketOrdAndValue(randomLongBetween(0, maxOwningBucketOrd), randomLong())); + seen.add(values[i]); + } + for (int i = 0; i < values.length; i++) { + assertThat(ords.add(values[i].owningBucketOrd, values[i].value), equalTo(i + 2L)); + if (randomBoolean()) { + assertThat(ords.add(0, 0), equalTo(-1L)); + } + } + for (int i = 0; i < values.length; i++) { + assertThat(ords.add(values[i].owningBucketOrd, values[i].value), equalTo(-1 - (i + 2L))); + } + + // And the explicit values are still ok + assertThat(ords.add(0, 0), equalTo(-1L)); + assertThat(ords.add(1, 0), equalTo(-2L)); + + for (long owningBucketOrd = 0; owningBucketOrd <= maxOwningBucketOrd; owningBucketOrd++) { + LongKeyedBucketOrds.BucketOrdsEnum ordEnum = ords.ordsEnum(owningBucketOrd); + if (owningBucketOrd <= 1) { + assertTrue(ordEnum.next()); + assertThat(ordEnum.ord(), equalTo(owningBucketOrd)); + assertThat(ordEnum.value(), equalTo(0L)); + } + for (int i = 0; i < values.length; i++) { + if (values[i].owningBucketOrd == owningBucketOrd) { + assertTrue(ordEnum.next()); + assertThat(ordEnum.ord(), equalTo(i + 2L)); + assertThat(ordEnum.value(), equalTo(values[i].value)); + } + } + assertFalse(ordEnum.next()); + } + assertFalse(ords.ordsEnum(randomLongBetween(maxOwningBucketOrd + 1, Long.MAX_VALUE)).next()); + } + } + + private class OwningBucketOrdAndValue { + private final long owningBucketOrd; + private final long value; + + OwningBucketOrdAndValue(long owningBucketOrd, long value) { + this.owningBucketOrd = owningBucketOrd; + this.value = value; + } + + @Override + public String toString() { + return owningBucketOrd + "/" + value; + } + + @Override + public boolean equals(Object obj) { + if (obj == null || getClass() != obj.getClass()) { + return false; + } + OwningBucketOrdAndValue other = (OwningBucketOrdAndValue) obj; + return owningBucketOrd == other.owningBucketOrd && value == other.value; + } + + @Override + public int hashCode() { + return Objects.hash(owningBucketOrd, value); + } + } +} diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java index 64dd51fdeafea..33dcf993e7c56 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java @@ -1291,6 +1291,37 @@ public void testNumberToStringValueScript() throws IOException { }, fieldType); } + public void testThreeLayerLong() throws IOException { + try (Directory dir = newDirectory()) { + try (RandomIndexWriter writer = new RandomIndexWriter(random(), dir)) { + Document d = new Document(); + for (int i = 0; i < 10; i++) { + for (int j = 0; j < 10; j++) { + for (int k = 0; k < 10; k++) { + d.add(new SortedNumericDocValuesField("i", i)); + d.add(new SortedNumericDocValuesField("j", j)); + d.add(new SortedNumericDocValuesField("k", k)); + writer.addDocument(d); + } + } + } + try (IndexReader reader = maybeWrapReaderEs(writer.getReader())) { + IndexSearcher searcher = newIndexSearcher(reader); + TermsAggregationBuilder request = new TermsAggregationBuilder("i").field("i") + .subAggregation(new TermsAggregationBuilder("j").field("j") + .subAggregation(new TermsAggregationBuilder("k").field("k"))); + InternalTerms result = search(searcher, new MatchAllDocsQuery(), request, + longField("i"), longField("j"), longField("k")); + } + } + } + } + + private NumberFieldMapper.NumberFieldType longField(String name) { + NumberFieldMapper.NumberFieldType type = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG); + type.setName(name); + return type; + } private void assertNestedTopHitsScore(InternalMultiBucketAggregation terms, boolean withScore) { assertThat(terms.getBuckets().size(), equalTo(9)); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/MaxAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/MaxAggregatorTests.java index 1e90a93180b51..db2b551289dd6 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/MaxAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/MaxAggregatorTests.java @@ -897,17 +897,17 @@ public void testNestedEarlyTermination() throws Exception { indexSearcher.search(new MatchAllDocsQuery(), bucketCollector); bucketCollector.postCollection(); - InternalMax max = (InternalMax) maxAggregator.buildAggregation(0L); + InternalMax max = (InternalMax) maxAggregator.buildTopLevel(); assertNotNull(max); assertEquals(12.0, max.getValue(), 0); assertEquals("max", max.getName()); - InternalValueCount count = (InternalValueCount) countAggregator.buildAggregation(0L); + InternalValueCount count = (InternalValueCount) countAggregator.buildTopLevel(); assertNotNull(count); assertEquals(20L, count.getValue()); assertEquals("count", count.getName()); - Terms terms = (Terms) termsAggregator.buildAggregation(0L); + Terms terms = (Terms) termsAggregator.buildTopLevel(); assertNotNull(terms); List buckets = terms.getBuckets(); assertNotNull(buckets); diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index 9989088878e6b..8bc73cbef2f4c 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -424,9 +424,9 @@ protected A search(IndexSe searcher.search(query, a); a.postCollection(); @SuppressWarnings("unchecked") - A internalAgg = (A) a.buildAggregation(0L); - InternalAggregationTestCase.assertMultiBucketConsumer(internalAgg, bucketConsumer); - return internalAgg; + A result = (A) a.buildTopLevel(); + InternalAggregationTestCase.assertMultiBucketConsumer(result, bucketConsumer); + return result; } protected A searchAndReduce(IndexSearcher searcher, @@ -494,7 +494,7 @@ protected A searchAndReduc a.preCollection(); subSearcher.search(weight, a); a.postCollection(); - InternalAggregation agg = a.buildAggregation(0L); + InternalAggregation agg = a.buildTopLevel(); aggs.add(agg); InternalAggregationTestCase.assertMultiBucketConsumer(agg, shardBucketConsumer); } From f3d9476117ddfdfc9cf51df6cda84622ae992595 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 28 Apr 2020 14:39:40 -0400 Subject: [PATCH 02/14] Fix funny test --- .../analytics/stringstats/StringStatsAggregatorTests.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/analytics/src/test/java/org/elasticsearch/xpack/analytics/stringstats/StringStatsAggregatorTests.java b/x-pack/plugin/analytics/src/test/java/org/elasticsearch/xpack/analytics/stringstats/StringStatsAggregatorTests.java index 6a762b9245fec..b115c0b31f5aa 100644 --- a/x-pack/plugin/analytics/src/test/java/org/elasticsearch/xpack/analytics/stringstats/StringStatsAggregatorTests.java +++ b/x-pack/plugin/analytics/src/test/java/org/elasticsearch/xpack/analytics/stringstats/StringStatsAggregatorTests.java @@ -101,7 +101,7 @@ protected void tes aggregator.postCollection(); @SuppressWarnings("unchecked") - V aggregation = (V) aggregator.buildAggregation(0L); + V aggregation = (V) aggregator.buildTopLevel(); verify.accept(aggregation); indexReader.close(); @@ -297,7 +297,7 @@ public void testNestedAggregation() throws IOException { indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - Terms terms = (Terms) aggregator.buildAggregation(0L); + Terms terms = (Terms) aggregator.buildTopLevel(); assertNotNull(terms); List buckets = terms.getBuckets(); assertNotNull(buckets); From aad9c1a2adca61ea3469ebbcc82b82c5728d8e11 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 28 Apr 2020 17:07:31 -0400 Subject: [PATCH 03/14] Oh rollup --- .../xpack/rollup/RollupResponseTranslationTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/RollupResponseTranslationTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/RollupResponseTranslationTests.java index 32aa68e6216eb..ae93dde5d05b8 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/RollupResponseTranslationTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/RollupResponseTranslationTests.java @@ -1357,7 +1357,7 @@ private InternalAggregation doQuery(Query query, aggregator.preCollection(); indexSearcher.search(query, aggregator); aggregator.postCollection(); - return aggregator.buildAggregation(0L); + return aggregator.buildTopLevel(); } finally { indexReader.close(); directory.close(); From 9f29f36ebfc1a29cf27c0dc6c8c902b8d9d9cf6d Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 29 Apr 2020 14:16:14 -0400 Subject: [PATCH 04/14] WIP --- .../ChildrenToParentAggregator.java | 3 +- .../ParentToChildrenAggregator.java | 3 +- server/hs_err_pid55263.log | 21 +++ .../search/aggregations/Aggregator.java | 15 +- .../bucket/BestBucketsDeferringCollector.java | 15 ++ .../bucket/BucketsAggregator.java | 30 +++- .../bucket/DeferableBucketAggregator.java | 149 +----------------- .../adjacency/AdjacencyMatrixAggregator.java | 69 +++++--- .../bucket/geogrid/GeoGridAggregator.java | 11 +- .../bucket/terms/LongTermsAggregator.java | 119 +++++++------- .../bucket/terms/StringTermsAggregator.java | 10 +- 11 files changed, 209 insertions(+), 236 deletions(-) create mode 100644 server/hs_err_pid55263.log diff --git a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregator.java b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregator.java index 82bbee4184f82..9bef000b880ad 100644 --- a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregator.java +++ b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregator.java @@ -46,8 +46,9 @@ public ChildrenToParentAggregator(String name, AggregatorFactories factories, @Override public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { + assert owningBucketOrdinal == 0; return new InternalParent(name, bucketDocCount(owningBucketOrdinal), - bucketAggregations(owningBucketOrdinal), metadata()); + buildSubAggsForBuckets(new long [0])[0], metadata()); } @Override diff --git a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregator.java b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregator.java index f37424223c60c..5a0fe890eb0bb 100644 --- a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregator.java +++ b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregator.java @@ -42,8 +42,9 @@ public ParentToChildrenAggregator(String name, AggregatorFactories factories, @Override public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { + assert owningBucketOrdinal == 0; return new InternalChildren(name, bucketDocCount(owningBucketOrdinal), - bucketAggregations(owningBucketOrdinal), metadata()); + buildSubAggsForBuckets(new long[0])[0], metadata()); } @Override diff --git a/server/hs_err_pid55263.log b/server/hs_err_pid55263.log new file mode 100644 index 0000000000000..83ea30bb2f4e3 --- /dev/null +++ b/server/hs_err_pid55263.log @@ -0,0 +1,21 @@ +# +# A fatal error has been detected by the Java Runtime Environment: +# +# SIGSEGV (0xb) at pc=0x00007fc8544e8d15, pid=55263, tid=55279 +# +# JRE version: OpenJDK Runtime Environment (13.0.2+8) (build 13.0.2+8) +# Java VM: OpenJDK 64-Bit Server VM (13.0.2+8, mixed mode, tiered, compressed oops, g1 gc, linux-amd64) +# Problematic frame: +# C [libjimage.so+0x2d15] +# +# Core dump will be written. Default location: Core dumps may be processed with "/usr/lib/systemd/systemd-coredump %P %u %g %s %t %c %h" (or dumping to /home/manybubbles/Code/Elastic/elasticsearch/server/core.55263) +# +# If you would like to submit a bug report, please visit: +# http://bugreport.java.com/bugreport/crash.jsp +# + +--------------- S U M M A R Y ------------ + +Command Line: -agentlib:jdwp=transport=dt_socket,suspend=y,address=localhost:46465 -ea -javaagent:/home/manybubbles/Bin/eclipse/java-2020-03/eclipse/configuration/org.eclipse.osgi/217/0/.cp/lib/javaagent-shaded.jar -Dfile.encoding=UTF-8 org.eclipse.jdt.internal.junit.runner.RemoteTestRunner -version 3 -port 44413 -testLoaderClass org.eclipse.jdt.internal.junit4.runner.JUnit4TestLoader -loaderpluginname org.eclipse.jdt.junit4.runtime -test org.elasticsearch.search.aggregations.bucket.terms.TermsAggregatorTests:testThreeLayerLong + +Host: \ No newline at end of file diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java index c6514226955df..90e1840822faa 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java @@ -33,6 +33,7 @@ import org.elasticsearch.search.sort.SortOrder; import java.io.IOException; +import java.util.Arrays; import java.util.Iterator; /** @@ -167,7 +168,17 @@ public interface BucketComparator { * generally accomplished by calling {@link #buildTopLevel()} on each top * level aggregator. */ - public abstract InternalAggregation buildAggregation(long owningBucketOrd) throws IOException; + public InternalAggregation buildAggregation(long owningBucketOrd) throws IOException { + throw new UnsupportedOperationException(); + } + + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + InternalAggregation[] results = new InternalAggregation[owningBucketOrds.length]; + for (int o = 0; o < owningBucketOrds.length; o++) { + results[o] = buildAggregation(owningBucketOrds[o]); + } + return results; + } /** * Build the result of this aggregation if it is on top level of the @@ -181,7 +192,7 @@ public interface BucketComparator { */ public final InternalAggregation buildTopLevel() throws IOException { assert parent() == null; - InternalAggregation result = buildAggregation(0); + InternalAggregation result = buildAggregations(new long[0])[0]; if (runDeferredCollections()) { return result.undefer(); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java index 79d8496ee4934..fc03d8fca0b48 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java @@ -221,6 +221,21 @@ public InternalAggregation buildAggregation(long bucket) throws IOException { return in.buildAggregation(rebasedBucket); } + @Override + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + if (selectedBuckets == null) { + throw new IllegalStateException("Collection has not been replayed yet."); + } + assert owningBucketOrds.length == selectedBuckets.size(); + long[] rebasedOrds = new long[owningBucketOrds.length]; + for (int ord = 0; ord < owningBucketOrds.length; ord++) { + final long rebasedBucket = selectedBuckets.find(owningBucketOrds[ord]); + if (rebasedBucket == -1) { + throw new IllegalStateException("Cannot build for a bucket which has not been collected"); + } + } + return in.buildAggregations(rebasedOrds); + } }; } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java index da33eeebddf14..8623b44e6146c 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java @@ -32,6 +32,7 @@ import org.elasticsearch.search.sort.SortOrder; import java.io.IOException; +import java.util.AbstractList; import java.util.Arrays; import java.util.Iterator; import java.util.Map; @@ -132,18 +133,39 @@ public final int bucketDocCount(long bucketOrd) { * the maximum number of buckets allowed in a response */ protected final void consumeBucketsAndMaybeBreak(int count) { + // NOCOMMIT do we need to keep this? multiBucketConsumer.accept(count); } /** * Required method to build the child aggregations of the given bucket (identified by the bucket ordinal). */ - protected final InternalAggregations bucketAggregations(long bucket) throws IOException { - final InternalAggregation[] aggregations = new InternalAggregation[subAggregators.length]; + protected final InternalAggregations[] buildSubAggsForBuckets(long[] ordsToCollect) throws IOException { + consumeBucketsAndMaybeBreak(ordsToCollect.length); + InternalAggregation[][] aggregations = new InternalAggregation[subAggregators.length][]; for (int i = 0; i < subAggregators.length; i++) { - aggregations[i] = subAggregators[i].buildAggregation(bucket); + aggregations[i] = subAggregators[i].buildAggregations(ordsToCollect); } - return new InternalAggregations(Arrays.asList(aggregations)); + InternalAggregations[] result = new InternalAggregations[ordsToCollect.length]; + for (int ord = 0; ord < ordsToCollect.length; ord++) { + InternalAggregation[] slice = new InternalAggregation[subAggregators.length]; + for (int i = 0; i < subAggregators.length; i++) { + slice[i] = aggregations[i][ord]; + } + final int thisOrd = ord; + result[ord] = new InternalAggregations(new AbstractList() { + @Override + public InternalAggregation get(int index) { + return aggregations[index][thisOrd]; + } + + @Override + public int size() { + return aggregations.length; + } + }); + } + return result; } /** diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferableBucketAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferableBucketAggregator.java index 325bfa5c7931e..448b5e6d81521 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferableBucketAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferableBucketAggregator.java @@ -29,6 +29,7 @@ import org.elasticsearch.search.aggregations.AggregatorFactory.MultiBucketAggregatorWrapper; import org.elasticsearch.search.aggregations.BucketCollector; import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.MultiBucketCollector; import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregator; import org.elasticsearch.search.internal.SearchContext; @@ -44,23 +45,6 @@ public abstract class DeferableBucketAggregator extends BucketsAggregator { * been deferred. */ private DeferringBucketCollector recordingWrapper; - /** - * Hash of surviving sub-aggregation ordinals. Non-null if there are any - * surviving ordinals. - */ - private LongHash survivingOrds; - /** - * List of deferred results to build when running deferred collections. - * Non-null if any results have been deferred. - *

- * Its kind of a shame to keep a {@linkplain List} around to keep the - * reference to these but we must build the results after - * running our deferred collections and before running our - * sub-aggregation's deferred collections. This is because it is building - * our results that marks which gives our sub-aggregators the chance to - * mark any of their buckets as surviving. - */ - private List deferred; protected DeferableBucketAggregator(String name, AggregatorFactories factories, SearchContext context, Aggregator parent, Map metadata) throws IOException { @@ -139,132 +123,11 @@ protected final void runDeferredCollections(long... bucketOrds) throws IOExcepti } } - @Override - public final boolean runDeferredCollections() throws IOException { - if (survivingOrds == null) { - /* - * If there aren't any ords that need to be replayed we just - * function as a regular aggregator, giving our sub-aggregation's - * a chance to run deferred collections if they need to. - */ - return super.runDeferredCollections(); - } - /* - * If we have any deferred collections we reply them and then - * build our results. Building our results will have the side - * effect of marking any of the our children's surviving buckets - * for replay. So, after that, we can finally run our children's - * deferred collections by calling super.runDeferredCollections. - */ - recordingWrapper.prepareSelectedBuckets(survivingOrds); - for (DeferredInternalAggregation d : deferred) { - d.buildRealResult(); - } - super.runDeferredCollections(); - return true; - } - - /** - * Record an ordinal to be replayed before suppliers passed to - * {@link #deferred(IOSupplier)} are called. - */ - protected final void recordSurvingOrd(long ord) { - if (recordingWrapper == null) { - /* - * recording wrapper is null if no child aggregations have - * been delayed. - */ - return; - } - if (survivingOrds == null) { - survivingOrds = new LongHash(1, context.bigArrays()); - } - survivingOrds.add(ord); - } - - /** - * Build a "deferred" aggregation result. The provided supplier is called - * after all ordinals recorded with {@link #recordSurvingOrd(long)} are - * replayed on child aggregations. - */ - protected final InternalAggregation deferred(IOSupplier buildRealResult) throws IOException { - if (survivingOrds == null) { - /* - * No aggregations have been delayed so we are safe to finish - * building the result right away. - */ - return buildRealResult.get(); - } - if (deferred == null) { - deferred = new ArrayList<>(); - } - DeferredInternalAggregation d = new DeferredInternalAggregation(name, buildRealResult); - deferred.add(d); - return d; - } - - @Override - protected void doClose() { - super.doClose(); - Releasables.close(survivingOrds); - } - - private static class DeferredInternalAggregation extends InternalAggregation { - private final IOSupplier buildRealResult; - private InternalAggregation realResult; - - DeferredInternalAggregation(String name, IOSupplier buildRealResult) { - super(name, null); - this.buildRealResult = buildRealResult; - } - - @Override - public String toString() { - if (realResult == null) { - return "deferred [" + name + "/" + buildRealResult + "]"; - } else { - return realResult.toString(); - } - } - - private void buildRealResult() throws IOException { - realResult = buildRealResult.get(); - if (realResult == null) { - throw new IllegalStateException("undeferring bucket didn't build anything"); - } - } - - @Override - public InternalAggregation undefer() { - if (realResult == null) { - throw new IllegalStateException("deferred collections not replayed!"); - } - return realResult.undefer(); - } - - @Override - public String getWriteableName() { - throw new IllegalStateException(); - } - - @Override - protected void doWriteTo(StreamOutput out) throws IOException { - throw new IllegalStateException(); - } - - @Override - public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) { - throw new IllegalStateException(); - } - - @Override - public Object getProperty(List path) { - throw new IllegalStateException(); - } - - @Override - public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { - throw new IllegalStateException(); + protected final InternalAggregations[] runDeferredCollections(LongHash bucketOrds) throws IOException { + // NOCOMMIT maybe remove this entirely and just piggy back on building buckets? + if (recordingWrapper != null) { + recordingWrapper.prepareSelectedBuckets(bucketOrds); } + long[] } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregator.java index f019fb1c3d0c8..2e6a20a6b08d8 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregator.java @@ -35,6 +35,7 @@ import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; @@ -174,41 +175,59 @@ public void collect(int doc, long bucket) throws IOException { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + InternalAggregation[] results = new InternalAggregation[owningBucketOrds.length]; // Buckets are ordered into groups - [keyed filters] [key1&key2 intersects] - - List buckets = new ArrayList<>(filters.length); - for (int i = 0; i < keys.length; i++) { - long bucketOrd = bucketOrd(owningBucketOrdinal, i); - int docCount = bucketDocCount(bucketOrd); - // Empty buckets are not returned because this aggregation will commonly be used under a - // a date-histogram where we will look for transactions over time and can expect many - // empty buckets. - if (docCount > 0) { - InternalAdjacencyMatrix.InternalBucket bucket = new InternalAdjacencyMatrix.InternalBucket(keys[i], - docCount, bucketAggregations(bucketOrd)); - buckets.add(bucket); - consumeBucketsAndMaybeBreak(1); + int maxOrd = owningBucketOrds.length * totalNumKeys; + int totalBucketsToBuild = 0; + for (int ord = 0; ord < maxOrd; ord++) { + if (bucketDocCount(ord) > 0) { + totalBucketsToBuild++; } } - int pos = keys.length; - for (int i = 0; i < keys.length; i++) { - for (int j = i + 1; j < keys.length; j++) { - long bucketOrd = bucketOrd(owningBucketOrdinal, pos); + long[] bucketOrdsToBuild = new long[totalBucketsToBuild]; + int builtBucketIndex = 0; + for (int ord = 0; ord < maxOrd; ord++) { + if (bucketDocCount(ord) > 0) { + bucketOrdsToBuild[builtBucketIndex++] = ord; + } + } + + builtBucketIndex = 0; + List buckets = new ArrayList<>(filters.length); + InternalAggregations[] bucketSubAggs = buildSubAggsForBuckets(bucketOrdsToBuild); + for (int ord = 0; ord < owningBucketOrds.length; ord++) { + for (int i = 0; i < keys.length; i++) { + long bucketOrd = bucketOrd(owningBucketOrds[ord], i); int docCount = bucketDocCount(bucketOrd); - // Empty buckets are not returned due to potential for very sparse matrices + // Empty buckets are not returned because this aggregation will commonly be used under a + // a date-histogram where we will look for transactions over time and can expect many + // empty buckets. if (docCount > 0) { - String intersectKey = keys[i] + separator + keys[j]; - InternalAdjacencyMatrix.InternalBucket bucket = new InternalAdjacencyMatrix.InternalBucket(intersectKey, - docCount, bucketAggregations(bucketOrd)); + InternalAdjacencyMatrix.InternalBucket bucket = new InternalAdjacencyMatrix.InternalBucket(keys[i], + docCount, bucketSubAggs[builtBucketIndex++]); buckets.add(bucket); - consumeBucketsAndMaybeBreak(1); } - pos++; } + int pos = keys.length; + for (int i = 0; i < keys.length; i++) { + for (int j = i + 1; j < keys.length; j++) { + long bucketOrd = bucketOrd(owningBucketOrds[ord], pos); + int docCount = bucketDocCount(bucketOrd); + // Empty buckets are not returned due to potential for very sparse matrices + if (docCount > 0) { + String intersectKey = keys[i] + separator + keys[j]; + InternalAdjacencyMatrix.InternalBucket bucket = new InternalAdjacencyMatrix.InternalBucket(intersectKey, + docCount, bucketSubAggs[builtBucketIndex++]); + buckets.add(bucket); + } + pos++; + } + } + results[ord] = new InternalAdjacencyMatrix(name, buckets, metadata()); } - return new InternalAdjacencyMatrix(name, buckets, metadata()); + return results; } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregator.java index 95be75c49f45f..1aec319f74cce 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregator.java @@ -25,6 +25,7 @@ import org.elasticsearch.common.util.LongHash; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; @@ -124,11 +125,15 @@ public InternalGeoGrid buildAggregation(long owningBucketOrdinal) throws IOExcep spare = ordered.insertWithOverflow(spare); } + long[] ordsToCollect = new long[ordered.size()]; final InternalGeoGridBucket[] list = new InternalGeoGridBucket[ordered.size()]; for (int i = ordered.size() - 1; i >= 0; --i) { - final InternalGeoGridBucket bucket = ordered.pop(); - bucket.aggregations = bucketAggregations(bucket.bucketOrd); - list[i] = bucket; + list[i] = ordered.pop(); + ordsToCollect[i] = list[i].bucketOrd; + } + InternalAggregations[] sub = buildSubAggsForBuckets(ordsToCollect); + for (int i = 0; i < list.length; i++) { + list[i].aggregations = sub[i]; } return buildAggregation(name, requiredSize, Arrays.asList(list), metadata()); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java index 6f2bf359b8f7f..1bbed26c7c8fb 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java @@ -22,11 +22,13 @@ import org.apache.lucene.index.SortedNumericDocValues; import org.apache.lucene.search.ScoreMode; import org.elasticsearch.common.lease.Releasables; +import org.elasticsearch.common.util.LongHash; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.BucketOrder; import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalOrder; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; @@ -106,68 +108,79 @@ public void collect(int doc, long owningBucketOrd) throws IOException { } @Override - public InternalAggregation buildAggregation(long owningBucketOrd) throws IOException { - long bucketsInOrd = bucketOrds.bucketsInOrd(owningBucketOrd); - if (bucketCountThresholds.getMinDocCount() == 0 && (InternalOrder.isCountDesc(order) == false || - bucketsInOrd < bucketCountThresholds.getRequiredSize())) { - // we need to fill-in the blanks - for (LeafReaderContext ctx : context.searcher().getTopReaderContext().leaves()) { - final SortedNumericDocValues values = getValues(valuesSource, ctx); - for (int docId = 0; docId < ctx.reader().maxDoc(); ++docId) { - if (values.advanceExact(docId)) { - final int valueCount = values.docValueCount(); - for (int i = 0; i < valueCount; ++i) { - long value = values.nextValue(); - if (longFilter == null || longFilter.accept(value)) { - bucketOrds.add(owningBucketOrd, value); + public InternalAggregation[] buildAggregations(int owningBucketOrdsToCollect) throws IOException { + try (LongHash survivingOrds = new LongHash(owningBucketOrdsToCollect * bucketCountThresholds.getShardSize(), context.bigArrays())) { + LongTerms.Bucket[][] topBuckets = new LongTerms.Bucket[owningBucketOrdsToCollect][]; + long[] otherDocCounts = new long[owningBucketOrdsToCollect]; + for (int owningBucketOrd = 0; owningBucketOrd < owningBucketOrdsToCollect; owningBucketOrd++) { + long bucketsInOrd = bucketOrds.bucketsInOrd(owningBucketOrd); + if (bucketCountThresholds.getMinDocCount() == 0 && (InternalOrder.isCountDesc(order) == false || + bucketsInOrd < bucketCountThresholds.getRequiredSize())) { + // we need to fill-in the blanks + for (LeafReaderContext ctx : context.searcher().getTopReaderContext().leaves()) { + final SortedNumericDocValues values = getValues(valuesSource, ctx); + for (int docId = 0; docId < ctx.reader().maxDoc(); ++docId) { + if (values.advanceExact(docId)) { + final int valueCount = values.docValueCount(); + for (int i = 0; i < valueCount; ++i) { + long value = values.nextValue(); + if (longFilter == null || longFilter.accept(value)) { + bucketOrds.add(owningBucketOrd, value); + } + } } } } + bucketsInOrd = bucketOrds.bucketsInOrd(owningBucketOrd); } - } - bucketsInOrd = bucketOrds.bucketsInOrd(owningBucketOrd); - } - - final int size = (int) Math.min(bucketsInOrd, bucketCountThresholds.getShardSize()); - long otherDocCount = 0; - BucketPriorityQueue ordered = new BucketPriorityQueue<>(size, partiallyBuiltBucketComparator); - LongTerms.Bucket spare = null; - BucketOrdsEnum ordsEnum = bucketOrds.ordsEnum(owningBucketOrd); - while (ordsEnum.next()) { - if (spare == null) { - spare = new LongTerms.Bucket(0, 0, null, showTermDocCountError, 0, format); - } - spare.term = ordsEnum.value(); - spare.docCount = bucketDocCount(ordsEnum.ord()); - otherDocCount += spare.docCount; - spare.bucketOrd = ordsEnum.ord(); - if (bucketCountThresholds.getShardMinDocCount() <= spare.docCount) { - spare = ordered.insertWithOverflow(spare); - if (spare == null) { - consumeBucketsAndMaybeBreak(1); + + final int size = (int) Math.min(bucketsInOrd, bucketCountThresholds.getShardSize()); + BucketPriorityQueue ordered = new BucketPriorityQueue<>(size, partiallyBuiltBucketComparator); + LongTerms.Bucket spare = null; + BucketOrdsEnum ordsEnum = bucketOrds.ordsEnum(owningBucketOrd); + while (ordsEnum.next()) { + if (spare == null) { + spare = new LongTerms.Bucket(0, 0, null, showTermDocCountError, 0, format); + } + spare.term = ordsEnum.value(); + spare.docCount = bucketDocCount(ordsEnum.ord()); + otherDocCounts[owningBucketOrd] += spare.docCount; + spare.bucketOrd = ordsEnum.ord(); + if (bucketCountThresholds.getShardMinDocCount() <= spare.docCount) { + spare = ordered.insertWithOverflow(spare); + if (spare == null) { + consumeBucketsAndMaybeBreak(1); + } + } + } + + // Get the top buckets + LongTerms.Bucket[] list = topBuckets[owningBucketOrd] = new LongTerms.Bucket[ordered.size()]; + for (int i = ordered.size() - 1; i >= 0; --i) { + final LongTerms.Bucket bucket = ordered.pop(); + bucket.bucketOrd = survivingOrds.add(bucket.bucketOrd); + if (bucket.bucketOrd < 0) { + bucket.bucketOrd = 1 - bucket.bucketOrd; + } + list[i] = bucket; + otherDocCounts[owningBucketOrd] -= bucket.docCount; } } - } - // Get the top buckets - final LongTerms.Bucket[] list = new LongTerms.Bucket[ordered.size()]; - for (int i = ordered.size() - 1; i >= 0; --i) { - final LongTerms.Bucket bucket = ordered.pop(); - recordSurvingOrd(bucket.bucketOrd); - list[i] = bucket; - otherDocCount -= bucket.docCount; - } + runDeferredCollections(survivingOrds); + InternalAggregations[] bucketAggs = buildSubAggsForBuckets(survivingOrds.size()); - // Return a deferred agg that will build the buckets when undeferred - final long finalOtherDocCount = otherDocCount; - return deferred(() -> { - for (int i = 0; i < list.length; i++) { - list[i].aggregations = bucketAggregations(list[i].bucketOrd); - list[i].docCountError = 0; - } + InternalAggregation[] result = new InternalAggregation[owningBucketOrdsToCollect]; + for (int owningBucketOrd = 0; owningBucketOrd < owningBucketOrdsToCollect; owningBucketOrd++) { + for (int i = 0; i < topBuckets[0].length; i++) { + topBuckets[owningBucketOrd][i].aggregations = bucketAggs[(int) survivingOrds.find(topBuckets[0][i].bucketOrd)]; + topBuckets[owningBucketOrd][i].docCountError = 0; + } - return buildResult(finalOtherDocCount, Arrays.asList(list)); - }); + result[owningBucketOrd] = buildResult(otherDocCounts[owningBucketOrd], Arrays.asList(topBuckets[owningBucketOrd])); + } + return result; + } } protected InternalAggregation buildResult(long otherDocCount, List buckets) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java index aebff761636a7..22fe8644f884e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java @@ -30,6 +30,7 @@ import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.BucketOrder; import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalOrder; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; @@ -164,12 +165,13 @@ public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOE } // replay any deferred collections runDeferredCollections(survivingBucketOrds); + InternalAggregations[] sub = buildSubAggsForBuckets(survivingBucketOrds); // Now build the aggs - for (final StringTerms.Bucket bucket : list) { - bucket.termBytes = BytesRef.deepCopyOf(bucket.termBytes); - bucket.aggregations = bucketAggregations(bucket.bucketOrd); - bucket.docCountError = 0; + for (int i = 0; i < list.length; i++) { + list[i].termBytes = BytesRef.deepCopyOf(list[i].termBytes); + list[i].aggregations = sub[i]; + list[i].docCountError = 0; } return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), From 1f5ed9d33fc64d647e42904b17984578f0449dfb Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 29 Apr 2020 16:38:38 -0400 Subject: [PATCH 05/14] huh --- .../ChildrenToParentAggregator.java | 7 +- .../ParentToChildrenAggregator.java | 7 +- server/hs_err_pid55263.log | 21 -- .../search/aggregations/AggregationPhase.java | 2 +- .../search/aggregations/Aggregator.java | 60 ++---- .../search/aggregations/AggregatorBase.java | 9 - .../aggregations/AggregatorFactory.java | 33 ++- .../aggregations/NonCollectingAggregator.java | 8 +- .../bucket/BestBucketsDeferringCollector.java | 7 +- .../bucket/BucketsAggregator.java | 191 +++++++++++++++++- .../bucket/DeferableBucketAggregator.java | 40 +--- .../bucket/DeferringBucketCollector.java | 13 +- .../adjacency/AdjacencyMatrixAggregator.java | 17 +- .../bucket/composite/CompositeAggregator.java | 25 ++- .../bucket/filter/FilterAggregator.java | 5 +- .../bucket/filter/FiltersAggregator.java | 26 +-- .../bucket/geogrid/GeoGridAggregator.java | 15 +- .../bucket/global/GlobalAggregator.java | 8 +- .../AutoDateHistogramAggregator.java | 42 ++-- .../histogram/DateHistogramAggregator.java | 37 ++-- .../DateRangeHistogramAggregator.java | 36 ++-- .../histogram/NumericHistogramAggregator.java | 36 ++-- .../histogram/RangeHistogramAggregator.java | 38 ++-- .../bucket/missing/MissingAggregator.java | 5 +- .../bucket/nested/NestedAggregator.java | 7 +- .../nested/ReverseNestedAggregator.java | 5 +- .../bucket/range/BinaryRangeAggregator.java | 17 +- .../bucket/range/RangeAggregator.java | 19 +- .../bucket/sampler/SamplerAggregator.java | 7 +- ...balOrdinalsSignificantTermsAggregator.java | 32 ++- .../SignificantLongTermsAggregator.java | 26 +-- .../SignificantStringTermsAggregator.java | 25 +-- .../SignificantTextAggregator.java | 17 +- .../GlobalOrdinalsStringTermsAggregator.java | 24 +-- .../bucket/terms/LongRareTermsAggregator.java | 13 +- .../bucket/terms/LongTermsAggregator.java | 110 +++++----- .../terms/StringRareTermsAggregator.java | 14 +- .../bucket/terms/StringTermsAggregator.java | 26 +-- .../metrics/MetricsAggregator.java | 10 + .../aggregation/AggregationTimingType.java | 1 - .../aggregation/ProfilingAggregator.java | 17 +- .../bucket/terms/TermsAggregatorTests.java | 18 +- 42 files changed, 542 insertions(+), 534 deletions(-) delete mode 100644 server/hs_err_pid55263.log diff --git a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregator.java b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregator.java index 9bef000b880ad..4e960fa0857be 100644 --- a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregator.java +++ b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregator.java @@ -45,10 +45,9 @@ public ChildrenToParentAggregator(String name, AggregatorFactories factories, } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - assert owningBucketOrdinal == 0; - return new InternalParent(name, bucketDocCount(owningBucketOrdinal), - buildSubAggsForBuckets(new long [0])[0], metadata()); + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + return buildAggregationsForSingleBucket(owningBucketOrds, (owningBucketOrd, subAggregationResults) -> + new InternalParent(name, bucketDocCount(owningBucketOrd), subAggregationResults, metadata())); } @Override diff --git a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregator.java b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregator.java index 5a0fe890eb0bb..f299e0c603e1b 100644 --- a/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregator.java +++ b/modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregator.java @@ -41,10 +41,9 @@ public ParentToChildrenAggregator(String name, AggregatorFactories factories, } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - assert owningBucketOrdinal == 0; - return new InternalChildren(name, bucketDocCount(owningBucketOrdinal), - buildSubAggsForBuckets(new long[0])[0], metadata()); + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + return buildAggregationsForSingleBucket(owningBucketOrds, (owningBucketOrd, subAggregationResults) -> + new InternalChildren(name, bucketDocCount(owningBucketOrd), subAggregationResults, metadata())); } @Override diff --git a/server/hs_err_pid55263.log b/server/hs_err_pid55263.log deleted file mode 100644 index 83ea30bb2f4e3..0000000000000 --- a/server/hs_err_pid55263.log +++ /dev/null @@ -1,21 +0,0 @@ -# -# A fatal error has been detected by the Java Runtime Environment: -# -# SIGSEGV (0xb) at pc=0x00007fc8544e8d15, pid=55263, tid=55279 -# -# JRE version: OpenJDK Runtime Environment (13.0.2+8) (build 13.0.2+8) -# Java VM: OpenJDK 64-Bit Server VM (13.0.2+8, mixed mode, tiered, compressed oops, g1 gc, linux-amd64) -# Problematic frame: -# C [libjimage.so+0x2d15] -# -# Core dump will be written. Default location: Core dumps may be processed with "/usr/lib/systemd/systemd-coredump %P %u %g %s %t %c %h" (or dumping to /home/manybubbles/Code/Elastic/elasticsearch/server/core.55263) -# -# If you would like to submit a bug report, please visit: -# http://bugreport.java.com/bugreport/crash.jsp -# - ---------------- S U M M A R Y ------------ - -Command Line: -agentlib:jdwp=transport=dt_socket,suspend=y,address=localhost:46465 -ea -javaagent:/home/manybubbles/Bin/eclipse/java-2020-03/eclipse/configuration/org.eclipse.osgi/217/0/.cp/lib/javaagent-shaded.jar -Dfile.encoding=UTF-8 org.eclipse.jdt.internal.junit.runner.RemoteTestRunner -version 3 -port 44413 -testLoaderClass org.eclipse.jdt.internal.junit4.runner.JUnit4TestLoader -loaderpluginname org.eclipse.jdt.junit4.runtime -test org.elasticsearch.search.aggregations.bucket.terms.TermsAggregatorTests:testThreeLayerLong - -Host: \ No newline at end of file diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java b/server/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java index 168880ee77241..5808ddf2aa5df 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java @@ -125,7 +125,7 @@ public void execute(SearchContext context) { for (Aggregator aggregator : context.aggregations().aggregators()) { try { aggregator.postCollection(); - aggregations.add(aggregator.buildTopLevel()); + aggregations.add(aggregator.buildAggregations(new long[] {0})[0]); } catch (IOException e) { throw new AggregationExecutionException("Failed to build aggregation [" + aggregator.name() + "]", e); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java index 90e1840822faa..7bb4a3d64669b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java @@ -33,7 +33,6 @@ import org.elasticsearch.search.sort.SortOrder; import java.io.IOException; -import java.util.Arrays; import java.util.Iterator; /** @@ -155,48 +154,33 @@ public interface BucketComparator { /** * Build an aggregation for data that has been collected into * {@code owningBucketOrd}. - *

- * Bucketing aggregations sometimes delay collecting their results if they - * are selective (like {@code terms}) or if they aren't sure which buckets - * their documents will ultimately fall into - * (like {@code auto_date_histogram}). If they do so then anything they - * return from this method can't be trusted. To turn them into "real" - * results you have to make all of the calls to this method that you - * will ever make, then call {@link #runDeferredCollections()}. If that - * returns {@code true} then every aggregation will need to be rewritten - * with {@code InternalAggregation#undefer()}. This entire dance is - * generally accomplished by calling {@link #buildTopLevel()} on each top - * level aggregator. + * @deprecated use {@link #buildAggregations(long[])} instead */ + @Deprecated public InternalAggregation buildAggregation(long owningBucketOrd) throws IOException { - throw new UnsupportedOperationException(); - } - - public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { - InternalAggregation[] results = new InternalAggregation[owningBucketOrds.length]; - for (int o = 0; o < owningBucketOrds.length; o++) { - results[o] = buildAggregation(owningBucketOrds[o]); + /* + * Temporarily check if it looks like we're being called from a test + * and try to answer with the top level agg. This just prevents us from + * having to modify 1231234134124 tests in one PR. + */ + if (owningBucketOrd == 0) { + return buildTopLevel(); } - return results; + throw new UnsupportedOperationException(); } /** - * Build the result of this aggregation if it is on top level of the - * aggregation tree, properly handling deferred collections. It is only - * correct to call this on aggregations that are at the - * top level of the aggregation tree. Calling it for other aggregations - * will trip assertions or return the wrong result. Those aggregtions will - * have {@link #buildAggregation(long)}, {@link #runDeferredCollections()} - * and {@link InternalAggregation#undefer()} called by their parent - * aggregations. + * Build the results of this aggregation. + * @param owningBucketOrds the ordinals of the buckets that we want to + * collect from this aggregation + * @return the results for each ordinal, in the same order as the array + * of ordinals */ - public final InternalAggregation buildTopLevel() throws IOException { + public abstract InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException; + + public InternalAggregation buildTopLevel() throws IOException { assert parent() == null; - InternalAggregation result = buildAggregations(new long[0])[0]; - if (runDeferredCollections()) { - return result.undefer(); - } - return result; + return buildAggregations(new long[] {0})[0]; } /** @@ -204,12 +188,6 @@ public final InternalAggregation buildTopLevel() throws IOException { */ public abstract InternalAggregation buildEmptyAggregation(); - /** - * Run any deferred collections that are required to - * {@link InternalAggregation#undefer()} this aggregations's results. - */ - public abstract boolean runDeferredCollections() throws IOException; - /** Aggregation mode for sub aggregations. */ public enum SubAggCollectionMode implements Writeable { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorBase.java b/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorBase.java index 3265cf396f3fb..11ec88a8ce8bd 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorBase.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorBase.java @@ -254,15 +254,6 @@ protected void doClose() {} protected void doPostCollection() throws IOException { } - @Override - public boolean runDeferredCollections() throws IOException { - boolean ran = false; - for (Aggregator sub : subAggregators) { - ran |= sub.runDeferredCollections(); - } - return ran; - } - protected final InternalAggregations buildEmptySubAggregations() { List aggs = new ArrayList<>(); for (Aggregator aggregator : subAggregators) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java index 536547eabf2fc..7618b78e23383 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java @@ -144,14 +144,21 @@ public void collect(int doc, long bucket) throws IOException { } @Override - public InternalAggregation buildAggregation(long bucket) throws IOException { - if (bucket < aggregators.size()) { - Aggregator aggregator = aggregators.get(bucket); - if (aggregator != null) { - return aggregator.buildAggregation(0); + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + InternalAggregation[] results = new InternalAggregation[owningBucketOrds.length]; + for (int ordIdx = 0; ordIdx < owningBucketOrds.length; ordIdx++) { + if (owningBucketOrds[ordIdx] < aggregators.size()) { + Aggregator aggregator = aggregators.get(owningBucketOrds[ordIdx]); + if (aggregator != null) { + results[ordIdx] = aggregator.buildAggregations(new long [] {0})[0]; + } else { + results[ordIdx] = buildEmptyAggregation(); + } + } else { + results[ordIdx] = buildEmptyAggregation(); } } - return buildEmptyAggregation(); + return results; } @Override @@ -159,18 +166,6 @@ public InternalAggregation buildEmptyAggregation() { return first.buildEmptyAggregation(); } - @Override - public boolean runDeferredCollections() throws IOException { - boolean ran = false; - for (long i = 0; i < aggregators.size(); ++i) { - final Aggregator aggregator = aggregators.get(i); - if (aggregator != null) { - ran |= aggregator.runDeferredCollections(); - } - } - return ran; - } - @Override public void close() { Releasables.close(aggregators, collectors); @@ -242,7 +237,9 @@ public AggregatorFactory getParent() { * Utility method. Given an {@link AggregatorFactory} that creates * {@link Aggregator}s that only know how to collect bucket {@code 0}, this * returns an aggregator that can collect any bucket. + * @deprecated implement the aggregator to handle many owning buckets */ + @Deprecated protected static Aggregator asMultiBucketAggregator(final AggregatorFactory factory, final SearchContext searchContext, final Aggregator parent) throws IOException { final Aggregator first = factory.create(searchContext, parent, true); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/NonCollectingAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/NonCollectingAggregator.java index 4573b93bde014..caa9d54f2fa6f 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/NonCollectingAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/NonCollectingAggregator.java @@ -48,7 +48,11 @@ public final LeafBucketCollector getLeafCollector(LeafReaderContext reader, Leaf } @Override - public final InternalAggregation buildAggregation(long owningBucketOrdinal) { - return buildEmptyAggregation(); + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + InternalAggregation[] results = new InternalAggregation[owningBucketOrds.length]; + for (int ordIdx = 0; ordIdx < owningBucketOrds.length; ordIdx++) { + results[ordIdx] = buildEmptyAggregation(); + } + return results; } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java index fc03d8fca0b48..0fd555e83e998 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java @@ -226,11 +226,10 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I if (selectedBuckets == null) { throw new IllegalStateException("Collection has not been replayed yet."); } - assert owningBucketOrds.length == selectedBuckets.size(); long[] rebasedOrds = new long[owningBucketOrds.length]; - for (int ord = 0; ord < owningBucketOrds.length; ord++) { - final long rebasedBucket = selectedBuckets.find(owningBucketOrds[ord]); - if (rebasedBucket == -1) { + for (int ordIdx = 0; ordIdx < owningBucketOrds.length; ordIdx++) { + rebasedOrds[ordIdx] = selectedBuckets.find(owningBucketOrds[ordIdx]); + if (rebasedOrds[ordIdx] == -1) { throw new IllegalStateException("Cannot build for a bucket which has not been collected"); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java index 8623b44e6146c..b170c049889bf 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java @@ -21,9 +21,11 @@ import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.IntArray; +import org.elasticsearch.common.util.LongHash; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorBase; import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.LeafBucketCollector; @@ -33,10 +35,15 @@ import java.io.IOException; import java.util.AbstractList; +import java.util.ArrayList; import java.util.Arrays; import java.util.Iterator; +import java.util.List; import java.util.Map; +import java.util.function.BiConsumer; +import java.util.function.Function; import java.util.function.IntConsumer; +import java.util.function.ToLongFunction; public abstract class BucketsAggregator extends AggregatorBase { @@ -133,21 +140,35 @@ public final int bucketDocCount(long bucketOrd) { * the maximum number of buckets allowed in a response */ protected final void consumeBucketsAndMaybeBreak(int count) { - // NOCOMMIT do we need to keep this? multiBucketConsumer.accept(count); } /** - * Required method to build the child aggregations of the given bucket (identified by the bucket ordinal). + * Hook to allow taking an action before building buckets. */ - protected final InternalAggregations[] buildSubAggsForBuckets(long[] ordsToCollect) throws IOException { - consumeBucketsAndMaybeBreak(ordsToCollect.length); + protected void beforeBuildingBuckets(long[] ordsToCollect) throws IOException {} + + /** + * Build the results of the sub-aggregations of the buckets at each of + * the provided ordinals. + *

+ * Most aggregations should probably use something like + * {@link #buildSubAggsForAllBuckets(Object[][], ToLongFunction, BiConsumer)} + * or {@link #buildAggregationsForVariableBuckets(long[], LongHash, BucketBuilderForVariable, Function)} + * or {@link #buildAggregationsForFixedBucketCount(long[], int, BucketBuilderForFixedCount, Function)} + * or {@link #buildAggregationsForSingleBucket(long[], SingleBucketResultBuilder)} + * instead of calling this directly. + * @return the sub-aggregation results in the same order as the provided + * array of ordinals + */ + protected final InternalAggregations[] buildSubAggsForBuckets(long[] bucketOrdsToCollect) throws IOException { + beforeBuildingBuckets(bucketOrdsToCollect); InternalAggregation[][] aggregations = new InternalAggregation[subAggregators.length][]; for (int i = 0; i < subAggregators.length; i++) { - aggregations[i] = subAggregators[i].buildAggregations(ordsToCollect); + aggregations[i] = subAggregators[i].buildAggregations(bucketOrdsToCollect); } - InternalAggregations[] result = new InternalAggregations[ordsToCollect.length]; - for (int ord = 0; ord < ordsToCollect.length; ord++) { + InternalAggregations[] result = new InternalAggregations[bucketOrdsToCollect.length]; + for (int ord = 0; ord < bucketOrdsToCollect.length; ord++) { InternalAggregation[] slice = new InternalAggregation[subAggregators.length]; for (int i = 0; i < subAggregators.length; i++) { slice[i] = aggregations[i][ord]; @@ -168,6 +189,162 @@ public int size() { return result; } + /** + * Build the sub aggregation results for a list of buckets and set them on + * the buckets. This is usually used by aggregations that are selective + * in which bucket they build. They use some mechanism of selecting a list + * of buckets to build use this method to "finish" building the results. + * @param buckets the buckets to finish building + * @param bucketToOrd how to convert a bucket into an ordinal + * @param setAggs how to set the sub-aggregation results on a bucket + */ + protected final void buildSubAggsForBuckets(B[] buckets, + ToLongFunction bucketToOrd, BiConsumer setAggs) throws IOException { + InternalAggregations[] results = buildSubAggsForBuckets(Arrays.stream(buckets).mapToLong(bucketToOrd).toArray()); + for (int i = 0; i < buckets.length; i++) { + setAggs.accept(buckets[i], results[i]); + } + } + + /** + * Build the sub aggregation results for a list of buckets and set them on + * the buckets. This is usually used by aggregations that are selective + * in which bucket they build. They use some mechanism of selecting a list + * of buckets to build use this method to "finish" building the results. + * @param buckets the buckets to finish building + * @param bucketToOrd how to convert a bucket into an ordinal + * @param setAggs how to set the sub-aggregation results on a bucket + */ + protected final void buildSubAggsForAllBuckets(B[][] buckets, + ToLongFunction bucketToOrd, BiConsumer setAggs) throws IOException { + int totalBucketOrdsToCollect = 0; + for (B[] bucketsForOneResult : buckets) { + totalBucketOrdsToCollect += bucketsForOneResult.length; + } + long[] bucketOrdsToCollect = new long[totalBucketOrdsToCollect]; + int s = 0; + for (B[] bucketsForOneResult : buckets) { + for (B bucket : bucketsForOneResult) { + bucketOrdsToCollect[s++] = bucketToOrd.applyAsLong(bucket); + } + } + InternalAggregations[] results = buildSubAggsForBuckets(bucketOrdsToCollect); + s = 0; + for (int r = 0; r < buckets.length; r++) { + for (int b = 0; b < buckets[r].length; b++) { + setAggs.accept(buckets[r][b], results[s++]); + } + } + } + + /** + * Build the sub aggregation results for a list of buckets and set them on + * the buckets. This is usually used by aggregations that are selective + * in which bucket they build. They use some mechanism of selecting a list + * of buckets to build use this method to "finish" building the results. + * @param buckets the buckets to finish building + * @param bucketToOrd how to convert a bucket into an ordinal + * @param setAggs how to set the sub-aggregation results on a bucket + */ + protected final void buildSubAggsForBuckets(List buckets, + ToLongFunction bucketToOrd, BiConsumer setAggs) throws IOException { + InternalAggregations[] results = buildSubAggsForBuckets(buckets.stream().mapToLong(bucketToOrd).toArray()); + for (int i = 0; i < buckets.size(); i++) { + setAggs.accept(buckets.get(i), results[i]); + } + } + + /** + * Build aggregation results for an aggregator that has a fixed number of buckets per owning ordinal. + * @param the type of the bucket + * @param owningBucketOrds owning bucket ordinals for which to build the results + * @param bucketsPerOwningBucketOrd how many buckets there are per ord + * @param bucketBuilder how to build a bucket + * @param resultBuilder how to build a result from buckets + */ + protected final InternalAggregation[] buildAggregationsForFixedBucketCount(long[] owningBucketOrds, int bucketsPerOwningBucketOrd, + BucketBuilderForFixedCount bucketBuilder, Function, InternalAggregation> resultBuilder) throws IOException { + int totalBuckets = owningBucketOrds.length * bucketsPerOwningBucketOrd; + consumeBucketsAndMaybeBreak(totalBuckets); + long[] bucketOrdsToCollect = new long[totalBuckets]; + int bucketOrdIdx = 0; + for (long owningBucketOrd : owningBucketOrds) { + long ord = owningBucketOrd * bucketsPerOwningBucketOrd; + for (int offsetInOwningOrd = 0; offsetInOwningOrd < bucketsPerOwningBucketOrd; offsetInOwningOrd++) { + bucketOrdsToCollect[bucketOrdIdx++] = ord++; + } + } + bucketOrdIdx = 0; + InternalAggregations[] subAggregationResults = buildSubAggsForBuckets(bucketOrdsToCollect); + InternalAggregation[] results = new InternalAggregation[owningBucketOrds.length]; + for (int owningOrdIdx = 0; owningOrdIdx < owningBucketOrds.length; owningOrdIdx++) { + List buckets = new ArrayList<>(bucketsPerOwningBucketOrd); + for (int offsetInOwningOrd = 0; offsetInOwningOrd < bucketsPerOwningBucketOrd; offsetInOwningOrd++) { + buckets.add(bucketBuilder.build( + offsetInOwningOrd, bucketDocCount(bucketOrdsToCollect[bucketOrdIdx]), subAggregationResults[bucketOrdIdx++])); + } + results[owningOrdIdx] = resultBuilder.apply(buckets); + } + return results; + } + @FunctionalInterface + protected interface BucketBuilderForFixedCount { + B build(int offsetInOwningOrd, int docCount, InternalAggregations subAggregationResults); + } + + /** + * Build aggregation results for an aggregator that always contain a single bucket. + * @param owningBucketOrds owning bucket ordinals for which to build the results + * @param resultBuilder how to build a result from the sub aggregation results + */ + protected final InternalAggregation[] buildAggregationsForSingleBucket(long[] owningBucketOrds, + SingleBucketResultBuilder resultBuilder) throws IOException { + /* + * It'd be entirely reasonable to call + * `consumeBucketsAndMaybeBreak(owningBucketOrds.length)` + * here but we don't because single bucket aggs never have. + */ + InternalAggregations[] subAggregationResults = buildSubAggsForBuckets(owningBucketOrds); + InternalAggregation[] results = new InternalAggregation[owningBucketOrds.length]; + for (int ordIdx = 0; ordIdx < owningBucketOrds.length; ordIdx++) { + results[ordIdx] = resultBuilder.build(owningBucketOrds[ordIdx], subAggregationResults[ordIdx]); + } + return results; + } + @FunctionalInterface + protected interface SingleBucketResultBuilder { + InternalAggregation build(long owningBucketOrd, InternalAggregations subAggregationResults); + } + + /** + * Build aggregation results for an aggregator with a varying number of + * {@code long} keyed buckets that is at the top level or wrapped in + * {@link AggregatorFactory#asMultiBucketAggregator}. + * @param owningBucketOrds owning bucket ordinals for which to build the results + * @param bucketOrds hash of values to the bucket ordinal + */ + protected final InternalAggregation[] buildAggregationsForVariableBuckets(long[] owningBucketOrds, LongHash bucketOrds, + BucketBuilderForVariable bucketBuilder, Function, InternalAggregation> resultBuilder) throws IOException { + assert owningBucketOrds.length == 1 && owningBucketOrds[0] == 0; + consumeBucketsAndMaybeBreak((int) bucketOrds.size()); + long[] bucketOrdsToCollect = new long[(int) bucketOrds.size()]; + for (int bucketOrd = 0; bucketOrd < bucketOrds.size(); bucketOrd++) { + bucketOrdsToCollect[bucketOrd] = bucketOrd; + } + + InternalAggregations[] subAggregationResults = buildSubAggsForBuckets(bucketOrdsToCollect); + List buckets = new ArrayList<>((int) bucketOrds.size()); + for (int bucketOrd = 0; bucketOrd < bucketOrds.size(); bucketOrd++) { + buckets.add(bucketBuilder.build(bucketOrds.get(bucketOrd), bucketDocCount(bucketOrd), subAggregationResults[bucketOrd])); + } + + return new InternalAggregation[] { resultBuilder.apply(buckets) }; + } + @FunctionalInterface + protected interface BucketBuilderForVariable { + B build(long bucketValue, int docCount, InternalAggregations subAggregationResults); + } + /** * Utility method to build empty aggregations of the sub aggregators. */ diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferableBucketAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferableBucketAggregator.java index 448b5e6d81521..220581b1235f5 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferableBucketAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferableBucketAggregator.java @@ -19,17 +19,9 @@ package org.elasticsearch.search.aggregations.bucket; -import org.apache.lucene.util.IOSupplier; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.lease.Releasables; -import org.elasticsearch.common.util.LongHash; -import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; -import org.elasticsearch.search.aggregations.AggregatorFactory.MultiBucketAggregatorWrapper; import org.elasticsearch.search.aggregations.BucketCollector; -import org.elasticsearch.search.aggregations.InternalAggregation; -import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.MultiBucketCollector; import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregator; import org.elasticsearch.search.internal.SearchContext; @@ -92,13 +84,9 @@ public DeferringBucketCollector getDeferringCollector() { /** * This method should be overridden by subclasses that want to defer * calculation of a child aggregation until a first pass is complete and a - * set of buckets has been pruned. Deferring collection will require the - * recording of all doc/bucketIds from the first pass and then the sub class - * should call {@link #runDeferredCollections(long...)} for the selected set - * of buckets that survive the pruning. + * set of buckets has been pruned. * - * @param aggregator - * the child aggregator + * @param aggregator the child aggregator * @return true if the aggregator should be deferred until a first pass at * collection has completed */ @@ -106,28 +94,10 @@ protected boolean shouldDefer(Aggregator aggregator) { return false; } - /** - * Collect sub aggregations for a list of bucket ordinals. This may - * only be called once so any aggregation that calls this must be - * wrapped in {@link MultiBucketAggregatorWrapper}. - * @deprecated prefer delaying construction of the result with many calls - * to {@link #recordSurvingOrd(long)} and returning - * {@link #deferred(IOSupplier)}. - */ - @Deprecated - protected final void runDeferredCollections(long... bucketOrds) throws IOException { - // Being lenient here - ignore calls where there are no deferred - // collections to playback - if (recordingWrapper != null) { - recordingWrapper.replay(bucketOrds); - } - } - - protected final InternalAggregations[] runDeferredCollections(LongHash bucketOrds) throws IOException { - // NOCOMMIT maybe remove this entirely and just piggy back on building buckets? + @Override + protected void beforeBuildingBuckets(long[] ordsToCollect) throws IOException { if (recordingWrapper != null) { - recordingWrapper.prepareSelectedBuckets(bucketOrds); + recordingWrapper.replay(ordsToCollect); } - long[] } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java index 96dabbc1cf86a..ef91323c48f7e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java @@ -49,8 +49,8 @@ public DeferringBucketCollector() {} public final void replay(long... selectedBuckets) throws IOException { LongHash hash = new LongHash(selectedBuckets.length, BigArrays.NON_RECYCLING_INSTANCE); - for (long bucket : selectedBuckets) { - hash.add(bucket); + for (long ord : selectedBuckets) { + hash.add(ord); } prepareSelectedBuckets(hash); } @@ -110,8 +110,8 @@ public Aggregator subAggregator(String name) { } @Override - public InternalAggregation buildAggregation(long bucket) throws IOException { - return in.buildAggregation(bucket); + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + return in.buildAggregations(owningBucketOrds); } @Override @@ -119,11 +119,6 @@ public InternalAggregation buildEmptyAggregation() { return in.buildEmptyAggregation(); } - @Override - public boolean runDeferredCollections() throws IOException { - return in.runDeferredCollections(); - } - @Override public LeafBucketCollector getLeafCollector(LeafReaderContext ctx) throws IOException { throw new IllegalStateException( diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregator.java index 2e6a20a6b08d8..fed6916b2fcca 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregator.java @@ -176,8 +176,6 @@ public void collect(int doc, long bucket) throws IOException { @Override public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { - InternalAggregation[] results = new InternalAggregation[owningBucketOrds.length]; - // Buckets are ordered into groups - [keyed filters] [key1&key2 intersects] int maxOrd = owningBucketOrds.length * totalNumKeys; int totalBucketsToBuild = 0; @@ -186,6 +184,7 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I totalBucketsToBuild++; } } + consumeBucketsAndMaybeBreak(totalBucketsToBuild); long[] bucketOrdsToBuild = new long[totalBucketsToBuild]; int builtBucketIndex = 0; for (int ord = 0; ord < maxOrd; ord++) { @@ -193,13 +192,14 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I bucketOrdsToBuild[builtBucketIndex++] = ord; } } - + assert builtBucketIndex == totalBucketsToBuild; builtBucketIndex = 0; - List buckets = new ArrayList<>(filters.length); InternalAggregations[] bucketSubAggs = buildSubAggsForBuckets(bucketOrdsToBuild); - for (int ord = 0; ord < owningBucketOrds.length; ord++) { + InternalAggregation[] results = new InternalAggregation[owningBucketOrds.length]; + for (int owningBucketOrdIdx = 0; owningBucketOrdIdx < owningBucketOrds.length; owningBucketOrdIdx++) { + List buckets = new ArrayList<>(filters.length); for (int i = 0; i < keys.length; i++) { - long bucketOrd = bucketOrd(owningBucketOrds[ord], i); + long bucketOrd = bucketOrd(owningBucketOrds[owningBucketOrdIdx], i); int docCount = bucketDocCount(bucketOrd); // Empty buckets are not returned because this aggregation will commonly be used under a // a date-histogram where we will look for transactions over time and can expect many @@ -213,7 +213,7 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I int pos = keys.length; for (int i = 0; i < keys.length; i++) { for (int j = i + 1; j < keys.length; j++) { - long bucketOrd = bucketOrd(owningBucketOrds[ord], pos); + long bucketOrd = bucketOrd(owningBucketOrds[owningBucketOrdIdx], pos); int docCount = bucketDocCount(bucketOrd); // Empty buckets are not returned due to potential for very sparse matrices if (docCount > 0) { @@ -225,8 +225,9 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I pos++; } } - results[ord] = new InternalAdjacencyMatrix(name, buckets, metadata()); + results[owningBucketOrdIdx] = new InternalAdjacencyMatrix(name, buckets, metadata()); } + assert builtBucketIndex == totalBucketsToBuild; return results; } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregator.java index 439c4df3c15a7..ed8f2d20427ac 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregator.java @@ -133,26 +133,34 @@ protected void doPostCollection() throws IOException { } @Override - public InternalAggregation buildAggregation(long zeroBucket) throws IOException { - assert zeroBucket == 0L; + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + // Composite aggregator must be at the top of the aggregation tree + assert owningBucketOrds.length == 1 && owningBucketOrds[0] == 0L; consumeBucketsAndMaybeBreak(queue.size()); if (deferredCollectors != NO_OP_COLLECTOR) { // Replay all documents that contain at least one top bucket (collected during the first pass). - runCompositeDeferredCollections(); + runDeferredCollections(); } int num = Math.min(size, queue.size()); final InternalComposite.InternalBucket[] buckets = new InternalComposite.InternalBucket[num]; + long[] bucketOrdsToCollect = new long[queue.size()]; + for (int i = 0; i < queue.size(); i++) { + bucketOrdsToCollect[i] = i; + } + InternalAggregations[] subAggsForBuckets = buildSubAggsForBuckets(bucketOrdsToCollect); while (queue.size() > 0) { int slot = queue.pop(); CompositeKey key = queue.toCompositeKey(slot); - InternalAggregations aggs = bucketAggregations(slot); + InternalAggregations aggs = subAggsForBuckets[slot]; int docCount = queue.getDocCount(slot); buckets[queue.size()] = new InternalComposite.InternalBucket(sourceNames, formats, key, reverseMuls, docCount, aggs); } CompositeKey lastBucket = num > 0 ? buckets[num-1].getRawKey() : null; - return new InternalComposite(name, size, sourceNames, formats, Arrays.asList(buckets), lastBucket, reverseMuls, - earlyTerminated, metadata()); + return new InternalAggregation[] { + new InternalComposite(name, size, sourceNames, formats, Arrays.asList(buckets), lastBucket, reverseMuls, + earlyTerminated, metadata()) + }; } @Override @@ -352,11 +360,8 @@ public void collect(int doc, long bucket) throws IOException { /** * Replay the documents that might contain a top bucket and pass top buckets to * the {@link #deferredCollectors}. - *

- * Note: this is distinct from {@link #runDeferredCollections()} because it - * predates it and works differently. */ - private void runCompositeDeferredCollections() throws IOException { + private void runDeferredCollections() throws IOException { final boolean needsScores = scoreMode().needsScores(); Weight weight = null; if (needsScores) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregator.java index 2f8a4265a1ddd..49ed42a8e7304 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregator.java @@ -68,8 +68,9 @@ public void collect(int doc, long bucket) throws IOException { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - return new InternalFilter(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metadata()); + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + return buildAggregationsForSingleBucket(owningBucketOrds, (owningBucketOrd, subAggregationResults) -> + new InternalFilter(name, bucketDocCount(owningBucketOrd), subAggregationResults, metadata())); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FiltersAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FiltersAggregator.java index 76bea1f64ec56..cb5f4c143b118 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FiltersAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FiltersAggregator.java @@ -163,23 +163,15 @@ public void collect(int doc, long bucket) throws IOException { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - consumeBucketsAndMaybeBreak(keys.length + (showOtherBucket ? 1 : 0)); - List buckets = new ArrayList<>(keys.length); - for (int i = 0; i < keys.length; i++) { - long bucketOrd = bucketOrd(owningBucketOrdinal, i); - InternalFilters.InternalBucket bucket = new InternalFilters.InternalBucket(keys[i], bucketDocCount(bucketOrd), - bucketAggregations(bucketOrd), keyed); - buckets.add(bucket); - } - // other bucket - if (showOtherBucket) { - long bucketOrd = bucketOrd(owningBucketOrdinal, keys.length); - InternalFilters.InternalBucket bucket = new InternalFilters.InternalBucket(otherBucketKey, bucketDocCount(bucketOrd), - bucketAggregations(bucketOrd), keyed); - buckets.add(bucket); - } - return new InternalFilters(name, buckets, keyed, metadata()); + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + return buildAggregationsForFixedBucketCount(owningBucketOrds, keys.length + (showOtherBucket ? 1 : 0), + (offsetInOwningOrd, docCount, subAggregationResults) -> { + if (offsetInOwningOrd < keys.length) { + return new InternalFilters.InternalBucket(keys[offsetInOwningOrd], docCount, + subAggregationResults, keyed); + } + return new InternalFilters.InternalBucket(otherBucketKey, docCount, subAggregationResults, keyed); + }, buckets -> new InternalFilters(name, buckets, keyed, metadata())); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregator.java index 1aec319f74cce..803a11e5e7b9b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregator.java @@ -25,7 +25,7 @@ import org.elasticsearch.common.util.LongHash; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; -import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; @@ -105,8 +105,8 @@ public void collect(int doc, long bucket) throws IOException { abstract InternalGeoGridBucket newEmptyBucket(); @Override - public InternalGeoGrid buildAggregation(long owningBucketOrdinal) throws IOException { - assert owningBucketOrdinal == 0; + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + assert owningBucketOrds.length == 1 && owningBucketOrds[0] == 0; final int size = (int) Math.min(bucketOrds.size(), shardSize); consumeBucketsAndMaybeBreak(size); @@ -125,17 +125,12 @@ public InternalGeoGrid buildAggregation(long owningBucketOrdinal) throws IOExcep spare = ordered.insertWithOverflow(spare); } - long[] ordsToCollect = new long[ordered.size()]; final InternalGeoGridBucket[] list = new InternalGeoGridBucket[ordered.size()]; for (int i = ordered.size() - 1; i >= 0; --i) { list[i] = ordered.pop(); - ordsToCollect[i] = list[i].bucketOrd; } - InternalAggregations[] sub = buildSubAggsForBuckets(ordsToCollect); - for (int i = 0; i < list.length; i++) { - list[i].aggregations = sub[i]; - } - return buildAggregation(name, requiredSize, Arrays.asList(list), metadata()); + buildSubAggsForBuckets(list, b -> b.bucketOrd, (b, aggs) -> b.aggregations = aggs); + return new InternalAggregation[] {buildAggregation(name, requiredSize, Arrays.asList(list), metadata())}; } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregator.java index 52e8854014411..1219d4fa63be6 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregator.java @@ -50,9 +50,11 @@ public void collect(int doc, long bucket) throws IOException { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - assert owningBucketOrdinal == 0 : "global aggregator can only be a top level aggregator"; - return new InternalGlobal(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metadata()); + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + assert owningBucketOrds.length == 1 && owningBucketOrds[0] == 0: "global aggregator can only be a top level aggregator"; + return buildAggregationsForSingleBucket(owningBucketOrds, (owningBucketOrd, subAggregationResults) -> + new InternalGlobal(name, bucketDocCount(owningBucketOrd), subAggregationResults, metadata()) + ); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java index 6be20aa6fece6..9e4b2e0cc1767 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java @@ -41,9 +41,7 @@ import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; -import java.util.ArrayList; import java.util.Collections; -import java.util.List; import java.util.Map; /** @@ -160,31 +158,21 @@ private void increaseRounding() { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - assert owningBucketOrdinal == 0; - consumeBucketsAndMaybeBreak((int) bucketOrds.size()); - - long[] bucketOrdArray = new long[(int) bucketOrds.size()]; - for (int i = 0; i < bucketOrds.size(); i++) { - bucketOrdArray[i] = i; - } - - runDeferredCollections(bucketOrdArray); - - List buckets = new ArrayList<>((int) bucketOrds.size()); - for (long i = 0; i < bucketOrds.size(); i++) { - buckets.add(new InternalAutoDateHistogram.Bucket(bucketOrds.get(i), bucketDocCount(i), formatter, bucketAggregations(i))); - } - - // the contract of the histogram aggregation is that shards must return - // buckets ordered by key in ascending order - CollectionUtil.introSort(buckets, BucketOrder.key(true).comparator()); - - // value source will be null for unmapped fields - InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundingInfos, roundingIdx, - buildEmptySubAggregations()); - - return new InternalAutoDateHistogram(name, buckets, targetBuckets, emptyBucketInfo, formatter, metadata(), 1); + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + return buildAggregationsForVariableBuckets(owningBucketOrds, bucketOrds, + (bucketValue, docCount, subAggregationResults) -> + new InternalAutoDateHistogram.Bucket(bucketValue, docCount, formatter, subAggregationResults), + buckets -> { + // the contract of the histogram aggregation is that shards must return + // buckets ordered by key in ascending order + CollectionUtil.introSort(buckets, BucketOrder.key(true).comparator()); + + // value source will be null for unmapped fields + InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundingInfos, + roundingIdx, buildEmptySubAggregations()); + + return new InternalAutoDateHistogram(name, buckets, targetBuckets, emptyBucketInfo, formatter, metadata(), 1); + }); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java index 5fe2f31090468..563f2b28f2e8d 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java @@ -38,9 +38,7 @@ import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; -import java.util.ArrayList; import java.util.Collections; -import java.util.List; import java.util.Map; /** @@ -130,25 +128,22 @@ public void collect(int doc, long bucket) throws IOException { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - assert owningBucketOrdinal == 0; - consumeBucketsAndMaybeBreak((int) bucketOrds.size()); - - List buckets = new ArrayList<>((int) bucketOrds.size()); - for (long i = 0; i < bucketOrds.size(); i++) { - buckets.add(new InternalDateHistogram.Bucket(bucketOrds.get(i), bucketDocCount(i), keyed, formatter, bucketAggregations(i))); - } - - // the contract of the histogram aggregation is that shards must return buckets ordered by key in ascending order - CollectionUtil.introSort(buckets, BucketOrder.key(true).comparator()); - - // value source will be null for unmapped fields - // Important: use `rounding` here, not `shardRounding` - InternalDateHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 - ? new InternalDateHistogram.EmptyBucketInfo(rounding.withoutOffset(), buildEmptySubAggregations(), extendedBounds) - : null; - return new InternalDateHistogram(name, buckets, order, minDocCount, rounding.offset(), emptyBucketInfo, formatter, - keyed, metadata()); + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + return buildAggregationsForVariableBuckets(owningBucketOrds, bucketOrds, + (bucketValue, docCount, subAggregationResults) -> { + return new InternalDateHistogram.Bucket(bucketValue, docCount, keyed, formatter, subAggregationResults); + }, buckets -> { + // the contract of the histogram aggregation is that shards must return buckets ordered by key in ascending order + CollectionUtil.introSort(buckets, BucketOrder.key(true).comparator()); + + // value source will be null for unmapped fields + // Important: use `rounding` here, not `shardRounding` + InternalDateHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 + ? new InternalDateHistogram.EmptyBucketInfo(rounding.withoutOffset(), buildEmptySubAggregations(), extendedBounds) + : null; + return new InternalDateHistogram(name, buckets, order, minDocCount, rounding.offset(), emptyBucketInfo, formatter, + keyed, metadata()); + }); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateRangeHistogramAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateRangeHistogramAggregator.java index c26db7ef3748d..1352096339deb 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateRangeHistogramAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateRangeHistogramAggregator.java @@ -41,7 +41,6 @@ import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; -import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; @@ -150,25 +149,22 @@ public void collect(int doc, long bucket) throws IOException { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - assert owningBucketOrdinal == 0; - consumeBucketsAndMaybeBreak((int) bucketOrds.size()); - - List buckets = new ArrayList<>((int) bucketOrds.size()); - for (long i = 0; i < bucketOrds.size(); i++) { - buckets.add(new InternalDateHistogram.Bucket(bucketOrds.get(i), bucketDocCount(i), keyed, formatter, bucketAggregations(i))); - } - - // the contract of the histogram aggregation is that shards must return buckets ordered by key in ascending order - CollectionUtil.introSort(buckets, BucketOrder.key(true).comparator()); - - // value source will be null for unmapped fields - // Important: use `rounding` here, not `shardRounding` - InternalDateHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 - ? new InternalDateHistogram.EmptyBucketInfo(rounding.withoutOffset(), buildEmptySubAggregations(), extendedBounds) - : null; - return new InternalDateHistogram(name, buckets, order, minDocCount, rounding.offset(), emptyBucketInfo, formatter, - keyed, metadata()); + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + return buildAggregationsForVariableBuckets(owningBucketOrds, bucketOrds, + (bucketValue, docCount, subAggregationResults) -> + new InternalDateHistogram.Bucket(bucketValue, docCount, keyed, formatter, subAggregationResults), + buckets -> { + // the contract of the histogram aggregation is that shards must return buckets ordered by key in ascending order + CollectionUtil.introSort(buckets, BucketOrder.key(true).comparator()); + + // value source will be null for unmapped fields + // Important: use `rounding` here, not `shardRounding` + InternalDateHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 + ? new InternalDateHistogram.EmptyBucketInfo(rounding.withoutOffset(), buildEmptySubAggregations(), extendedBounds) + : null; + return new InternalDateHistogram(name, buckets, order, minDocCount, rounding.offset(), emptyBucketInfo, formatter, + keyed, metadata()); + }); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/NumericHistogramAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/NumericHistogramAggregator.java index 61bb0ddf00a4d..b23e98faecb5b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/NumericHistogramAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/NumericHistogramAggregator.java @@ -39,9 +39,7 @@ import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; -import java.util.ArrayList; import java.util.Collections; -import java.util.List; import java.util.Map; /** @@ -131,24 +129,22 @@ public void collect(int doc, long bucket) throws IOException { } @Override - public InternalAggregation buildAggregation(long bucket) throws IOException { - assert bucket == 0; - consumeBucketsAndMaybeBreak((int) bucketOrds.size()); - List buckets = new ArrayList<>((int) bucketOrds.size()); - for (long i = 0; i < bucketOrds.size(); i++) { - double roundKey = Double.longBitsToDouble(bucketOrds.get(i)); - double key = roundKey * interval + offset; - buckets.add(new InternalHistogram.Bucket(key, bucketDocCount(i), keyed, formatter, bucketAggregations(i))); - } - - // the contract of the histogram aggregation is that shards must return buckets ordered by key in ascending order - CollectionUtil.introSort(buckets, BucketOrder.key(true).comparator()); - - EmptyBucketInfo emptyBucketInfo = null; - if (minDocCount == 0) { - emptyBucketInfo = new EmptyBucketInfo(interval, offset, minBound, maxBound, buildEmptySubAggregations()); - } - return new InternalHistogram(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed, metadata()); + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + return buildAggregationsForVariableBuckets(owningBucketOrds, bucketOrds, + (bucketValue, docCount, subAggregationResults) -> { + double roundKey = Double.longBitsToDouble(bucketValue); + double key = roundKey * interval + offset; + return new InternalHistogram.Bucket(key, docCount, keyed, formatter, subAggregationResults); + }, buckets -> { + // the contract of the histogram aggregation is that shards must return buckets ordered by key in ascending order + CollectionUtil.introSort(buckets, BucketOrder.key(true).comparator()); + + EmptyBucketInfo emptyBucketInfo = null; + if (minDocCount == 0) { + emptyBucketInfo = new EmptyBucketInfo(interval, offset, minBound, maxBound, buildEmptySubAggregations()); + } + return new InternalHistogram(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed, metadata()); + }); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/RangeHistogramAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/RangeHistogramAggregator.java index 8760ad1b28c2a..a121c50af75d0 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/RangeHistogramAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/RangeHistogramAggregator.java @@ -36,11 +36,11 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; +import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram.EmptyBucketInfo; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; -import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; @@ -132,27 +132,25 @@ public void collect(int doc, long bucket) throws IOException { }; } - // TODO: buildAggregation and buildEmptyAggregation are literally just copied out of NumericHistogramAggregator. We could refactor + // TODO: buildAggregations and buildEmptyAggregation are literally just copied out of NumericHistogramAggregator. We could refactor // this to an abstract super class, if we wanted to. Might be overkill. @Override - public InternalAggregation buildAggregation(long bucket) throws IOException { - assert bucket == 0; - consumeBucketsAndMaybeBreak((int) bucketOrds.size()); - List buckets = new ArrayList<>((int) bucketOrds.size()); - for (long i = 0; i < bucketOrds.size(); i++) { - double roundKey = Double.longBitsToDouble(bucketOrds.get(i)); - double key = roundKey * interval + offset; - buckets.add(new InternalHistogram.Bucket(key, bucketDocCount(i), keyed, formatter, bucketAggregations(i))); - } - - // the contract of the histogram aggregation is that shards must return buckets ordered by key in ascending order - CollectionUtil.introSort(buckets, BucketOrder.key(true).comparator()); - - InternalHistogram.EmptyBucketInfo emptyBucketInfo = null; - if (minDocCount == 0) { - emptyBucketInfo = new InternalHistogram.EmptyBucketInfo(interval, offset, minBound, maxBound, buildEmptySubAggregations()); - } - return new InternalHistogram(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed, metadata()); + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + return buildAggregationsForVariableBuckets(owningBucketOrds, bucketOrds, + (bucketValue, docCount, subAggregationResults) -> { + double roundKey = Double.longBitsToDouble(bucketValue); + double key = roundKey * interval + offset; + return new InternalHistogram.Bucket(key, docCount, keyed, formatter, subAggregationResults); + }, buckets -> { + // the contract of the histogram aggregation is that shards must return buckets ordered by key in ascending order + CollectionUtil.introSort(buckets, BucketOrder.key(true).comparator()); + + EmptyBucketInfo emptyBucketInfo = null; + if (minDocCount == 0) { + emptyBucketInfo = new EmptyBucketInfo(interval, offset, minBound, maxBound, buildEmptySubAggregations()); + } + return new InternalHistogram(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed, metadata()); + }); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregator.java index 0974d7e6eb22d..e4966263450ff 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregator.java @@ -73,8 +73,9 @@ public void collect(int doc, long bucket) throws IOException { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - return new InternalMissing(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metadata()); + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + return buildAggregationsForSingleBucket(owningBucketOrds, (owningBucketOrd, subAggregationResults) -> + new InternalMissing(name, bucketDocCount(owningBucketOrd), subAggregationResults, metadata())); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregator.java index 9e8e0783ae5f1..07390e43a667d 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregator.java @@ -122,11 +122,12 @@ private void processBufferedDocs() throws IOException { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - return new InternalNested(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metadata()); + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + return buildAggregationsForSingleBucket(owningBucketOrds, (owningBucketOrd, subAggregationResults) -> + new InternalNested(name, bucketDocCount(owningBucketOrd), subAggregationResults, metadata())); } - @Override + @Override public InternalAggregation buildEmptyAggregation() { return new InternalNested(name, 0, buildEmptySubAggregations(), metadata()); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregator.java index 79ecd188f8154..8f9726a44c40e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregator.java @@ -91,8 +91,9 @@ public void collect(int childDoc, long bucket) throws IOException { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - return new InternalReverseNested(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metadata()); + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + return buildAggregationsForSingleBucket(owningBucketOrds, (owningBucketOrd, subAggregationResults) -> + new InternalReverseNested(name, bucketDocCount(owningBucketOrd), subAggregationResults, metadata())); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/BinaryRangeAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/BinaryRangeAggregator.java index fab0dad2f6dd2..003afa4393f6a 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/BinaryRangeAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/BinaryRangeAggregator.java @@ -34,7 +34,6 @@ import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; -import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; import java.util.List; @@ -326,16 +325,12 @@ private int collect(int doc, BytesRef value, long bucket, int lowBound) throws I } @Override - public InternalAggregation buildAggregation(long bucket) throws IOException { - consumeBucketsAndMaybeBreak(ranges.length); - List buckets = new ArrayList<>(ranges.length); - for (int i = 0; i < ranges.length; ++i) { - long bucketOrd = bucket * ranges.length + i; - buckets.add(new InternalBinaryRange.Bucket(format, keyed, - ranges[i].key, ranges[i].from, ranges[i].to, - bucketDocCount(bucketOrd), bucketAggregations(bucketOrd))); - } - return new InternalBinaryRange(name, format, keyed, buckets, metadata()); + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + return buildAggregationsForFixedBucketCount(owningBucketOrds, ranges.length, + (offsetInOwningOrd, docCount, subAggregationResults) -> { + Range range = ranges[offsetInOwningOrd]; + return new InternalBinaryRange.Bucket(format, keyed, range.key, range.from, range.to, docCount, subAggregationResults); + }, buckets -> new InternalBinaryRange(name, format, keyed, buckets, metadata())); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregator.java index ff905da95eeab..86b75f5455605 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregator.java @@ -325,19 +325,12 @@ private long subBucketOrdinal(long owningBucketOrdinal, int rangeOrd) { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - consumeBucketsAndMaybeBreak(ranges.length); - List buckets = new ArrayList<>(ranges.length); - for (int i = 0; i < ranges.length; i++) { - Range range = ranges[i]; - final long bucketOrd = subBucketOrdinal(owningBucketOrdinal, i); - org.elasticsearch.search.aggregations.bucket.range.Range.Bucket bucket = - rangeFactory.createBucket(range.key, range.from, range.to, bucketDocCount(bucketOrd), - bucketAggregations(bucketOrd), keyed, format); - buckets.add(bucket); - } - // value source can be null in the case of unmapped fields - return rangeFactory.create(name, buckets, format, keyed, metadata()); + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + return buildAggregationsForFixedBucketCount(owningBucketOrds, ranges.length, + (offsetInOwningOrd, docCount, subAggregationResults) -> { + Range range = ranges[offsetInOwningOrd]; + return rangeFactory.createBucket(range.key, range.from, range.to, docCount, subAggregationResults, keyed, format); + }, buckets -> rangeFactory.create(name, buckets, format, keyed, metadata())); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregator.java index 458242f386402..6ace7b093fc42 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregator.java @@ -164,10 +164,9 @@ protected boolean shouldDefer(Aggregator aggregator) { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - runDeferredCollections(owningBucketOrdinal); - return new InternalSampler(name, bdd == null ? 0 : bdd.getDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), - metadata()); + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + return buildAggregationsForSingleBucket(owningBucketOrds, (owningBucketOrd, subAggregationResults) -> + new InternalSampler(name, bdd == null ? 0 : bdd.getDocCount(owningBucketOrd), subAggregationResults, metadata())); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/GlobalOrdinalsSignificantTermsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/GlobalOrdinalsSignificantTermsAggregator.java index bfb4a01e0a30a..9e5b2f2b0baf2 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/GlobalOrdinalsSignificantTermsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/GlobalOrdinalsSignificantTermsAggregator.java @@ -25,6 +25,7 @@ import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic; @@ -80,10 +81,10 @@ public void collect(int doc, long bucket) throws IOException { } @Override - public SignificantStringTerms buildAggregation(long owningBucketOrdinal) throws IOException { - assert owningBucketOrdinal == 0; + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + assert owningBucketOrds.length == 1 && owningBucketOrds[0] == 0; if (valueCount == 0) { // no context in this reader - return buildEmptyAggregation(); + return new InternalAggregation[] {buildEmptyAggregation()}; } final int size; @@ -143,23 +144,20 @@ public SignificantStringTerms buildAggregation(long owningBucketOrdinal) throws } final SignificantStringTerms.Bucket[] list = new SignificantStringTerms.Bucket[ordered.size()]; - final long[] survivingBucketOrds = new long[ordered.size()]; for (int i = ordered.size() - 1; i >= 0; i--) { - final SignificantStringTerms.Bucket bucket = ordered.pop(); - survivingBucketOrds[i] = bucket.bucketOrd; - list[i] = bucket; + list[i] = ordered.pop(); + /* + * The terms are owned by the BytesRefHash which will close after + * we're finished building the aggregation so we need to pull a copy. + */ + list[i].termBytes = BytesRef.deepCopyOf(list[i].termBytes); } + buildSubAggsForBuckets(list, b -> b.bucketOrd, (b, aggs) -> b.aggregations = aggs); - runDeferredCollections(survivingBucketOrds); - - for (SignificantStringTerms.Bucket bucket : list) { - // the terms are owned by the BytesRefHash, we need to pull a copy since the BytesRef hash data may be recycled at some point - bucket.termBytes = BytesRef.deepCopyOf(bucket.termBytes); - bucket.aggregations = bucketAggregations(bucket.bucketOrd); - } - - return new SignificantStringTerms(name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), - metadata(), format, subsetSize, supersetSize, significanceHeuristic, Arrays.asList(list)); + return new InternalAggregation[] { + new SignificantStringTerms(name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), + metadata(), format, subsetSize, supersetSize, significanceHeuristic, Arrays.asList(list)) + }; } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTermsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTermsAggregator.java index ec628b18f9929..6cfca2257d48d 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTermsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTermsAggregator.java @@ -24,6 +24,7 @@ import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic; @@ -70,10 +71,10 @@ public void collect(int doc, long bucket) throws IOException { } @Override - public SignificantLongTerms buildAggregation(long owningBucketOrd) throws IOException { - assert owningBucketOrd == 0; + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + assert owningBucketOrds.length == 1 && owningBucketOrds[0] == 0; - long bucketsInOrd = bucketOrds.bucketsInOrd(owningBucketOrd); + long bucketsInOrd = bucketOrds.bucketsInOrd(0); final int size = (int) Math.min(bucketsInOrd, bucketCountThresholds.getShardSize()); long supersetSize = termsAggFactory.getSupersetNumDocs(); @@ -81,7 +82,7 @@ public SignificantLongTerms buildAggregation(long owningBucketOrd) throws IOExce BucketSignificancePriorityQueue ordered = new BucketSignificancePriorityQueue<>(size); SignificantLongTerms.Bucket spare = null; - BucketOrdsEnum ordsEnum = bucketOrds.ordsEnum(owningBucketOrd); + BucketOrdsEnum ordsEnum = bucketOrds.ordsEnum(0); while (ordsEnum.next()) { final int docCount = bucketDocCount(ordsEnum.ord()); if (docCount < bucketCountThresholds.getShardMinDocCount()) { @@ -107,21 +108,16 @@ public SignificantLongTerms buildAggregation(long owningBucketOrd) throws IOExce } SignificantLongTerms.Bucket[] list = new SignificantLongTerms.Bucket[ordered.size()]; - final long[] survivingBucketOrds = new long[ordered.size()]; for (int i = ordered.size() - 1; i >= 0; i--) { - final SignificantLongTerms.Bucket bucket = ordered.pop(); - survivingBucketOrds[i] = bucket.bucketOrd; - list[i] = bucket; + list[i] = ordered.pop(); } - runDeferredCollections(survivingBucketOrds); + buildSubAggsForBuckets(list, bucket -> bucket.bucketOrd, (bucket, aggs) -> bucket.aggregations = aggs); - for (SignificantLongTerms.Bucket bucket : list) { - bucket.aggregations = bucketAggregations(bucket.bucketOrd); - } - - return new SignificantLongTerms(name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), - metadata(), format, subsetSize, supersetSize, significanceHeuristic, Arrays.asList(list)); + return new InternalAggregation[] { + new SignificantLongTerms(name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), + metadata(), format, subsetSize, supersetSize, significanceHeuristic, Arrays.asList(list)) + }; } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTermsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTermsAggregator.java index b3668e12d30f6..a445d20b52f18 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTermsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTermsAggregator.java @@ -25,6 +25,7 @@ import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic; @@ -73,8 +74,8 @@ public void collect(int doc, long bucket) throws IOException { } @Override - public SignificantStringTerms buildAggregation(long owningBucketOrdinal) throws IOException { - assert owningBucketOrdinal == 0; + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + assert owningBucketOrds.length == 1 && owningBucketOrds[0] == 0; final int size = (int) Math.min(bucketOrds.size(), bucketCountThresholds.getShardSize()); long supersetSize = termsAggFactory.getSupersetNumDocs(); @@ -111,25 +112,19 @@ public SignificantStringTerms buildAggregation(long owningBucketOrdinal) throws } final SignificantStringTerms.Bucket[] list = new SignificantStringTerms.Bucket[ordered.size()]; - final long[] survivingBucketOrds = new long[ordered.size()]; for (int i = ordered.size() - 1; i >= 0; i--) { - final SignificantStringTerms.Bucket bucket = ordered.pop(); - survivingBucketOrds[i] = bucket.bucketOrd; - list[i] = bucket; - } - - runDeferredCollections(survivingBucketOrds); - - for (SignificantStringTerms.Bucket bucket : list) { + list[i] = ordered.pop(); // the terms are owned by the BytesRefHash, we need to pull a copy since the BytesRef hash data may be // recycled at some point - bucket.termBytes = BytesRef.deepCopyOf(bucket.termBytes); - bucket.aggregations = bucketAggregations(bucket.bucketOrd); + list[i].termBytes = BytesRef.deepCopyOf(list[i].termBytes); } - return new SignificantStringTerms( name, bucketCountThresholds.getRequiredSize(), + buildSubAggsForBuckets(list, b -> b.bucketOrd, (b, a) -> b.aggregations = a); + return new InternalAggregation[] { + new SignificantStringTerms( name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), - metadata(), format, subsetSize, supersetSize, significanceHeuristic, Arrays.asList(list)); + metadata(), format, subsetSize, supersetSize, significanceHeuristic, Arrays.asList(list)) + }; } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTextAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTextAggregator.java index fbafee57bf23a..c0f2c7e2d38cd 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTextAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTextAggregator.java @@ -34,6 +34,7 @@ import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; @@ -184,8 +185,8 @@ private void collectFromSource(int doc, long bucket, String indexedFieldName, St } @Override - public SignificantStringTerms buildAggregation(long owningBucketOrdinal) throws IOException { - assert owningBucketOrdinal == 0; + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + assert owningBucketOrds.length == 1 && owningBucketOrds[0] == 0; final int size = (int) Math.min(bucketOrds.size(), bucketCountThresholds.getShardSize()); long supersetSize = termsAggFactory.getSupersetNumDocs(); @@ -223,16 +224,16 @@ public SignificantStringTerms buildAggregation(long owningBucketOrdinal) throws final SignificantStringTerms.Bucket[] list = new SignificantStringTerms.Bucket[ordered.size()]; for (int i = ordered.size() - 1; i >= 0; i--) { - final SignificantStringTerms.Bucket bucket = ordered.pop(); + list[i] = ordered.pop(); // the terms are owned by the BytesRefHash, we need to pull a copy since the BytesRef hash data may be recycled at some point - bucket.termBytes = BytesRef.deepCopyOf(bucket.termBytes); - bucket.aggregations = bucketAggregations(bucket.bucketOrd); - list[i] = bucket; + list[i].termBytes = BytesRef.deepCopyOf(list[i].termBytes); } + buildSubAggsForBuckets(list, b -> b.bucketOrd, (b, a) -> b.aggregations = a); - return new SignificantStringTerms( name, bucketCountThresholds.getRequiredSize(), + return new InternalAggregation[] { new SignificantStringTerms(name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), - metadata(), format, subsetSize, supersetSize, significanceHeuristic, Arrays.asList(list)); + metadata(), format, subsetSize, supersetSize, significanceHeuristic, Arrays.asList(list)) + }; } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/GlobalOrdinalsStringTermsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/GlobalOrdinalsStringTermsAggregator.java index c9c40920e11b5..689f4f712d2b1 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/GlobalOrdinalsStringTermsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/GlobalOrdinalsStringTermsAggregator.java @@ -165,9 +165,10 @@ protected static void copy(BytesRef from, BytesRef to) { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + assert owningBucketOrds.length == 1 && owningBucketOrds[0] == 0; if (valueCount == 0) { // no context in this reader - return buildEmptyAggregation(); + return new InternalAggregation[] {buildEmptyAggregation()}; } final int size; @@ -215,29 +216,22 @@ public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOE // Get the top buckets final StringTerms.Bucket[] list = new StringTerms.Bucket[ordered.size()]; - long survivingBucketOrds[] = new long[ordered.size()]; for (int i = ordered.size() - 1; i >= 0; --i) { final OrdBucket bucket = ordered.pop(); - survivingBucketOrds[i] = bucket.bucketOrd; BytesRef scratch = new BytesRef(); copy(lookupGlobalOrd.apply(bucket.globalOrd), scratch); list[i] = new StringTerms.Bucket(scratch, bucket.docCount, null, showTermDocCountError, 0, format); list[i].bucketOrd = bucket.bucketOrd; otherDocCount -= list[i].docCount; + list[i].docCountError = 0; } - //replay any deferred collections - runDeferredCollections(survivingBucketOrds); - - //Now build the aggs - for (int i = 0; i < list.length; i++) { - StringTerms.Bucket bucket = list[i]; - bucket.aggregations = bucket.docCount == 0 ? bucketEmptyAggregations() : bucketAggregations(bucket.bucketOrd); - bucket.docCountError = 0; - } + buildSubAggsForBuckets(list, b -> b.bucketOrd, (b, aggs) -> b.aggregations = aggs); - return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), + return new InternalAggregation[] { + new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), metadata(), format, bucketCountThresholds.getShardSize(), showTermDocCountError, - otherDocCount, Arrays.asList(list), 0); + otherDocCount, Arrays.asList(list), 0) + }; } /** diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongRareTermsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongRareTermsAggregator.java index dc6412c17b4bd..fde74735c9e1a 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongRareTermsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongRareTermsAggregator.java @@ -137,18 +137,13 @@ private List buildSketch() { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - assert owningBucketOrdinal == 0; + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + assert owningBucketOrds.length == 1 && owningBucketOrds[0] == 0; List buckets = buildSketch(); - runDeferredCollections(buckets.stream().mapToLong(b -> b.bucketOrd).toArray()); - - // Finalize the buckets - for (LongRareTerms.Bucket bucket : buckets) { - bucket.aggregations = bucketAggregations(bucket.bucketOrd); - } + buildSubAggsForBuckets(buckets, b -> b.bucketOrd, (b, aggs) -> b.aggregations = aggs); CollectionUtil.introSort(buckets, ORDER.comparator()); - return new LongRareTerms(name, ORDER, metadata(), format, buckets, maxDocCount, filter); + return new InternalAggregation[] {new LongRareTerms(name, ORDER, metadata(), format, buckets, maxDocCount, filter)}; } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java index 1bbed26c7c8fb..9c3ee1b38e03b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java @@ -22,13 +22,11 @@ import org.apache.lucene.index.SortedNumericDocValues; import org.apache.lucene.search.ScoreMode; import org.elasticsearch.common.lease.Releasables; -import org.elasticsearch.common.util.LongHash; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.BucketOrder; import org.elasticsearch.search.aggregations.InternalAggregation; -import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalOrder; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; @@ -108,79 +106,67 @@ public void collect(int doc, long owningBucketOrd) throws IOException { } @Override - public InternalAggregation[] buildAggregations(int owningBucketOrdsToCollect) throws IOException { - try (LongHash survivingOrds = new LongHash(owningBucketOrdsToCollect * bucketCountThresholds.getShardSize(), context.bigArrays())) { - LongTerms.Bucket[][] topBuckets = new LongTerms.Bucket[owningBucketOrdsToCollect][]; - long[] otherDocCounts = new long[owningBucketOrdsToCollect]; - for (int owningBucketOrd = 0; owningBucketOrd < owningBucketOrdsToCollect; owningBucketOrd++) { - long bucketsInOrd = bucketOrds.bucketsInOrd(owningBucketOrd); - if (bucketCountThresholds.getMinDocCount() == 0 && (InternalOrder.isCountDesc(order) == false || - bucketsInOrd < bucketCountThresholds.getRequiredSize())) { - // we need to fill-in the blanks - for (LeafReaderContext ctx : context.searcher().getTopReaderContext().leaves()) { - final SortedNumericDocValues values = getValues(valuesSource, ctx); - for (int docId = 0; docId < ctx.reader().maxDoc(); ++docId) { - if (values.advanceExact(docId)) { - final int valueCount = values.docValueCount(); - for (int i = 0; i < valueCount; ++i) { - long value = values.nextValue(); - if (longFilter == null || longFilter.accept(value)) { - bucketOrds.add(owningBucketOrd, value); - } + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + LongTerms.Bucket[][] topBucketsPerOrd = new LongTerms.Bucket[owningBucketOrds.length][]; + long[] otherDocCounts = new long[owningBucketOrds.length]; + for (int ordIdx = 0; ordIdx < owningBucketOrds.length; ordIdx++) { + long bucketsInOrd = bucketOrds.bucketsInOrd(owningBucketOrds[ordIdx]); + if (bucketCountThresholds.getMinDocCount() == 0 && (InternalOrder.isCountDesc(order) == false || + bucketsInOrd < bucketCountThresholds.getRequiredSize())) { + // we need to fill-in the blanks + for (LeafReaderContext ctx : context.searcher().getTopReaderContext().leaves()) { + final SortedNumericDocValues values = getValues(valuesSource, ctx); + for (int docId = 0; docId < ctx.reader().maxDoc(); ++docId) { + if (values.advanceExact(docId)) { + final int valueCount = values.docValueCount(); + for (int v = 0; v < valueCount; ++v) { + long value = values.nextValue(); + if (longFilter == null || longFilter.accept(value)) { + bucketOrds.add(owningBucketOrds[ordIdx], value); } } } } - bucketsInOrd = bucketOrds.bucketsInOrd(owningBucketOrd); } - - final int size = (int) Math.min(bucketsInOrd, bucketCountThresholds.getShardSize()); - BucketPriorityQueue ordered = new BucketPriorityQueue<>(size, partiallyBuiltBucketComparator); - LongTerms.Bucket spare = null; - BucketOrdsEnum ordsEnum = bucketOrds.ordsEnum(owningBucketOrd); - while (ordsEnum.next()) { - if (spare == null) { - spare = new LongTerms.Bucket(0, 0, null, showTermDocCountError, 0, format); - } - spare.term = ordsEnum.value(); - spare.docCount = bucketDocCount(ordsEnum.ord()); - otherDocCounts[owningBucketOrd] += spare.docCount; - spare.bucketOrd = ordsEnum.ord(); - if (bucketCountThresholds.getShardMinDocCount() <= spare.docCount) { - spare = ordered.insertWithOverflow(spare); - if (spare == null) { - consumeBucketsAndMaybeBreak(1); - } - } + bucketsInOrd = bucketOrds.bucketsInOrd(owningBucketOrds[ordIdx]); + } + + final int size = (int) Math.min(bucketsInOrd, bucketCountThresholds.getShardSize()); + BucketPriorityQueue ordered = new BucketPriorityQueue<>(size, partiallyBuiltBucketComparator); + LongTerms.Bucket spare = null; + BucketOrdsEnum ordsEnum = bucketOrds.ordsEnum(owningBucketOrds[ordIdx]); + while (ordsEnum.next()) { + if (spare == null) { + spare = new LongTerms.Bucket(0, 0, null, showTermDocCountError, 0, format); } - - // Get the top buckets - LongTerms.Bucket[] list = topBuckets[owningBucketOrd] = new LongTerms.Bucket[ordered.size()]; - for (int i = ordered.size() - 1; i >= 0; --i) { - final LongTerms.Bucket bucket = ordered.pop(); - bucket.bucketOrd = survivingOrds.add(bucket.bucketOrd); - if (bucket.bucketOrd < 0) { - bucket.bucketOrd = 1 - bucket.bucketOrd; + spare.term = ordsEnum.value(); + spare.docCount = bucketDocCount(ordsEnum.ord()); + otherDocCounts[ordIdx] += spare.docCount; + spare.bucketOrd = ordsEnum.ord(); + if (bucketCountThresholds.getShardMinDocCount() <= spare.docCount) { + spare = ordered.insertWithOverflow(spare); + if (spare == null) { + consumeBucketsAndMaybeBreak(1); } - list[i] = bucket; - otherDocCounts[owningBucketOrd] -= bucket.docCount; } } - runDeferredCollections(survivingOrds); - InternalAggregations[] bucketAggs = buildSubAggsForBuckets(survivingOrds.size()); + // Get the top buckets + LongTerms.Bucket[] list = topBucketsPerOrd[ordIdx] = new LongTerms.Bucket[ordered.size()]; + for (int b = ordered.size() - 1; b >= 0; --b) { + list[b] = ordered.pop(); + list[b].docCountError = 0; + otherDocCounts[ordIdx] -= list[b].docCount; + } + } - InternalAggregation[] result = new InternalAggregation[owningBucketOrdsToCollect]; - for (int owningBucketOrd = 0; owningBucketOrd < owningBucketOrdsToCollect; owningBucketOrd++) { - for (int i = 0; i < topBuckets[0].length; i++) { - topBuckets[owningBucketOrd][i].aggregations = bucketAggs[(int) survivingOrds.find(topBuckets[0][i].bucketOrd)]; - topBuckets[owningBucketOrd][i].docCountError = 0; - } + buildSubAggsForAllBuckets(topBucketsPerOrd, b -> b.bucketOrd, (b, aggs) -> b.aggregations = aggs); - result[owningBucketOrd] = buildResult(otherDocCounts[owningBucketOrd], Arrays.asList(topBuckets[owningBucketOrd])); - } - return result; + InternalAggregation[] result = new InternalAggregation[owningBucketOrds.length]; + for (int ordIdx = 0; ordIdx < owningBucketOrds.length; ordIdx++) { + result[ordIdx] = buildResult(otherDocCounts[ordIdx], Arrays.asList(topBucketsPerOrd[ordIdx])); } + return result; } protected InternalAggregation buildResult(long otherDocCount, List buckets) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringRareTermsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringRareTermsAggregator.java index e8dde8ff5f8fc..56c664f265848 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringRareTermsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringRareTermsAggregator.java @@ -143,19 +143,13 @@ private List buildSketch() { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - assert owningBucketOrdinal == 0; - + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + assert owningBucketOrds.length == 1 && owningBucketOrds[0] == 0; List buckets = buildSketch(); - runDeferredCollections(buckets.stream().mapToLong(b -> b.bucketOrd).toArray()); - - // Finalize the buckets - for (StringRareTerms.Bucket bucket : buckets) { - bucket.aggregations = bucketAggregations(bucket.bucketOrd); - } + buildSubAggsForBuckets(buckets, b -> b.bucketOrd, (b, aggs) -> b.aggregations = aggs); CollectionUtil.introSort(buckets, ORDER.comparator()); - return new StringRareTerms(name, ORDER, metadata(), format, buckets, maxDocCount, filter); + return new InternalAggregation[] {new StringRareTerms(name, ORDER, metadata(), format, buckets, maxDocCount, filter)}; } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java index 22fe8644f884e..84ada76cff83a 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java @@ -30,7 +30,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.BucketOrder; import org.elasticsearch.search.aggregations.InternalAggregation; -import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalOrder; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; @@ -109,8 +108,8 @@ public void collect(int doc, long bucket) throws IOException { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - assert owningBucketOrdinal == 0; + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + assert owningBucketOrds.length == 1 && owningBucketOrds[0] == 0; if (bucketCountThresholds.getMinDocCount() == 0 && (InternalOrder.isCountDesc(order) == false @@ -156,27 +155,20 @@ public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOE // Get the top buckets final StringTerms.Bucket[] list = new StringTerms.Bucket[ordered.size()]; - long survivingBucketOrds[] = new long[ordered.size()]; for (int i = ordered.size() - 1; i >= 0; --i) { final StringTerms.Bucket bucket = ordered.pop(); - survivingBucketOrds[i] = bucket.bucketOrd; list[i] = bucket; otherDocCount -= bucket.docCount; - } - // replay any deferred collections - runDeferredCollections(survivingBucketOrds); - InternalAggregations[] sub = buildSubAggsForBuckets(survivingBucketOrds); - - // Now build the aggs - for (int i = 0; i < list.length; i++) { - list[i].termBytes = BytesRef.deepCopyOf(list[i].termBytes); - list[i].aggregations = sub[i]; - list[i].docCountError = 0; + bucket.termBytes = BytesRef.deepCopyOf(list[i].termBytes); + bucket.docCountError = 0; } - return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), + buildSubAggsForBuckets(list, b -> b.bucketOrd, (b, a) -> b.aggregations = a); + return new InternalAggregation[] { + new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), metadata(), format, bucketCountThresholds.getShardSize(), showTermDocCountError, otherDocCount, - Arrays.asList(list), 0); + Arrays.asList(list), 0) + }; } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MetricsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MetricsAggregator.java index 19b96290edc3a..67e72053c1c05 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MetricsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MetricsAggregator.java @@ -22,6 +22,7 @@ import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorBase; import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; @@ -31,4 +32,13 @@ public abstract class MetricsAggregator extends AggregatorBase { protected MetricsAggregator(String name, SearchContext context, Aggregator parent, Map metadata) throws IOException { super(name, AggregatorFactories.EMPTY, context, parent, metadata); } + + @Override + public final InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + InternalAggregation[] results = new InternalAggregation[owningBucketOrds.length]; + for (int ordIdx = 0; ordIdx < owningBucketOrds.length; ordIdx++) { + results[ordIdx] = buildAggregation(owningBucketOrds[ordIdx]); + } + return results; + } } diff --git a/server/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationTimingType.java b/server/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationTimingType.java index d7039e90d7973..d1c5d3dd53859 100644 --- a/server/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationTimingType.java +++ b/server/src/main/java/org/elasticsearch/search/profile/aggregation/AggregationTimingType.java @@ -25,7 +25,6 @@ public enum AggregationTimingType { INITIALIZE, COLLECT, BUILD_AGGREGATION, - RUN_DEFERRED_COLLECTIONS, REDUCE; @Override diff --git a/server/src/main/java/org/elasticsearch/search/profile/aggregation/ProfilingAggregator.java b/server/src/main/java/org/elasticsearch/search/profile/aggregation/ProfilingAggregator.java index 032f89c303607..b6d34d193f1f3 100644 --- a/server/src/main/java/org/elasticsearch/search/profile/aggregation/ProfilingAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/profile/aggregation/ProfilingAggregator.java @@ -84,16 +84,14 @@ public BucketComparator bucketComparator(String key, SortOrder order) { } @Override - public InternalAggregation buildAggregation(long bucket) throws IOException { + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { Timer timer = profileBreakdown.getTimer(AggregationTimingType.BUILD_AGGREGATION); timer.start(); - InternalAggregation result; try { - result = delegate.buildAggregation(bucket); + return delegate.buildAggregations(owningBucketOrds); } finally { timer.stop(); } - return result; } @Override @@ -101,17 +99,6 @@ public InternalAggregation buildEmptyAggregation() { return delegate.buildEmptyAggregation(); } - @Override - public boolean runDeferredCollections() throws IOException { - Timer timer = profileBreakdown.getTimer(AggregationTimingType.RUN_DEFERRED_COLLECTIONS); - timer.start(); - try { - return delegate.runDeferredCollections(); - } finally { - timer.stop(); - } - } - @Override public LeafBucketCollector getLeafCollector(LeafReaderContext ctx) throws IOException { return new ProfilingLeafBucketCollector(delegate.getLeafCollector(ctx), profileBreakdown); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java index 33dcf993e7c56..052287b0db9e1 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java @@ -1294,10 +1294,10 @@ public void testNumberToStringValueScript() throws IOException { public void testThreeLayerLong() throws IOException { try (Directory dir = newDirectory()) { try (RandomIndexWriter writer = new RandomIndexWriter(random(), dir)) { - Document d = new Document(); for (int i = 0; i < 10; i++) { for (int j = 0; j < 10; j++) { for (int k = 0; k < 10; k++) { + Document d = new Document(); d.add(new SortedNumericDocValuesField("i", i)); d.add(new SortedNumericDocValuesField("j", j)); d.add(new SortedNumericDocValuesField("k", k)); @@ -1310,8 +1310,22 @@ public void testThreeLayerLong() throws IOException { TermsAggregationBuilder request = new TermsAggregationBuilder("i").field("i") .subAggregation(new TermsAggregationBuilder("j").field("j") .subAggregation(new TermsAggregationBuilder("k").field("k"))); - InternalTerms result = search(searcher, new MatchAllDocsQuery(), request, + LongTerms result = search(searcher, new MatchAllDocsQuery(), request, longField("i"), longField("j"), longField("k")); + for (int i = 0; i < 10; i++) { + LongTerms.Bucket iBucket = result.getBucketByKey(Integer.toString(i)); + assertThat(iBucket.getDocCount(), equalTo(100L)); + LongTerms jAgg = iBucket.getAggregations().get("j"); + for (int j = 0; j < 10; j++) { + LongTerms.Bucket jBucket = jAgg.getBucketByKey(Integer.toString(j)); + assertThat(jBucket.getDocCount(), equalTo(10L)); + LongTerms kAgg = jBucket.getAggregations().get("k"); + for (int k = 0; k < 10; k++) { + LongTerms.Bucket kBucket = kAgg.getBucketByKey(Integer.toString(k)); + assertThat(kBucket.getDocCount(), equalTo(1L)); + } + } + } } } } From 22ad5c4c56c5cc9080fa9ee1b6e9a868a0b1099f Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Sun, 3 May 2020 12:20:56 -0400 Subject: [PATCH 06/14] Cleanup --- .../aggregations/InternalAggregation.java | 37 ------------------- .../InternalMultiBucketAggregation.java | 24 +++--------- .../bucket/BestBucketsDeferringCollector.java | 10 +++-- .../bucket/DeferringBucketCollector.java | 15 +------- .../InternalSingleBucketAggregation.java | 4 -- .../sampler/BestDocsDeferringCollector.java | 3 +- 6 files changed, 16 insertions(+), 77 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/InternalAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/InternalAggregation.java index 573d25b03dd17..f09ebe5c20c28 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/InternalAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/InternalAggregation.java @@ -31,7 +31,6 @@ import org.elasticsearch.search.aggregations.support.AggregationPath; import java.io.IOException; -import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -333,40 +332,4 @@ public double sortValue(AggregationPath.PathElement head, Iterator { - List results = null; - for (int a = 0; a < ia.aggregations.size(); a++) { - InternalAggregation orig = ((InternalAggregation) ia.aggregations.get(a)); - InternalAggregation undeferred = orig.undefer(); - if (undeferred == orig) { - if (results != null) { - results.add(orig); - } - continue; - } - if (results == null) { - results = new ArrayList<>(ia.aggregations.size()); - for (int fillIn = 0; fillIn < a; fillIn++) { - results.add((InternalAggregation) ia.aggregations.get(fillIn)); - } - } - results.add(undeferred); - } - return results == null ? null : new InternalAggregations(results); - }); - } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/InternalMultiBucketAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/InternalMultiBucketAggregation.java index d85a407a25e5e..a1e1efd1f5152 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/InternalMultiBucketAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/InternalMultiBucketAggregation.java @@ -158,27 +158,19 @@ public final InternalAggregation reducePipelines( @Override public InternalAggregation copyWithRewritenBuckets(Function rewriter) { - List orig = getBuckets(); - List newBuckets = null; - for (int b = 0; b < orig.size(); b++) { - B bucket = orig.get(b); + boolean modified = false; + List newBuckets = new ArrayList<>(); + for (B bucket : getBuckets()) { InternalAggregations rewritten = rewriter.apply((InternalAggregations) bucket.getAggregations()); if (rewritten == null) { - if (newBuckets != null) { - newBuckets.add(bucket); - } + newBuckets.add(bucket); continue; } - if (newBuckets == null) { - newBuckets = new ArrayList<>(orig.size()); - for (int fillIn = 0; fillIn < b; fillIn++) { - newBuckets.add(orig.get(fillIn)); - } - } + modified = true; B newBucket = createBucket(rewritten, bucket); newBuckets.add(newBucket); } - return newBuckets == null ? this : create(newBuckets); + return modified ? create(newBuckets) : this; } @Override @@ -201,10 +193,6 @@ private List reducePipelineBuckets(ReduceContext reduceContext, PipelineTree return reducedBuckets; } - public final InternalAggregation undefer() { - return undeferBuckets(); - } - public abstract static class InternalBucket implements Bucket, Writeable { public Object getProperty(String containingAggName, List path) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java index 0fd555e83e998..cd399eaafc9f9 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java @@ -29,6 +29,7 @@ import org.apache.lucene.search.Weight; import org.apache.lucene.util.packed.PackedInts; import org.apache.lucene.util.packed.PackedLongValues; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.LongHash; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.BucketCollector; @@ -145,7 +146,7 @@ public void postCollection() throws IOException { * Replay the wrapped collector, but only on a selection of buckets. */ @Override - public void prepareSelectedBuckets(LongHash selectedBuckets) throws IOException { + public void prepareSelectedBuckets(long[] selectedBuckets) throws IOException { if (finished == false) { throw new IllegalStateException("Cannot replay yet, collection is not finished: postCollect() has not been called"); } @@ -153,7 +154,10 @@ public void prepareSelectedBuckets(LongHash selectedBuckets) throws IOException throw new IllegalStateException("Already been replayed"); } - this.selectedBuckets = selectedBuckets; + this.selectedBuckets = new LongHash(selectedBuckets.length, BigArrays.NON_RECYCLING_INSTANCE); + for (long ord : selectedBuckets) { + this.selectedBuckets.add(ord); + } boolean needsScores = scoreMode().needsScores(); Weight weight = null; @@ -180,7 +184,7 @@ public void prepareSelectedBuckets(LongHash selectedBuckets) throws IOException for (long i = 0, end = entry.docDeltas.size(); i < end; ++i) { doc += docDeltaIterator.next(); final long bucket = buckets.next(); - final long rebasedBucket = selectedBuckets.find(bucket); + final long rebasedBucket = this.selectedBuckets.find(bucket); if (rebasedBucket != -1) { if (needsScores) { if (scoreIt.docID() < doc) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java index ef91323c48f7e..9321e3c2d4093 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java @@ -21,9 +21,6 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.ScoreMode; -import org.elasticsearch.common.lease.Releasable; -import org.elasticsearch.common.util.BigArrays; -import org.elasticsearch.common.util.LongHash; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.BucketCollector; import org.elasticsearch.search.aggregations.InternalAggregation; @@ -48,21 +45,13 @@ public DeferringBucketCollector() {} public abstract void setDeferredCollector(Iterable deferredCollectors); public final void replay(long... selectedBuckets) throws IOException { - LongHash hash = new LongHash(selectedBuckets.length, BigArrays.NON_RECYCLING_INSTANCE); - for (long ord : selectedBuckets) { - hash.add(ord); - } - prepareSelectedBuckets(hash); + prepareSelectedBuckets(selectedBuckets); } /** * Replay some selected buckets. - *

- * The collector might retain a reference to the provided - * buckets but it doesn't take responsibility for - * {@link Releasable#close() closing} it. */ - public abstract void prepareSelectedBuckets(LongHash selectedBuckets) throws IOException; + public abstract void prepareSelectedBuckets(long[] selectedBuckets) throws IOException; /** * Wrap the provided aggregator so that it behaves (almost) as if it had diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregation.java index 2a265c2c30c07..a05c317a9908d 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregation.java @@ -186,10 +186,6 @@ public void forEachBucket(Consumer consumer) { consumer.accept(aggregations); } - public final InternalAggregation undefer() { - return undeferBuckets(); - } - @Override public boolean equals(Object obj) { if (this == obj) return true; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/BestDocsDeferringCollector.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/BestDocsDeferringCollector.java index 82ba5dbb1912e..a1ba9cff6d581 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/BestDocsDeferringCollector.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/BestDocsDeferringCollector.java @@ -32,7 +32,6 @@ import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.util.BigArrays; -import org.elasticsearch.common.util.LongHash; import org.elasticsearch.common.util.ObjectArray; import org.elasticsearch.search.aggregations.BucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollector; @@ -132,7 +131,7 @@ public void postCollection() throws IOException { @Override - public void prepareSelectedBuckets(LongHash selectedBuckets) throws IOException { + public void prepareSelectedBuckets(long[] selectedBuckets) throws IOException { // no-op - deferred aggs processed in postCollection call } From da4447601e8a5939a0fdd44b439784ed28f15482 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Sun, 3 May 2020 13:06:19 -0400 Subject: [PATCH 07/14] Drop docs change --- docs/reference/search/profile.asciidoc | 36 +++++++++++--------------- 1 file changed, 15 insertions(+), 21 deletions(-) diff --git a/docs/reference/search/profile.asciidoc b/docs/reference/search/profile.asciidoc index 57b8f4b437b4a..561ed30a8cc74 100644 --- a/docs/reference/search/profile.asciidoc +++ b/docs/reference/search/profile.asciidoc @@ -784,52 +784,46 @@ This yields the following aggregation profile output: { "type" : "LongTermsAggregator", "description" : "my_scoped_agg", - "time_in_nanos" : 195421, + "time_in_nanos" : 195386, "breakdown" : { "reduce" : 0, - "build_aggregation" : 80927, + "build_aggregation" : 81171, "build_aggregation_count" : 1, - "initialize" : 3619, + "initialize" : 22753, "initialize_count" : 1, "reduce_count" : 0, - "collect" : 109778, - "collect_count" : 4, - "run_deferred_collections" : 1090, - "run_deferred_collections_count" : 1 + "collect" : 91456, + "collect_count" : 4 } }, { "type" : "GlobalAggregator", "description" : "my_global_agg", - "time_in_nanos" : 231761, + "time_in_nanos" : 190430, "breakdown" : { "reduce" : 0, - "build_aggregation" : 64334, + "build_aggregation" : 59990, "build_aggregation_count" : 1, - "initialize" : 17229, + "initialize" : 29619, "initialize_count" : 1, "reduce_count" : 0, - "collect" : 147490, - "collect_count" : 4, - "run_deferred_collections" : 2701, - "run_deferred_collections_count": 1 + "collect" : 100815, + "collect_count" : 4 }, "children" : [ { "type" : "LongTermsAggregator", "description" : "my_level_agg", - "time_in_nanos" : 198881, + "time_in_nanos" : 160329, "breakdown" : { "reduce" : 0, - "build_aggregation" : 58959, + "build_aggregation" : 55712, "build_aggregation_count" : 1, - "initialize" : 1801, + "initialize" : 10559, "initialize_count" : 1, "reduce_count" : 0, - "collect" : 137302, - "collect_count" : 4, - "run_deferred_collections" : 812, - "run_deferred_collections_count" : 1 + "collect" : 94052, + "collect_count" : 4 } } ] From e064c4c16850ffcb2fd86d6ab493de2ace7e6bf6 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Sun, 3 May 2020 13:21:03 -0400 Subject: [PATCH 08/14] Doc method --- .../org/elasticsearch/search/aggregations/Aggregator.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java index 7bb4a3d64669b..15ca4454f40d9 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java @@ -178,7 +178,13 @@ public InternalAggregation buildAggregation(long owningBucketOrd) throws IOExcep */ public abstract InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException; - public InternalAggregation buildTopLevel() throws IOException { + /** + * Build the result of this aggregation if it is at the "top level" + * of the aggregation tree. If, instead, it is a sub-aggregation of + * another aggregation then the aggregation that contains it will call + * {@link #buildAggregations(long[])}. + */ + public final InternalAggregation buildTopLevel() throws IOException { assert parent() == null; return buildAggregations(new long[] {0})[0]; } From c8c83b80eced0169629706d8ef8bf159a1bb883b Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Sun, 3 May 2020 13:26:03 -0400 Subject: [PATCH 09/14] WIP --- .../java/org/elasticsearch/search/aggregations/Aggregator.java | 1 + .../aggregations/bucket/BestBucketsDeferringCollector.java | 2 +- .../search/aggregations/bucket/DeferringBucketCollector.java | 2 +- .../aggregations/bucket/sampler/BestDocsDeferringCollector.java | 2 +- 4 files changed, 4 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java index 15ca4454f40d9..c6e17a92d9c75 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java @@ -166,6 +166,7 @@ public InternalAggregation buildAggregation(long owningBucketOrd) throws IOExcep if (owningBucketOrd == 0) { return buildTopLevel(); } + // TODO remove this method and redaclare in MetricAggregator as protected abstract throw new UnsupportedOperationException(); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java index cd399eaafc9f9..82a17d46072db 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java @@ -146,7 +146,7 @@ public void postCollection() throws IOException { * Replay the wrapped collector, but only on a selection of buckets. */ @Override - public void prepareSelectedBuckets(long[] selectedBuckets) throws IOException { + public void prepareSelectedBuckets(long... selectedBuckets) throws IOException { if (finished == false) { throw new IllegalStateException("Cannot replay yet, collection is not finished: postCollect() has not been called"); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java index 9321e3c2d4093..f345827301bdd 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java @@ -51,7 +51,7 @@ public final void replay(long... selectedBuckets) throws IOException { /** * Replay some selected buckets. */ - public abstract void prepareSelectedBuckets(long[] selectedBuckets) throws IOException; + public abstract void prepareSelectedBuckets(long... selectedBuckets) throws IOException; /** * Wrap the provided aggregator so that it behaves (almost) as if it had diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/BestDocsDeferringCollector.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/BestDocsDeferringCollector.java index a1ba9cff6d581..47174089cd5e8 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/BestDocsDeferringCollector.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/BestDocsDeferringCollector.java @@ -131,7 +131,7 @@ public void postCollection() throws IOException { @Override - public void prepareSelectedBuckets(long[] selectedBuckets) throws IOException { + public void prepareSelectedBuckets(long... selectedBuckets) throws IOException { // no-op - deferred aggs processed in postCollection call } From 7ae83964d7865efebf22e7f3b06787fe5475e754 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 4 May 2020 09:15:38 -0400 Subject: [PATCH 10/14] Remove useless method --- .../aggregations/bucket/DeferableBucketAggregator.java | 2 +- .../aggregations/bucket/DeferringBucketCollector.java | 6 +----- .../bucket/BestBucketsDeferringCollectorTests.java | 4 ++-- .../bucket/sampler/BestDocsDeferringCollectorTests.java | 2 +- 4 files changed, 5 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferableBucketAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferableBucketAggregator.java index 220581b1235f5..3851af370210b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferableBucketAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferableBucketAggregator.java @@ -97,7 +97,7 @@ protected boolean shouldDefer(Aggregator aggregator) { @Override protected void beforeBuildingBuckets(long[] ordsToCollect) throws IOException { if (recordingWrapper != null) { - recordingWrapper.replay(ordsToCollect); + recordingWrapper.prepareSelectedBuckets(ordsToCollect); } } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java index f345827301bdd..bc6ab0b746fe1 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java @@ -44,12 +44,8 @@ public DeferringBucketCollector() {} /** Set the deferred collectors. */ public abstract void setDeferredCollector(Iterable deferredCollectors); - public final void replay(long... selectedBuckets) throws IOException { - prepareSelectedBuckets(selectedBuckets); - } - /** - * Replay some selected buckets. + * Replay the deferred hits on the selected buckets. */ public abstract void prepareSelectedBuckets(long... selectedBuckets) throws IOException; diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollectorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollectorTests.java index 74ee21b2466cd..040c98d487fec 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollectorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollectorTests.java @@ -80,7 +80,7 @@ public ScoreMode scoreMode() { collector.preCollection(); indexSearcher.search(termQuery, collector); collector.postCollection(); - collector.replay(0); + collector.prepareSelectedBuckets(0); assertEquals(topDocs.scoreDocs.length, deferredCollectedDocIds.size()); for (ScoreDoc scoreDoc : topDocs.scoreDocs) { @@ -94,7 +94,7 @@ public ScoreMode scoreMode() { collector.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), collector); collector.postCollection(); - collector.replay(0); + collector.prepareSelectedBuckets(0); assertEquals(topDocs.scoreDocs.length, deferredCollectedDocIds.size()); for (ScoreDoc scoreDoc : topDocs.scoreDocs) { diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/sampler/BestDocsDeferringCollectorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/sampler/BestDocsDeferringCollectorTests.java index 740d47e5ec23f..76ab7ee53ac52 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/sampler/BestDocsDeferringCollectorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/sampler/BestDocsDeferringCollectorTests.java @@ -75,7 +75,7 @@ public void testReplay() throws Exception { collector.preCollection(); indexSearcher.search(termQuery, collector); collector.postCollection(); - collector.replay(0); + collector.prepareSelectedBuckets(0); assertEquals(topDocs.scoreDocs.length, deferredCollectedDocIds.size()); for (ScoreDoc scoreDoc : topDocs.scoreDocs) { From e872b238774841974ccacbc7002e77422b869a90 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 4 May 2020 11:06:06 -0400 Subject: [PATCH 11/14] Add tests for nested and reverse nested --- .../bucket/terms/LongKeyedBucketOrds.java | 3 + .../bucket/nested/NestedAggregatorTests.java | 95 ++++++++++++++++--- .../nested/ReverseNestedAggregatorTests.java | 53 ++++++++++- .../terms/LongKeyedBucketOrdsTests.java | 11 +++ .../terms/RareTermsAggregatorTests.java | 4 +- .../bucket/terms/TermsAggregatorTests.java | 2 +- .../aggregations/AggregatorTestCase.java | 23 +++-- 7 files changed, 165 insertions(+), 26 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongKeyedBucketOrds.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongKeyedBucketOrds.java index c0dca23b51b53..68dd377757c27 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongKeyedBucketOrds.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongKeyedBucketOrds.java @@ -208,6 +208,9 @@ private Buckets bucketsForOrd(long owningBucketOrd) { @Override public long bucketsInOrd(long owningBucketOrd) { + if (owningBucketOrd >= owningOrdToBuckets.size()) { + return 0; + } Buckets buckets = owningOrdToBuckets.get(owningBucketOrd); if (buckets == null) { return 0; diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTests.java index a2f4b3d096659..8f485509778f7 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTests.java @@ -37,6 +37,7 @@ import org.apache.lucene.search.TermQuery; import org.apache.lucene.store.Directory; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.CheckedConsumer; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.settings.Settings; @@ -54,6 +55,8 @@ import org.elasticsearch.script.ScriptModule; import org.elasticsearch.script.ScriptService; import org.elasticsearch.script.ScriptType; +import org.elasticsearch.search.aggregations.AggregationBuilder; +import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorTestCase; import org.elasticsearch.search.aggregations.BucketOrder; import org.elasticsearch.search.aggregations.InternalAggregation; @@ -61,6 +64,7 @@ import org.elasticsearch.search.aggregations.bucket.filter.FilterAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.terms.InternalTerms; import org.elasticsearch.search.aggregations.bucket.terms.LongTerms; +import org.elasticsearch.search.aggregations.bucket.terms.LongTermsAggregator; import org.elasticsearch.search.aggregations.bucket.terms.StringTerms; import org.elasticsearch.search.aggregations.bucket.terms.Terms; import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder; @@ -87,9 +91,12 @@ import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.DoubleStream; +import java.util.stream.LongStream; +import static java.util.stream.Collectors.toList; import static org.elasticsearch.search.aggregations.AggregationBuilders.max; import static org.elasticsearch.search.aggregations.AggregationBuilders.nested; +import static org.hamcrest.Matchers.equalTo; public class NestedAggregatorTests extends AggregatorTestCase { @@ -101,7 +108,7 @@ public class NestedAggregatorTests extends AggregatorTestCase { private static final String SUM_AGG_NAME = "sumAgg"; private static final String INVERSE_SCRIPT = "inverse"; - private final SeqNoFieldMapper.SequenceIDFields sequenceIDFields = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); + private static final SeqNoFieldMapper.SequenceIDFields sequenceIDFields = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); /** * For each provided field type, we also register an alias with name {@code -alias}. @@ -113,6 +120,14 @@ protected Map getFieldAliases(MappedFieldType... fieldT Function.identity())); } + /** + * Nested aggregations need the {@linkplain DirectoryReader} wrapped. + */ + @Override + protected IndexReader wrapDirectoryReader(DirectoryReader reader) throws IOException { + return wrapInMockESDirectoryReader(reader); + } + @Override protected ScriptService getMockScriptService() { Map, Object>> scripts = new HashMap<>(); @@ -130,7 +145,7 @@ public void testNoDocs() throws IOException { try (RandomIndexWriter iw = new RandomIndexWriter(random(), directory)) { // intentionally not writing any docs } - try (IndexReader indexReader = wrap(DirectoryReader.open(directory))) { + try (IndexReader indexReader = wrapInMockESDirectoryReader(DirectoryReader.open(directory))) { NestedAggregationBuilder nestedBuilder = new NestedAggregationBuilder(NESTED_AGG, NESTED_OBJECT); MaxAggregationBuilder maxAgg = new MaxAggregationBuilder(MAX_AGG_NAME) @@ -177,7 +192,7 @@ public void testSingleNestingMax() throws IOException { } iw.commit(); } - try (IndexReader indexReader = wrap(DirectoryReader.open(directory))) { + try (IndexReader indexReader = wrapInMockESDirectoryReader(DirectoryReader.open(directory))) { NestedAggregationBuilder nestedBuilder = new NestedAggregationBuilder(NESTED_AGG, NESTED_OBJECT); MaxAggregationBuilder maxAgg = new MaxAggregationBuilder(MAX_AGG_NAME) @@ -230,7 +245,7 @@ public void testDoubleNestingMax() throws IOException { } iw.commit(); } - try (IndexReader indexReader = wrap(DirectoryReader.open(directory))) { + try (IndexReader indexReader = wrapInMockESDirectoryReader(DirectoryReader.open(directory))) { NestedAggregationBuilder nestedBuilder = new NestedAggregationBuilder(NESTED_AGG, NESTED_OBJECT + "." + NESTED_OBJECT2); MaxAggregationBuilder maxAgg = new MaxAggregationBuilder(MAX_AGG_NAME) @@ -288,7 +303,7 @@ public void testOrphanedDocs() throws IOException { iw.addDocuments(documents); iw.commit(); } - try (IndexReader indexReader = wrap(DirectoryReader.open(directory))) { + try (IndexReader indexReader = wrapInMockESDirectoryReader(DirectoryReader.open(directory))) { NestedAggregationBuilder nestedBuilder = new NestedAggregationBuilder(NESTED_AGG, NESTED_OBJECT); SumAggregationBuilder sumAgg = new SumAggregationBuilder(SUM_AGG_NAME) @@ -371,7 +386,7 @@ public void testResetRootDocId() throws Exception { iw.commit(); iw.close(); } - try (IndexReader indexReader = wrap(DirectoryReader.open(directory))) { + try (IndexReader indexReader = wrapInMockESDirectoryReader(DirectoryReader.open(directory))) { NestedAggregationBuilder nestedBuilder = new NestedAggregationBuilder(NESTED_AGG, "nested_field"); @@ -409,7 +424,7 @@ public void testNestedOrdering() throws IOException { iw.addDocuments(generateBook("8", new String[]{"f"}, new int[]{12, 14})); iw.addDocuments(generateBook("9", new String[]{"g", "c", "e"}, new int[]{18, 8})); } - try (IndexReader indexReader = wrap(DirectoryReader.open(directory))) { + try (IndexReader indexReader = wrapInMockESDirectoryReader(DirectoryReader.open(directory))) { MappedFieldType fieldType1 = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG); fieldType1.setName("num_pages"); MappedFieldType fieldType2 = new KeywordFieldMapper.KeywordFieldType(); @@ -547,7 +562,7 @@ public void testNestedOrdering_random() throws IOException { return cmp; } }); - try (IndexReader indexReader = wrap(DirectoryReader.open(directory))) { + try (IndexReader indexReader = wrapInMockESDirectoryReader(DirectoryReader.open(directory))) { MappedFieldType fieldType1 = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG); fieldType1.setName("num_pages"); MappedFieldType fieldType2 = new KeywordFieldMapper.KeywordFieldType(); @@ -645,7 +660,7 @@ public void testPreGetChildLeafCollectors() throws IOException { iw.addDocuments(documents); iw.commit(); } - try (IndexReader indexReader = wrap(DirectoryReader.open(directory))) { + try (IndexReader indexReader = wrapInMockESDirectoryReader(DirectoryReader.open(directory))) { TermsAggregationBuilder valueBuilder = new TermsAggregationBuilder("value").userValueTypeHint(ValueType.STRING) .field("value"); TermsAggregationBuilder keyBuilder = new TermsAggregationBuilder("key").userValueTypeHint(ValueType.STRING) @@ -720,7 +735,7 @@ public void testFieldAlias() throws IOException { iw.commit(); } - try (IndexReader indexReader = wrap(DirectoryReader.open(directory))) { + try (IndexReader indexReader = wrapInMockESDirectoryReader(DirectoryReader.open(directory))) { NestedAggregationBuilder agg = nested(NESTED_AGG, NESTED_OBJECT).subAggregation( max(MAX_AGG_NAME).field(VALUE_FIELD_NAME)); NestedAggregationBuilder aliasAgg = nested(NESTED_AGG, NESTED_OBJECT).subAggregation( @@ -763,7 +778,7 @@ public void testNestedWithPipeline() throws IOException { } iw.commit(); } - try (IndexReader indexReader = wrap(DirectoryReader.open(directory))) { + try (IndexReader indexReader = wrapInMockESDirectoryReader(DirectoryReader.open(directory))) { NestedAggregationBuilder nestedBuilder = new NestedAggregationBuilder(NESTED_AGG, NESTED_OBJECT) .subAggregation(new TermsAggregationBuilder("terms").field(VALUE_FIELD_NAME).userValueTypeHint(ValueType.NUMERIC) .subAggregation(new MaxAggregationBuilder(MAX_AGG_NAME).field(VALUE_FIELD_NAME)) @@ -797,6 +812,61 @@ public void testNestedWithPipeline() throws IOException { } } + /** + * {@link LongTermsAggregator} is the first complex bucking aggregation + * that stopped wrapping itself in {@link AggregatorFactory#asMultiBucketAggregator} + * so this tests that nested works properly inside of it. + */ + public void testNestedUnderLongTerms() throws IOException { + int numProducts = scaledRandomIntBetween(1, 100); + int numResellers = scaledRandomIntBetween(1, 100); + + AggregationBuilder b = new TermsAggregationBuilder("products").field("product_id").size(numProducts) + .subAggregation(new NestedAggregationBuilder("nested", "nested_reseller") + .subAggregation(new TermsAggregationBuilder("resellers").field("reseller_id").size(numResellers))); + testCase(b, new MatchAllDocsQuery(), buildResellerData(numProducts, numResellers), result -> { + LongTerms products = (LongTerms) result; + assertThat(products.getBuckets().stream().map(LongTerms.Bucket::getKeyAsNumber).collect(toList()), + equalTo(LongStream.range(0, numProducts).mapToObj(Long::valueOf).collect(toList()))); + for (int p = 0; p < numProducts; p++) { + LongTerms.Bucket bucket = products.getBucketByKey(Integer.toString(p)); + assertThat(bucket.getDocCount(), equalTo(1L)); + InternalNested nested = bucket.getAggregations().get("nested"); + assertThat(nested.getDocCount(), equalTo((long) numResellers)); + LongTerms resellers = nested.getAggregations().get("resellers"); + assertThat(resellers.getBuckets().stream().map(LongTerms.Bucket::getKeyAsNumber).collect(toList()), + equalTo(LongStream.range(0, numResellers).mapToObj(Long::valueOf).collect(toList()))); + } + }, resellersMappedFields()); + } + + public static CheckedConsumer buildResellerData(int numProducts, int numResellers) { + return iw -> { + for (int p = 0; p < numProducts; p++) { + List documents = new ArrayList<>(); + generateDocuments(documents, numResellers, p, "nested_reseller", "value"); + for (int r = 0; r < documents.size(); r++) { + documents.get(r).add(new SortedNumericDocValuesField("reseller_id", r)); + } + Document document = new Document(); + document.add(new Field(IdFieldMapper.NAME, Uid.encodeId(Integer.toString(p)), IdFieldMapper.Defaults.FIELD_TYPE)); + document.add(new Field(NestedPathFieldMapper.NAME, "test", NestedPathFieldMapper.Defaults.FIELD_TYPE)); + document.add(sequenceIDFields.primaryTerm); + document.add(new SortedNumericDocValuesField("product_id", p)); + documents.add(document); + iw.addDocuments(documents); + } + }; + } + + public static MappedFieldType[] resellersMappedFields() { + MappedFieldType productIdField = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG); + productIdField.setName("product_id"); + MappedFieldType resellerIdField = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG); + resellerIdField.setName("reseller_id"); + return new MappedFieldType[] {productIdField, resellerIdField}; + } + private double generateMaxDocs(List documents, int numNestedDocs, int id, String path, String fieldName) { return DoubleStream.of(generateDocuments(documents, numNestedDocs, id, path, fieldName)) .max().orElse(Double.NEGATIVE_INFINITY); @@ -806,8 +876,7 @@ private double generateSumDocs(List documents, int numNestedDocs, int return DoubleStream.of(generateDocuments(documents, numNestedDocs, id, path, fieldName)).sum(); } - private double[] generateDocuments(List documents, int numNestedDocs, int id, String path, String fieldName) { - + private static double[] generateDocuments(List documents, int numNestedDocs, int id, String path, String fieldName) { double[] values = new double[numNestedDocs]; for (int nested = 0; nested < numNestedDocs; nested++) { Document document = new Document(); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregatorTests.java index 5f1edcfded813..4d55db28a916e 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregatorTests.java @@ -33,8 +33,13 @@ import org.elasticsearch.index.mapper.NumberFieldMapper; import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.search.aggregations.AggregationBuilder; +import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorTestCase; import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.bucket.terms.LongTerms; +import org.elasticsearch.search.aggregations.bucket.terms.LongTermsAggregator; +import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.InternalMax; import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder; @@ -45,10 +50,13 @@ import java.util.Map; import java.util.function.Function; import java.util.stream.Collectors; +import java.util.stream.LongStream; +import static java.util.stream.Collectors.toList; import static org.elasticsearch.search.aggregations.AggregationBuilders.max; import static org.elasticsearch.search.aggregations.AggregationBuilders.nested; import static org.elasticsearch.search.aggregations.AggregationBuilders.reverseNested; +import static org.hamcrest.Matchers.equalTo; public class ReverseNestedAggregatorTests extends AggregatorTestCase { @@ -68,12 +76,20 @@ protected Map getFieldAliases(MappedFieldType... fieldT Function.identity())); } + /** + * Nested aggregations need the {@linkplain DirectoryReader} wrapped. + */ + @Override + protected IndexReader wrapDirectoryReader(DirectoryReader reader) throws IOException { + return wrapInMockESDirectoryReader(reader); + } + public void testNoDocs() throws IOException { try (Directory directory = newDirectory()) { try (RandomIndexWriter iw = new RandomIndexWriter(random(), directory)) { // intentionally not writing any docs } - try (IndexReader indexReader = wrap(DirectoryReader.open(directory))) { + try (IndexReader indexReader = wrapInMockESDirectoryReader(DirectoryReader.open(directory))) { NestedAggregationBuilder nestedBuilder = new NestedAggregationBuilder(NESTED_AGG, NESTED_OBJECT); ReverseNestedAggregationBuilder reverseNestedBuilder @@ -137,7 +153,7 @@ public void testMaxFromParentDocs() throws IOException { } iw.commit(); } - try (IndexReader indexReader = wrap(DirectoryReader.open(directory))) { + try (IndexReader indexReader = wrapInMockESDirectoryReader(DirectoryReader.open(directory))) { NestedAggregationBuilder nestedBuilder = new NestedAggregationBuilder(NESTED_AGG, NESTED_OBJECT); ReverseNestedAggregationBuilder reverseNestedBuilder @@ -207,7 +223,7 @@ public void testFieldAlias() throws IOException { iw.commit(); } - try (IndexReader indexReader = wrap(DirectoryReader.open(directory))) { + try (IndexReader indexReader = wrapInMockESDirectoryReader(DirectoryReader.open(directory))) { MaxAggregationBuilder maxAgg = max(MAX_AGG_NAME).field(VALUE_FIELD_NAME); MaxAggregationBuilder aliasMaxAgg = max(MAX_AGG_NAME).field(VALUE_FIELD_NAME + "-alias"); @@ -230,4 +246,35 @@ public void testFieldAlias() throws IOException { } } + /** + * {@link LongTermsAggregator} is the first complex bucking aggregation + * that stopped wrapping itself in {@link AggregatorFactory#asMultiBucketAggregator} + * so this tests that nested works properly inside of it. + */ + public void testNestedUnderLongTerms() throws IOException { + int numProducts = scaledRandomIntBetween(1, 100); + int numResellers = scaledRandomIntBetween(1, 100); + + AggregationBuilder b = new NestedAggregationBuilder("nested", "nested_reseller") + .subAggregation(new TermsAggregationBuilder("resellers").field("reseller_id").size(numResellers) + .subAggregation(new ReverseNestedAggregationBuilder("reverse_nested") + .subAggregation(new TermsAggregationBuilder("products").field("product_id").size(numProducts)))); + testCase(b, new MatchAllDocsQuery(), NestedAggregatorTests.buildResellerData(numProducts, numResellers), result -> { + InternalNested nested = (InternalNested) result; + assertThat(nested.getDocCount(), equalTo((long) numProducts * numResellers)); + LongTerms resellers = nested.getAggregations().get("resellers"); + assertThat(resellers.getBuckets().stream().map(LongTerms.Bucket::getKeyAsNumber).collect(toList()), + equalTo(LongStream.range(0, numResellers).mapToObj(Long::valueOf).collect(toList()))); + for (int r = 0; r < numResellers; r++) { + LongTerms.Bucket bucket = resellers.getBucketByKey(Integer.toString(r)); + assertThat(bucket.getDocCount(), equalTo((long) numProducts)); + InternalReverseNested reverseNested = bucket.getAggregations().get("reverse_nested"); + assertThat(reverseNested.getDocCount(), equalTo((long) numProducts)); + LongTerms products = reverseNested.getAggregations().get("products"); + assertThat(products.getBuckets().stream().map(LongTerms.Bucket::getKeyAsNumber).collect(toList()), + equalTo(LongStream.range(0, numProducts).mapToObj(Long::valueOf).collect(toList()))); + } + }, NestedAggregatorTests.resellersMappedFields()); + } + } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/LongKeyedBucketOrdsTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/LongKeyedBucketOrdsTests.java index 137c9abf98414..64679e4b41630 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/LongKeyedBucketOrdsTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/LongKeyedBucketOrdsTests.java @@ -73,6 +73,10 @@ private void collectsFromSingleBucketCase(LongKeyedBucketOrds ords) { assertThat(ords.add(0, 0), equalTo(-1L)); assertThat(ords.add(0, 1000), equalTo(-2L)); + // Check counting values + assertThat(ords.bucketsInOrd(0), equalTo(values.length + 2L)); + + // Check iteration LongKeyedBucketOrds.BucketOrdsEnum ordEnum = ords.ordsEnum(0); assertTrue(ordEnum.next()); assertThat(ordEnum.ord(), equalTo(0L)); @@ -124,23 +128,30 @@ public void testCollectsFromManyBuckets() { assertThat(ords.add(0, 0), equalTo(-1L)); assertThat(ords.add(1, 0), equalTo(-2L)); + for (long owningBucketOrd = 0; owningBucketOrd <= maxOwningBucketOrd; owningBucketOrd++) { + long expectedCount = 0; LongKeyedBucketOrds.BucketOrdsEnum ordEnum = ords.ordsEnum(owningBucketOrd); if (owningBucketOrd <= 1) { + expectedCount++; assertTrue(ordEnum.next()); assertThat(ordEnum.ord(), equalTo(owningBucketOrd)); assertThat(ordEnum.value(), equalTo(0L)); } for (int i = 0; i < values.length; i++) { if (values[i].owningBucketOrd == owningBucketOrd) { + expectedCount++; assertTrue(ordEnum.next()); assertThat(ordEnum.ord(), equalTo(i + 2L)); assertThat(ordEnum.value(), equalTo(values[i].value)); } } assertFalse(ordEnum.next()); + + assertThat(ords.bucketsInOrd(owningBucketOrd), equalTo(expectedCount)); } assertFalse(ords.ordsEnum(randomLongBetween(maxOwningBucketOrd + 1, Long.MAX_VALUE)).next()); + assertThat(ords.bucketsInOrd(randomLongBetween(maxOwningBucketOrd + 1, Long.MAX_VALUE)), equalTo(0L)); } } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/RareTermsAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/RareTermsAggregatorTests.java index 5d5caf3d4edd6..ba70ded92a998 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/RareTermsAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/RareTermsAggregatorTests.java @@ -431,7 +431,7 @@ public void testWithNestedAggregations() throws IOException { MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG); fieldType.setHasDocValues(true); fieldType.setName("nested_value"); - try (IndexReader indexReader = wrap(DirectoryReader.open(directory))) { + try (IndexReader indexReader = wrapInMockESDirectoryReader(DirectoryReader.open(directory))) { InternalNested result = searchAndReduce(newIndexSearcher(indexReader), // match root document only new DocValuesFieldExistsQuery(PRIMARY_TERM_NAME), nested, fieldType); @@ -470,7 +470,7 @@ public void testWithNestedScoringAggregations() throws IOException { MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG); fieldType.setHasDocValues(true); fieldType.setName("nested_value"); - try (IndexReader indexReader = wrap(DirectoryReader.open(directory))) { + try (IndexReader indexReader = wrapInMockESDirectoryReader(DirectoryReader.open(directory))) { if (withScore) { diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java index 052287b0db9e1..baba27d34d7bf 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java @@ -1246,7 +1246,7 @@ public void testWithNestedAggregations() throws IOException { MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG); fieldType.setHasDocValues(true); fieldType.setName("nested_value"); - try (IndexReader indexReader = wrap(DirectoryReader.open(directory))) { + try (IndexReader indexReader = wrapInMockESDirectoryReader(DirectoryReader.open(directory))) { { InternalNested result = search(newSearcher(indexReader, false, true), // match root document only diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index 391cbc5471da0..19c93310bfba1 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -101,6 +101,7 @@ import org.elasticsearch.script.ScriptService; import org.elasticsearch.search.SearchModule; import org.elasticsearch.search.aggregations.MultiBucketConsumerService.MultiBucketConsumer; +import org.elasticsearch.search.aggregations.bucket.nested.NestedAggregationBuilder; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator.PipelineTree; import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; @@ -540,17 +541,17 @@ protected void doAssertReducedMultiBucketConsumer(Aggregation agg, MultiBucketCo } protected void testCase( - T aggregationBuilder, - Query query, - CheckedConsumer buildIndex, - Consumer verify, - MappedFieldType... fieldTypes) throws IOException { + T aggregationBuilder, + Query query, + CheckedConsumer buildIndex, + Consumer verify, + MappedFieldType... fieldTypes) throws IOException { try (Directory directory = newDirectory()) { RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory); buildIndex.accept(indexWriter); indexWriter.close(); - try (IndexReader indexReader = DirectoryReader.open(directory)) { + try (IndexReader indexReader = wrapDirectoryReader(DirectoryReader.open(directory))) { IndexSearcher indexSearcher = newSearcher(indexReader, true, true); V agg = searchAndReduce(indexSearcher, query, aggregationBuilder, fieldTypes); @@ -559,6 +560,14 @@ protected void tes } } + /** + * Override to wrap the {@linkplain DirectoryReader} for aggs like + * {@link NestedAggregationBuilder}. + */ + protected IndexReader wrapDirectoryReader(DirectoryReader reader) throws IOException { + return reader; + } + private static class ShardSearcher extends IndexSearcher { private final List ctx; @@ -577,7 +586,7 @@ public String toString() { } } - protected static DirectoryReader wrap(DirectoryReader directoryReader) throws IOException { + protected static DirectoryReader wrapInMockESDirectoryReader(DirectoryReader directoryReader) throws IOException { return ElasticsearchDirectoryReader.wrap(directoryReader, new ShardId(new Index("_index", "_na_"), 0)); } From 651673c380c6192f1e8130b58feb999e7e1018b6 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 6 May 2020 17:23:07 -0400 Subject: [PATCH 12/14] Finish up --- .../search/aggregations/AggregationPhase.java | 2 +- .../search/aggregations/Aggregator.java | 19 -------- .../MultiBucketConsumerService.java | 4 +- .../bucket/BestBucketsDeferringCollector.java | 13 ------ .../metrics/MetricsAggregator.java | 6 +++ .../bucket/GlobalAggregatorTests.java | 2 +- .../geogrid/GeoGridAggregatorTestCase.java | 2 +- .../terms/RareTermsAggregatorTests.java | 2 +- .../bucket/terms/TermsAggregatorTests.java | 44 +++++++++---------- .../metrics/AvgAggregatorTests.java | 2 +- .../metrics/MaxAggregatorTests.java | 6 +-- ...eAggregatedPercentilesAggregatorTests.java | 2 +- ...eAggregatedPercentilesAggregatorTests.java | 2 +- .../xpack/rollup/job/IndexerUtilsTests.java | 12 ++--- 14 files changed, 46 insertions(+), 72 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java b/server/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java index 5808ddf2aa5df..168880ee77241 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java @@ -125,7 +125,7 @@ public void execute(SearchContext context) { for (Aggregator aggregator : context.aggregations().aggregators()) { try { aggregator.postCollection(); - aggregations.add(aggregator.buildAggregations(new long[] {0})[0]); + aggregations.add(aggregator.buildTopLevel()); } catch (IOException e) { throw new AggregationExecutionException("Failed to build aggregation [" + aggregator.name() + "]", e); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java index c6e17a92d9c75..054bc2337f894 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java @@ -151,25 +151,6 @@ public interface BucketComparator { int compare(long lhs, long rhs); } - /** - * Build an aggregation for data that has been collected into - * {@code owningBucketOrd}. - * @deprecated use {@link #buildAggregations(long[])} instead - */ - @Deprecated - public InternalAggregation buildAggregation(long owningBucketOrd) throws IOException { - /* - * Temporarily check if it looks like we're being called from a test - * and try to answer with the top level agg. This just prevents us from - * having to modify 1231234134124 tests in one PR. - */ - if (owningBucketOrd == 0) { - return buildTopLevel(); - } - // TODO remove this method and redaclare in MetricAggregator as protected abstract - throw new UnsupportedOperationException(); - } - /** * Build the results of this aggregation. * @param owningBucketOrds the ordinals of the buckets that we want to diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/MultiBucketConsumerService.java b/server/src/main/java/org/elasticsearch/search/aggregations/MultiBucketConsumerService.java index 1fede42b694d7..a3a7e52167a7f 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/MultiBucketConsumerService.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/MultiBucketConsumerService.java @@ -34,7 +34,7 @@ /** * An aggregation service that creates instances of {@link MultiBucketConsumer}. * The consumer is used by {@link BucketsAggregator} and {@link InternalMultiBucketAggregation} to limit the number of buckets created - * in {@link Aggregator#buildAggregation} and {@link InternalAggregation#reduce}. + * in {@link Aggregator#buildAggregations} and {@link InternalAggregation#reduce}. * The limit can be set by changing the `search.max_buckets` cluster setting and defaults to 10000. */ public class MultiBucketConsumerService { @@ -94,7 +94,7 @@ protected void metadataToXContent(XContentBuilder builder, Params params) throws * An {@link IntConsumer} that throws a {@link TooManyBucketsException} * when the sum of the provided values is above the limit (`search.max_buckets`). * It is used by aggregators to limit the number of bucket creation during - * {@link Aggregator#buildAggregation} and {@link InternalAggregation#reduce}. + * {@link Aggregator#buildAggregations} and {@link InternalAggregation#reduce}. */ public static class MultiBucketConsumer implements IntConsumer { private final int limit; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java index 82a17d46072db..2c9ca8d8ae7e9 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BestBucketsDeferringCollector.java @@ -212,19 +212,6 @@ public void prepareSelectedBuckets(long... selectedBuckets) throws IOException { public Aggregator wrap(final Aggregator in) { return new WrappedAggregator(in) { - - @Override - public InternalAggregation buildAggregation(long bucket) throws IOException { - if (selectedBuckets == null) { - throw new IllegalStateException("Collection has not been replayed yet."); - } - final long rebasedBucket = selectedBuckets.find(bucket); - if (rebasedBucket == -1) { - throw new IllegalStateException("Cannot build for a bucket which has not been collected"); - } - return in.buildAggregation(rebasedBucket); - } - @Override public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { if (selectedBuckets == null) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MetricsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MetricsAggregator.java index 67e72053c1c05..005bbcb86b8b4 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MetricsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MetricsAggregator.java @@ -33,6 +33,12 @@ protected MetricsAggregator(String name, SearchContext context, Aggregator paren super(name, AggregatorFactories.EMPTY, context, parent, metadata); } + /** + * Build an aggregation for data that has been collected into + * {@code owningBucketOrd}. + */ + public abstract InternalAggregation buildAggregation(long owningBucketOrd) throws IOException; + @Override public final InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { InternalAggregation[] results = new InternalAggregation[owningBucketOrds.length]; diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GlobalAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GlobalAggregatorTests.java index 232c9f07510a5..433f08618efc6 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GlobalAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GlobalAggregatorTests.java @@ -82,7 +82,7 @@ private void testCase(CheckedConsumer buildIndex aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - InternalGlobal result = (InternalGlobal) aggregator.buildAggregation(0L); + InternalGlobal result = (InternalGlobal) aggregator.buildTopLevel(); verify.accept(result, (InternalMin) result.getAggregations().asMap().get("in_global")); indexReader.close(); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregatorTestCase.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregatorTestCase.java index 7657754b85e9e..28b3c8d4eef37 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregatorTestCase.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregatorTestCase.java @@ -248,7 +248,7 @@ private void testCase(Query query, int precision, GeoBoundingBox geoBoundingBox, aggregator.preCollection(); indexSearcher.search(query, aggregator); aggregator.postCollection(); - verify.accept((InternalGeoGrid) aggregator.buildAggregation(0L)); + verify.accept((InternalGeoGrid) aggregator.buildTopLevel()); indexReader.close(); directory.close(); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/RareTermsAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/RareTermsAggregatorTests.java index ba70ded92a998..202570a84c7ad 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/RareTermsAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/RareTermsAggregatorTests.java @@ -276,7 +276,7 @@ public void testUnmapped() throws Exception { aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - RareTerms result = (RareTerms) aggregator.buildAggregation(0L); + RareTerms result = (RareTerms) aggregator.buildTopLevel(); assertEquals("_name", result.getName()); assertEquals(0, result.getBuckets().size()); } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java index baba27d34d7bf..fd17d548020a0 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java @@ -258,7 +258,7 @@ public void testSimple() throws Exception { aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - Terms result = (Terms) aggregator.buildAggregation(0L); + Terms result = (Terms) aggregator.buildTopLevel(); assertEquals(5, result.getBuckets().size()); assertEquals("", result.getBuckets().get(0).getKeyAsString()); assertEquals(2L, result.getBuckets().get(0).getDocCount()); @@ -329,7 +329,7 @@ public void testStringIncludeExclude() throws Exception { aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - Terms result = (Terms) aggregator.buildAggregation(0L); + Terms result = (Terms) aggregator.buildTopLevel(); assertEquals(10, result.getBuckets().size()); assertEquals("val000", result.getBuckets().get(0).getKeyAsString()); assertEquals(1L, result.getBuckets().get(0).getDocCount()); @@ -366,7 +366,7 @@ public void testStringIncludeExclude() throws Exception { aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - result = (Terms) aggregator.buildAggregation(0L); + result = (Terms) aggregator.buildTopLevel(); assertEquals(5, result.getBuckets().size()); assertEquals("val001", result.getBuckets().get(0).getKeyAsString()); assertEquals(1L, result.getBuckets().get(0).getDocCount()); @@ -390,7 +390,7 @@ public void testStringIncludeExclude() throws Exception { aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - result = (Terms) aggregator.buildAggregation(0L); + result = (Terms) aggregator.buildTopLevel(); assertEquals(8, result.getBuckets().size()); assertEquals("val002", result.getBuckets().get(0).getKeyAsString()); assertEquals(1L, result.getBuckets().get(0).getDocCount()); @@ -419,7 +419,7 @@ public void testStringIncludeExclude() throws Exception { aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - result = (Terms) aggregator.buildAggregation(0L); + result = (Terms) aggregator.buildTopLevel(); assertEquals(2, result.getBuckets().size()); assertEquals("val010", result.getBuckets().get(0).getKeyAsString()); assertEquals(1L, result.getBuckets().get(0).getDocCount()); @@ -436,7 +436,7 @@ public void testStringIncludeExclude() throws Exception { aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - result = (Terms) aggregator.buildAggregation(0L); + result = (Terms) aggregator.buildTopLevel(); assertEquals(2, result.getBuckets().size()); assertEquals("val000", result.getBuckets().get(0).getKeyAsString()); assertEquals(1L, result.getBuckets().get(0).getDocCount()); @@ -454,7 +454,7 @@ public void testStringIncludeExclude() throws Exception { aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - result = (Terms) aggregator.buildAggregation(0L); + result = (Terms) aggregator.buildTopLevel(); assertEquals(2, result.getBuckets().size()); assertEquals("val000", result.getBuckets().get(0).getKeyAsString()); assertEquals(1L, result.getBuckets().get(0).getDocCount()); @@ -510,7 +510,7 @@ public void testNumericIncludeExclude() throws Exception { aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - Terms result = (Terms) aggregator.buildAggregation(0L); + Terms result = (Terms) aggregator.buildTopLevel(); assertEquals(2, result.getBuckets().size()); assertEquals(0L, result.getBuckets().get(0).getKey()); assertEquals(1L, result.getBuckets().get(0).getDocCount()); @@ -527,7 +527,7 @@ public void testNumericIncludeExclude() throws Exception { aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - result = (Terms) aggregator.buildAggregation(0L); + result = (Terms) aggregator.buildTopLevel(); assertEquals(4, result.getBuckets().size()); assertEquals(1L, result.getBuckets().get(0).getKey()); assertEquals(1L, result.getBuckets().get(0).getDocCount()); @@ -551,7 +551,7 @@ public void testNumericIncludeExclude() throws Exception { aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - result = (Terms) aggregator.buildAggregation(0L); + result = (Terms) aggregator.buildTopLevel(); assertEquals(2, result.getBuckets().size()); assertEquals(0.0, result.getBuckets().get(0).getKey()); assertEquals(1L, result.getBuckets().get(0).getDocCount()); @@ -568,7 +568,7 @@ public void testNumericIncludeExclude() throws Exception { aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - result = (Terms) aggregator.buildAggregation(0L); + result = (Terms) aggregator.buildTopLevel(); assertEquals(4, result.getBuckets().size()); assertEquals(1.0, result.getBuckets().get(0).getKey()); assertEquals(1L, result.getBuckets().get(0).getDocCount()); @@ -734,7 +734,7 @@ private void termsAggregator(ValueType valueType, MappedFieldType fieldType, aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - Terms result = (Terms) aggregator.buildAggregation(0L); + Terms result = (Terms) aggregator.buildTopLevel(); assertEquals(size, result.getBuckets().size()); for (int i = 0; i < size; i++) { Map.Entry expected = expectedBuckets.get(i); @@ -761,7 +761,7 @@ private void termsAggregator(ValueType valueType, MappedFieldType fieldType, aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - result = ((Filter) aggregator.buildAggregation(0L)).getAggregations().get("_name2"); + result = ((Filter) aggregator.buildTopLevel()).getAggregations().get("_name2"); int expectedFilteredCounts = 0; for (Integer count : filteredCounts.values()) { if (count > 0) { @@ -837,7 +837,7 @@ private void termsAggregatorWithNestedMaxAgg(ValueType valueType, MappedFiel aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - Terms result = (Terms) aggregator.buildAggregation(0L); + Terms result = (Terms) aggregator.buildTopLevel(); assertEquals(size, result.getBuckets().size()); for (int i = 0; i < size; i++) { Map.Entry expected = expectedBuckets.get(i); @@ -872,7 +872,7 @@ public void testEmpty() throws Exception { aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - Terms result = (Terms) aggregator.buildAggregation(0L); + Terms result = (Terms) aggregator.buildTopLevel(); assertEquals("_name", result.getName()); assertEquals(0, result.getBuckets().size()); @@ -882,7 +882,7 @@ public void testEmpty() throws Exception { aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - result = (Terms) aggregator.buildAggregation(0L); + result = (Terms) aggregator.buildTopLevel(); assertEquals("_name", result.getName()); assertEquals(0, result.getBuckets().size()); @@ -892,7 +892,7 @@ public void testEmpty() throws Exception { aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - result = (Terms) aggregator.buildAggregation(0L); + result = (Terms) aggregator.buildTopLevel(); assertEquals("_name", result.getName()); assertEquals(0, result.getBuckets().size()); } @@ -915,7 +915,7 @@ public void testUnmapped() throws Exception { aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - Terms result = (Terms) aggregator.buildAggregation(0L); + Terms result = (Terms) aggregator.buildTopLevel(); assertEquals("_name", result.getName()); assertEquals(0, result.getBuckets().size()); assertFalse(AggregationInspectionHelper.hasValue((InternalTerms)result)); @@ -953,7 +953,7 @@ public void testUnmappedWithMissing() throws Exception { aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - Terms result = (Terms) aggregator.buildAggregation(0L); + Terms result = (Terms) aggregator.buildTopLevel(); assertEquals("_name", result.getName()); assertEquals(1, result.getBuckets().size()); assertEquals(missingValues[i], result.getBuckets().get(0).getKey()); @@ -1034,7 +1034,7 @@ public void testIpField() throws Exception { aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - Terms result = (Terms) aggregator.buildAggregation(0L); + Terms result = (Terms) aggregator.buildTopLevel(); assertEquals("_name", result.getName()); assertEquals(1, result.getBuckets().size()); assertEquals("192.168.100.42", result.getBuckets().get(0).getKey()); @@ -1086,7 +1086,7 @@ public void testNestedTermsAgg() throws Exception { aggregator.preCollection(); indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - Terms result = (Terms) aggregator.buildAggregation(0L); + Terms result = (Terms) aggregator.buildTopLevel(); assertEquals(3, result.getBuckets().size()); assertEquals("a", result.getBuckets().get(0).getKeyAsString()); assertEquals(1L, result.getBuckets().get(0).getDocCount()); @@ -1448,7 +1448,7 @@ private InternalAggregation buildInternalAggregation(TermsAggregationBuilder bui aggregator.preCollection(); searcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - return aggregator.buildAggregation(0L); + return aggregator.buildTopLevel(); } } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/AvgAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/AvgAggregatorTests.java index 7a69afd8a29e9..f119c64d5a610 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/AvgAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/AvgAggregatorTests.java @@ -546,7 +546,7 @@ public void testOrderByEmptyAggregation() throws IOException { indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - Terms terms = (Terms) aggregator.buildAggregation(0L); + Terms terms = (Terms) aggregator.buildTopLevel(); assertNotNull(terms); List buckets = terms.getBuckets(); assertNotNull(buckets); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/MaxAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/MaxAggregatorTests.java index 10e9568421483..0e3eb54b46b9a 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/MaxAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/MaxAggregatorTests.java @@ -463,7 +463,7 @@ public void testSingleValuedFieldGetProperty() throws IOException { indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - Global global = (Global) aggregator.buildAggregation(0L); + Global global = (Global) aggregator.buildTopLevel(); assertNotNull(global); assertEquals("global", global.getName()); assertEquals(10L, global.getDocCount()); @@ -729,7 +729,7 @@ public void testEmptyAggregation() throws Exception { indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - Global global = (Global) aggregator.buildAggregation(0L); + Global global = (Global) aggregator.buildTopLevel(); assertNotNull(global); assertEquals("global", global.getName()); assertEquals(0L, global.getDocCount()); @@ -772,7 +772,7 @@ public void testOrderByEmptyAggregation() throws IOException { indexSearcher.search(new MatchAllDocsQuery(), aggregator); aggregator.postCollection(); - Terms terms = (Terms) aggregator.buildAggregation(0L); + Terms terms = (Terms) aggregator.buildTopLevel(); assertNotNull(terms); List buckets = terms.getBuckets(); assertNotNull(buckets); diff --git a/x-pack/plugin/analytics/src/test/java/org/elasticsearch/xpack/analytics/aggregations/metrics/HDRPreAggregatedPercentilesAggregatorTests.java b/x-pack/plugin/analytics/src/test/java/org/elasticsearch/xpack/analytics/aggregations/metrics/HDRPreAggregatedPercentilesAggregatorTests.java index e881b29c3aa33..cd2ebe3ff8b3c 100644 --- a/x-pack/plugin/analytics/src/test/java/org/elasticsearch/xpack/analytics/aggregations/metrics/HDRPreAggregatedPercentilesAggregatorTests.java +++ b/x-pack/plugin/analytics/src/test/java/org/elasticsearch/xpack/analytics/aggregations/metrics/HDRPreAggregatedPercentilesAggregatorTests.java @@ -153,7 +153,7 @@ private void testCase(Query query, CheckedConsumer Date: Wed, 6 May 2020 18:07:08 -0400 Subject: [PATCH 13/14] geo! --- .../aggregations/bucket/geogrid/GeoShapeGeoGridTestCase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/bucket/geogrid/GeoShapeGeoGridTestCase.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/bucket/geogrid/GeoShapeGeoGridTestCase.java index 6c54c24f45e75..41fcb0b9c5316 100644 --- a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/bucket/geogrid/GeoShapeGeoGridTestCase.java +++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/bucket/geogrid/GeoShapeGeoGridTestCase.java @@ -306,7 +306,7 @@ private void testCase(Query query, int precision, GeoBoundingBox geoBoundingBox, indexSearcher.search(query, aggregator); aggregator.postCollection(); - verify.accept((InternalGeoGrid) aggregator.buildAggregation(0L)); + verify.accept((InternalGeoGrid) aggregator.buildTopLevel()); indexReader.close(); directory.close(); From 7f88ea81d5af81f0ad91c7e23c2b6db39ed3a692 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 8 May 2020 13:18:59 -0400 Subject: [PATCH 14/14] Explain --- .../search/aggregations/AggregatorFactory.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java index af12fdba98c86..22fef9bea13f0 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java @@ -152,6 +152,12 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I if (owningBucketOrds[ordIdx] < aggregators.size()) { Aggregator aggregator = aggregators.get(owningBucketOrds[ordIdx]); if (aggregator != null) { + /* + * This is the same call as buildTopLevel but since + * this aggregator may not be the top level we don't + * call that method here. It'd be weird sounding. And + * it'd trip assertions. Both bad. + */ results[ordIdx] = aggregator.buildAggregations(new long [] {0})[0]; } else { results[ordIdx] = buildEmptyAggregation();