From 342fb4eb66d453c2ec9ce3360c446123c3cdf9c7 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 19 May 2020 14:50:12 -0400 Subject: [PATCH 01/18] Save memory when auto_date_histogram is not on top This rebuilds `auto_date_histogram`'s aggregator to function without `asMultiBucketAggregator` which should save a significant amount of memory when it is not the top most aggregator. It isn't possible to "just port the aggregator" without taking a pretty significant performance hit because we used to rewrite all of the buckets every time we switched to a coarser and coarser rounding configuration. Without some major surgery to how to delay sub-aggs we'd end up rewriting the delay list zillions of time if there are many buckets. This change replaces the constant rewrites with a "budget" of "wasted" buckets and only rewrites all of the buckets when we exceed that budget. Now that we don't rebucket every time we increase the rounding we can no longer get an accurate count of the number of buckets! So instead the aggregator uses an esimate of the number of buckets to trigger switching to a coarser rounding. This estimate is likely to be *terrible* when buckets are far apart compared to the rounding. So it also uses the difference between the first and last bucket to trigger switching to a coarser rounding. Which covers for the shortcomings of the bucket estimation technique pretty well. It also causes the aggregator to emit fewer buckets in cases where they'd be reduced together on the coordinating node. This is wonderful! But probably fairly rare. After all that, it amounts to about the same performance, in the benchmarks that I've run. But the memory savings is totaly still at thing! Relates to #56487 --- .../bucket/BucketsAggregator.java | 20 +- .../bucket/DeferableBucketAggregator.java | 11 - .../AutoDateHistogramAggregationBuilder.java | 4 + .../AutoDateHistogramAggregator.java | 341 ++++++++++-- .../AutoDateHistogramAggregatorFactory.java | 7 +- .../AutoDateHistogramAggregatorSupplier.java | 1 + .../histogram/DateHistogramAggregator.java | 2 +- .../DateRangeHistogramAggregator.java | 2 +- .../bucket/terms/LongKeyedBucketOrds.java | 21 +- .../AutoDateHistogramAggregatorTests.java | 501 +++++++++++------- .../DateHistogramAggregatorTestCase.java | 114 ++++ .../DateHistogramAggregatorTests.java | 71 +-- .../terms/LongKeyedBucketOrdsTests.java | 5 + 13 files changed, 770 insertions(+), 330 deletions(-) create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorTestCase.java 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 dca1c94feab0f..72a81fb408cd5 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 @@ -30,6 +30,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.LeafBucketCollector; +import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregator; import org.elasticsearch.search.aggregations.bucket.terms.LongKeyedBucketOrds; import org.elasticsearch.search.aggregations.support.AggregationPath; import org.elasticsearch.search.internal.SearchContext; @@ -351,7 +352,7 @@ protected final InternalAggregation[] buildAggregationsForVariableBuckets(lo * @param bucketOrds hash of values to the bucket ordinal */ protected final InternalAggregation[] buildAggregationsForVariableBuckets(long[] owningBucketOrds, LongKeyedBucketOrds bucketOrds, - BucketBuilderForVariable bucketBuilder, Function, InternalAggregation> resultBuilder) throws IOException { + BucketBuilderForVariable bucketBuilder, ResultBuilderForVariable resultBuilder) throws IOException { long totalOrdsToCollect = 0; for (int ordIdx = 0; ordIdx < owningBucketOrds.length; ordIdx++) { totalOrdsToCollect += bucketOrds.bucketsInOrd(owningBucketOrds[ordIdx]); @@ -383,7 +384,7 @@ protected final InternalAggregation[] buildAggregationsForVariableBuckets(lo } buckets.add(bucketBuilder.build(ordsEnum.value(), bucketDocCount(ordsEnum.ord()), subAggregationResults[b++])); } - results[ordIdx] = resultBuilder.apply(buckets); + results[ordIdx] = resultBuilder.build(owningBucketOrds[ordIdx], buckets); } return results; } @@ -391,6 +392,10 @@ protected final InternalAggregation[] buildAggregationsForVariableBuckets(lo protected interface BucketBuilderForVariable { B build(long bucketValue, int docCount, InternalAggregations subAggregationResults); } + @FunctionalInterface + protected interface ResultBuilderForVariable { + InternalAggregation build(long owninigBucketOrd, List buckets); + } /** * Utility method to build empty aggregations of the sub aggregators. @@ -430,4 +435,15 @@ public BucketComparator bucketComparator(String key, SortOrder order) { "Either drop the key (a la \"" + name() + "\") or change it to \"doc_count\" (a la \"" + name() + ".doc_count\") or \"key\"."); } + + public static boolean descendsFromGlobalAggregator(Aggregator parent) { + while (parent != null) { + if (parent.getClass() == GlobalAggregator.class) { + return true; + } + parent = parent.parent(); + } + return false; + } + } 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 807ec17e46358..d2499611e4ea5 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 @@ -23,7 +23,6 @@ import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.BucketCollector; import org.elasticsearch.search.aggregations.MultiBucketCollector; -import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregator; import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; @@ -70,16 +69,6 @@ protected void doPreCollection() throws IOException { collectableSubAggregators = MultiBucketCollector.wrap(collectors); } - public static boolean descendsFromGlobalAggregator(Aggregator parent) { - while (parent != null) { - if (parent.getClass() == GlobalAggregator.class) { - return true; - } - parent = parent.parent(); - } - return false; - } - public DeferringBucketCollector getDeferringCollector() { // Default impl is a collector that selects the best buckets // but an alternative defer policy may be based on best docs. diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java index 3ac8b5b314e66..a97c947744792 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java @@ -290,6 +290,10 @@ public long getRoughEstimateDurationMillis() { return roughEstimateDurationMillis; } + public long getMaximumRoughEstimateDurationMillis() { + return getRoughEstimateDurationMillis() * getMaximumInnerInterval(); + } + @Override public int hashCode() { return Objects.hash(rounding, Arrays.hashCode(innerIntervals), dateTimeUnit); 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 3725c24deebb9..2a982bfc8f801 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 @@ -25,6 +25,9 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Rounding; import org.elasticsearch.common.lease.Releasables; +import org.elasticsearch.common.util.ByteArray; +import org.elasticsearch.common.util.IntArray; +import org.elasticsearch.common.util.LongArray; import org.elasticsearch.common.util.LongHash; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.Aggregator; @@ -37,35 +40,132 @@ import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector; import org.elasticsearch.search.aggregations.bucket.MergingBucketsDeferringCollector; import org.elasticsearch.search.aggregations.bucket.histogram.AutoDateHistogramAggregationBuilder.RoundingInfo; +import org.elasticsearch.search.aggregations.bucket.terms.LongKeyedBucketOrds; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; import java.util.Collections; import java.util.Map; +import java.util.function.BiConsumer; import java.util.function.Function; /** * An aggregator for date values that attempts to return a specific number of * buckets, reconfiguring how it rounds dates to buckets on the fly as new - * data arrives. + * data arrives. + *

+ * Initially it uses the most fine grained rounding configuration possible but + * as more data arrives it uses two heuristics to shift to coarser and coarser + * rounding. The first heuristic is the number of buckets, specifically, it + * when there are more buckets than can "fit" in the current rounding it shifts + * to the next rounding. Instead of redoing the rounding, it estimates the + * number of buckets that will "survive" at the new rounding and uses + * that as the initial value for the bucket count that it + * increments in order to trigger another promotion to another coarser + * rounding. This works fairly well at containing the number of buckets, but + * it the estimate of the number of buckets will be wrong if the buckets are + * quite a spread out compared to the rounding. + *

+ * The second heuristic it uses to trigger promotion to a coarser rounding is + * the distance between the min and max bucket. When that distance is greater + * than what the current rounding supports it promotes. This is heuristic + * isn't good at limiting the number of buckets but is great when the buckets + * are spread out compared to the rounding. So it should complement the first + * heuristic. + *

+ * When promoting a rounding we keep the old buckets around because it is + * expensive to call {@link MergingBucketsDeferringCollector#mergeBuckets}. + * In particular it is {@code O(number_of_hits_collected_so_far)}. So if we + * called it frequently we'd end up in {@code O(n^2)} territory. Bad news for + * aggregations! Instead, we keep a "budget" of buckets that we're ok + * "wasting". When we promote the rounding and our estimate of the number of + * "dead" buckets that have data but have yet to be merged into the buckets + * that are valid for the current rounding exceeds the budget then we rebucket + * the entire aggregation and double the budget. + *

+ * Once we're done collecting and we know exactly which buckets we'll be + * returning we finally perform a "real", "perfect bucketing", + * rounding all of the keys for {@code owningBucketOrd} that we're going to + * collect and picking the rounding based on a real, accurate count and the + * min and max. */ class AutoDateHistogramAggregator extends DeferableBucketAggregator { - private final ValuesSource.Numeric valuesSource; private final DocValueFormat formatter; private final RoundingInfo[] roundingInfos; private final Function roundingPreparer; - private int roundingIdx = 0; - private Rounding.Prepared preparedRounding; - - private LongHash bucketOrds; - private int targetBuckets; + private final int targetBuckets; + private final boolean collectsFromSingleBucket; + /** + * An array of prepared roundings in the same order as + * {@link #roundingInfos}. The 0th entry is prepared initially, + * and other entries are null until first needed. + */ + private final Rounding.Prepared[] preparedRoundings; + /** + * Map from {@code owningBucketOrd, roundedDate} to {@code bucketOrdinal}. + */ + private LongKeyedBucketOrds bucketOrds; + /** + * The index of the rounding that each {@code owningBucketOrd} is + * currently using. + *

+ * During collection we use overestimates for how much buckets are save + * by bumping to the next rounding index. So we end up bumping less + * aggressively than a "perfect" algorithm. That is fine because we + * correct the error when we merge the buckets together all the way + * up in {@link InternalAutoDateHistogram#reduceBucket}. In particular, + * on final reduce we bump the rounding until it we appropriately + * cover the date range across all of the results returned by all of + * the {@link AutoDateHistogramAggregator}s. + */ + private ByteArray roundingIndices; + /** + * The min and max of each bucket's keys. min lives in indices of the form + * {@code 2n} and max in {@code 2n + 1}. + */ + private LongArray bounds; + /** + * A reference to the collector so we can + * {@link MergingBucketsDeferringCollector#mergeBuckets(long[])}. + */ private MergingBucketsDeferringCollector deferringCollector; + /** + * An underestimate of the number of buckets that are "live" in the + * current rounding for each {@code owningBucketOrdinal}. + */ + private IntArray liveBucketCountUnderestimate; + /** + * An over estimate of the number of wasted buckets. When this gets + * too high we {@link #rebucket()} which sets it to 0. + */ + private long wastedBucketsOverestimate = 0; + /** + * The next {@link #wastedBucketsOverestimate} that will trigger a + * {@link #rebucket() rebucketing}. + */ + private long nextRebucketAt = 1000; // TODO this could almost certainly start higher when asMultiBucketAggregator is gone + /** + * The number of times the aggregator had to {@link #rebucket()} the + * results. We keep this just to report to the profiler. + */ + private int rebucketCount = 0; - AutoDateHistogramAggregator(String name, AggregatorFactories factories, int numBuckets, RoundingInfo[] roundingInfos, - Function roundingPreparer, @Nullable ValuesSource valuesSource, DocValueFormat formatter, - SearchContext aggregationContext, Aggregator parent, Map metadata) throws IOException { + AutoDateHistogramAggregator( + String name, + AggregatorFactories factories, + int numBuckets, + RoundingInfo[] roundingInfos, + Function roundingPreparer, + @Nullable ValuesSource valuesSource, + DocValueFormat formatter, + SearchContext aggregationContext, + Aggregator parent, + boolean collectsFromSingleBucket, + Map metadata + ) + throws IOException { super(name, factories, aggregationContext, parent, metadata); this.targetBuckets = numBuckets; @@ -73,10 +173,17 @@ class AutoDateHistogramAggregator extends DeferableBucketAggregator { this.formatter = formatter; this.roundingInfos = roundingInfos; this.roundingPreparer = roundingPreparer; - preparedRounding = roundingPreparer.apply(roundingInfos[roundingIdx].rounding); - - bucketOrds = new LongHash(1, aggregationContext.bigArrays()); - + this.collectsFromSingleBucket = collectsFromSingleBucket; + assert roundingInfos.length < 127 : "Rounding must fit in a signed byte"; + roundingIndices = context.bigArrays().newByteArray(1, true); + bounds = context.bigArrays().newLongArray(2, true); + bounds.set(0, Long.MAX_VALUE); + bounds.set(1, Long.MIN_VALUE); + preparedRoundings = new Rounding.Prepared[roundingInfos.length]; + // Prepare the first rounding because we know we'll need it. + preparedRoundings[0] = roundingPreparer.apply(roundingInfos[0].rounding); + bucketOrds = LongKeyedBucketOrds.build(context.bigArrays(), collectsFromSingleBucket); + liveBucketCountUnderestimate = context.bigArrays().newIntArray(1, true); } @Override @@ -99,95 +206,219 @@ public DeferringBucketCollector getDeferringCollector() { } @Override - public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, - final LeafBucketCollector sub) throws IOException { + public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException { if (valuesSource == null) { return LeafBucketCollector.NO_OP_COLLECTOR; } - final SortedNumericDocValues values = valuesSource.longValues(ctx); + SortedNumericDocValues values = valuesSource.longValues(ctx); return new LeafBucketCollectorBase(sub, values) { @Override - public void collect(int doc, long bucket) throws IOException { - assert bucket == 0; + public void collect(int doc, long owningBucketOrd) throws IOException { if (values.advanceExact(doc)) { - final int valuesCount = values.docValueCount(); + int valuesCount = values.docValueCount(); long previousRounded = Long.MIN_VALUE; + byte roundingIdx = roundingIndexFor(owningBucketOrd); for (int i = 0; i < valuesCount; ++i) { long value = values.nextValue(); - long rounded = preparedRounding.round(value); + long rounded = preparedRoundings[roundingIdx].round(value); assert rounded >= previousRounded; if (rounded == previousRounded) { continue; } - long bucketOrd = bucketOrds.add(rounded); - if (bucketOrd < 0) { // already seen - bucketOrd = -1 - bucketOrd; - collectExistingBucket(sub, doc, bucketOrd); - } else { - collectBucket(sub, doc, bucketOrd); - while (roundingIdx < roundingInfos.length - 1 - && bucketOrds.size() > (targetBuckets * roundingInfos[roundingIdx].getMaximumInnerInterval())) { - increaseRounding(); - } - } + roundingIdx = collectValue(sub, owningBucketOrd, roundingIdx, doc, rounded); previousRounded = rounded; } } } - private void increaseRounding() { - try (LongHash oldBucketOrds = bucketOrds) { - LongHash newBucketOrds = new LongHash(1, context.bigArrays()); - long[] mergeMap = new long[(int) oldBucketOrds.size()]; - preparedRounding = roundingPreparer.apply(roundingInfos[++roundingIdx].rounding); - for (int i = 0; i < oldBucketOrds.size(); i++) { - long oldKey = oldBucketOrds.get(i); - long newKey = preparedRounding.round(oldKey); - long newBucketOrd = newBucketOrds.add(newKey); - if (newBucketOrd >= 0) { - mergeMap[i] = newBucketOrd; - } else { - mergeMap[i] = -1 - newBucketOrd; - } - } - mergeBuckets(mergeMap, newBucketOrds.size()); - if (deferringCollector != null) { - deferringCollector.mergeBuckets(mergeMap); + private byte collectValue(LeafBucketCollector sub, long owningBucketOrd, byte roundingIdx, int doc, long rounded) + throws IOException { + long bucketOrd = bucketOrds.add(owningBucketOrd, rounded); + if (bucketOrd < 0) { // already seen + bucketOrd = -1 - bucketOrd; + collectExistingBucket(sub, doc, bucketOrd); + return roundingIdx; + } + collectBucket(sub, doc, bucketOrd); + liveBucketCountUnderestimate = context.bigArrays().grow(liveBucketCountUnderestimate, owningBucketOrd + 1); + int estimatedBucketCount = liveBucketCountUnderestimate.increment(owningBucketOrd, 1); + return increaseRounding(owningBucketOrd, estimatedBucketCount, rounded, roundingIdx); + } + + /** + * Increase the rounding of {@code owningBucketOrd} using + * estimated, bucket counts, {@link #rebucket() rebucketing} the all + * buckets if the estimated number of wasted buckets is too high. + */ + private byte increaseRounding(long owningBucketOrd, int oldEstimatedBucketCount, long newKey, byte oldRounding) { + if (oldRounding >= roundingInfos.length - 1) { + return oldRounding; + } + if (bounds.size() < owningBucketOrd * 2 + 2) { + long oldSize = bounds.size(); + bounds = context.bigArrays().grow(bounds, owningBucketOrd * 2 + 2); + for (long b = oldSize; b < bounds.size(); b++) { + bounds.set(b, (b & 1L) == 0L ? Long.MAX_VALUE : Long.MIN_VALUE); } - bucketOrds = newBucketOrds; } + long min = Math.min(bounds.get(owningBucketOrd * 2), newKey); + bounds.set(owningBucketOrd * 2, min); + long max = Math.max(bounds.get(owningBucketOrd * 2 + 1), newKey); + bounds.set(owningBucketOrd * 2 + 1, max); + if (oldEstimatedBucketCount <= targetBuckets * roundingInfos[oldRounding].getMaximumInnerInterval() + && max - min <= targetBuckets * roundingInfos[oldRounding].getMaximumRoughEstimateDurationMillis()) { + return oldRounding; + } + long oldRoughDuration = roundingInfos[oldRounding].roughEstimateDurationMillis; + byte newRounding = oldRounding; + int newEstimatedBucketCount; + do { + newRounding++; + double ratio = (double) oldRoughDuration / (double) roundingInfos[newRounding].getRoughEstimateDurationMillis(); + newEstimatedBucketCount = (int) Math.ceil(oldEstimatedBucketCount * ratio); + } while (newRounding < roundingInfos.length - 1 && ( + newEstimatedBucketCount > targetBuckets * roundingInfos[newRounding].getMaximumInnerInterval() + || max - min > targetBuckets * roundingInfos[newRounding].getMaximumRoughEstimateDurationMillis())); + setRounding(owningBucketOrd, newRounding); + bounds.set(owningBucketOrd * 2, preparedRoundings[newRounding].round(bounds.get(owningBucketOrd * 2))); + bounds.set(owningBucketOrd * 2 + 1, preparedRoundings[newRounding].round(bounds.get(owningBucketOrd * 2 + 1))); + wastedBucketsOverestimate += oldEstimatedBucketCount - newEstimatedBucketCount; + if (wastedBucketsOverestimate > nextRebucketAt) { + rebucket(); + // Bump the threshold for the next rebucketing + wastedBucketsOverestimate = 0; + nextRebucketAt *= 2; + } else { + liveBucketCountUnderestimate.set(owningBucketOrd, newEstimatedBucketCount); + } + return newRounding; } }; } + private void rebucket() { + rebucketCount++; + try (LongKeyedBucketOrds oldOrds = bucketOrds) { + long[] mergeMap = new long[Math.toIntExact(oldOrds.size())]; + bucketOrds = LongKeyedBucketOrds.build(context.bigArrays(), collectsFromSingleBucket); + for (long owningBucketOrd = 0; owningBucketOrd <= oldOrds.maxOwningBucketOrd(); owningBucketOrd++) { + LongKeyedBucketOrds.BucketOrdsEnum ordsEnum = oldOrds.ordsEnum(owningBucketOrd); + Rounding.Prepared preparedRounding = preparedRoundings[roundingIndexFor(owningBucketOrd)]; + while (ordsEnum.next()) { + long oldKey = ordsEnum.value(); + long newKey = preparedRounding.round(oldKey); + long newBucketOrd = bucketOrds.add(owningBucketOrd, newKey); + mergeMap[(int) ordsEnum.ord()] = newBucketOrd >= 0 ? newBucketOrd : -1 - newBucketOrd; + } + liveBucketCountUnderestimate = context.bigArrays().grow(liveBucketCountUnderestimate, owningBucketOrd + 1); + liveBucketCountUnderestimate.set(owningBucketOrd, Math.toIntExact(bucketOrds.bucketsInOrd(owningBucketOrd))); + } + mergeBuckets(mergeMap, bucketOrds.size()); + if (deferringCollector != null) { + deferringCollector.mergeBuckets(mergeMap); + } + } + } + @Override public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + correctRounding(owningBucketOrds); + /* + * Now that we have the perfect rounding rebucket everything to merge + * all of the buckets together that we were too lazy to merge while + * collecting. + * + * TODO it'd be faster if we could apply the merging on the fly as we + * replay the hits and build the buckets. How much faster is + * *interesting*. Performance tests with a couple of sub-`stats` aggs + * show `date_histogram` to have about the same performance as + * `auto_date_histogram` so there isn't really much to be gained here. + * But if there is a non-delaying but selectivate aggregation "above" + * this one then the performance gain could be substantial. + */ + rebucket(); return buildAggregationsForVariableBuckets(owningBucketOrds, bucketOrds, (bucketValue, docCount, subAggregationResults) -> new InternalAutoDateHistogram.Bucket(bucketValue, docCount, formatter, subAggregationResults), - buckets -> { + (owningBucketOrd, 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()); + roundingIndexFor(owningBucketOrd), buildEmptySubAggregations()); return new InternalAutoDateHistogram(name, buckets, targetBuckets, emptyBucketInfo, formatter, metadata(), 1); }); } + /** + * Pick the correct rounding for the specifies {@code owningBucketOrds}. + */ + private void correctRounding(long[] owningBucketOrds) { + for (long owningBucketOrd : owningBucketOrds) { + byte oldRounding = roundingIndexFor(owningBucketOrd); + if (oldRounding >= roundingInfos.length - 1) { + continue; + } + byte newRounding = (byte)(oldRounding - 1); + long count; + long min = Long.MAX_VALUE; + long max = Long.MIN_VALUE; + do { + newRounding++; + try (LongHash perfect = new LongHash(liveBucketCountUnderestimate.get(owningBucketOrd), context.bigArrays())) { + LongKeyedBucketOrds.BucketOrdsEnum ordsEnum = bucketOrds.ordsEnum(owningBucketOrd); + Rounding.Prepared preparedRounding = preparedRoundings[roundingIndexFor(owningBucketOrd)]; + while (ordsEnum.next()) { + long oldKey = ordsEnum.value(); + long newKey = preparedRounding.round(oldKey); + min = Math.min(min, newKey); + max = Math.max(max, newKey); + bucketOrds.add(owningBucketOrd, newKey); + } + count = perfect.size(); + } + } while (newRounding < roundingInfos.length - 1 && ( + count > targetBuckets * roundingInfos[newRounding].getMaximumInnerInterval() + || max - min > targetBuckets * roundingInfos[newRounding].getMaximumRoughEstimateDurationMillis())); + setRounding(owningBucketOrd, newRounding); + wastedBucketsOverestimate += bucketOrds.bucketsInOrd(owningBucketOrd) - count; + } + } + + private void setRounding(long owningBucketOrd, byte newRounding) { + roundingIndices = context.bigArrays().grow(roundingIndices, owningBucketOrd + 1); + roundingIndices.set(owningBucketOrd, newRounding); + if (preparedRoundings[newRounding] == null) { + preparedRoundings[newRounding] = roundingPreparer.apply(roundingInfos[newRounding].rounding); + } + } + @Override public InternalAggregation buildEmptyAggregation() { - InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundingInfos, roundingIdx, + InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundingInfos, 0, buildEmptySubAggregations()); return new InternalAutoDateHistogram(name, Collections.emptyList(), targetBuckets, emptyBucketInfo, formatter, metadata(), 1); } + @Override + public void collectDebugInfo(BiConsumer add) { + super.collectDebugInfo(add); + add.accept("surviving_buckets", bucketOrds.size()); + add.accept("wasted_buckets_overestimate", wastedBucketsOverestimate); + add.accept("next_rebucket_at", nextRebucketAt); + add.accept("rebucket_count", rebucketCount); + } + + private byte roundingIndexFor(long owningBucketOrd) { + return owningBucketOrd < roundingIndices.size() ? roundingIndices.get(owningBucketOrd) : 0; + } + @Override public void doClose() { - Releasables.close(bucketOrds); + Releasables.close(bucketOrds, roundingIndices, bounds, liveBucketCountUnderestimate); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java index 36d43a4515436..8e425a2d209ff 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java @@ -69,9 +69,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(), AutoDateHistogramAggregationBuilder.NAME); if (aggregatorSupplier instanceof AutoDateHistogramAggregatorSupplier == false) { @@ -81,7 +78,7 @@ protected Aggregator doCreateInternal(ValuesSource valuesSource, Function roundingPreparer = valuesSource.roundingPreparer(searchContext.getQueryShardContext().getIndexReader()); return ((AutoDateHistogramAggregatorSupplier) aggregatorSupplier).build(name, factories, numBuckets, roundingInfos, - roundingPreparer, valuesSource, config.format(), searchContext, parent, metadata); + roundingPreparer, valuesSource, config.format(), searchContext, parent, collectsFromSingleBucket, metadata); } @Override @@ -89,6 +86,6 @@ protected Aggregator createUnmapped(SearchContext searchContext, Aggregator parent, Map metadata) throws IOException { return new AutoDateHistogramAggregator(name, factories, numBuckets, roundingInfos, Rounding::prepareForUnknown, null, - config.format(), searchContext, parent, metadata); + config.format(), searchContext, parent, false, metadata); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorSupplier.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorSupplier.java index e5c286f431c1b..80cac7d89f23e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorSupplier.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorSupplier.java @@ -45,6 +45,7 @@ Aggregator build( DocValueFormat formatter, SearchContext aggregationContext, Aggregator parent, + boolean collectsFromSingleBucket, Map metadata ) throws IOException; } 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 b3437cd7d9330..a33f40fb56b36 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 @@ -132,7 +132,7 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I return buildAggregationsForVariableBuckets(owningBucketOrds, bucketOrds, (bucketValue, docCount, subAggregationResults) -> { return new InternalDateHistogram.Bucket(bucketValue, docCount, keyed, formatter, subAggregationResults); - }, buckets -> { + }, (owningBucketOrd, 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()); 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 860c679381be3..6c27da8c38f41 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 @@ -159,7 +159,7 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I return buildAggregationsForVariableBuckets(owningBucketOrds, bucketOrds, (bucketValue, docCount, subAggregationResults) -> new InternalDateHistogram.Bucket(bucketValue, docCount, keyed, formatter, subAggregationResults), - buckets -> { + (owningBucketOrd, 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()); 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 cd49e47afde62..da3525085e937 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 @@ -47,7 +47,10 @@ private LongKeyedBucketOrds() {} public abstract long add(long owningBucketOrd, long value); /** - * Count the buckets in {@code owningBucketOrd}. + * The buckets in {@code owningBucketOrd}. + *

+ * Some aggregations expect this to be fast but most wouldn't + * mind particularly if it weren't. */ public abstract long bucketsInOrd(long owningBucketOrd); @@ -56,6 +59,11 @@ private LongKeyedBucketOrds() {} */ public abstract long size(); + /** + * The maximum possible used {@code owningBucketOrd}. + */ + public abstract long maxOwningBucketOrd(); + /** * Build an iterator for buckets inside {@code owningBucketOrd} in order * of increasing ord. @@ -96,7 +104,6 @@ public interface BucketOrdsEnum { }; } - /** * Implementation that only works if it is collecting from a single bucket. */ @@ -124,6 +131,11 @@ public long size() { return ords.size(); } + @Override + public long maxOwningBucketOrd() { + return 0; + } + @Override public BucketOrdsEnum ordsEnum(long owningBucketOrd) { assert owningBucketOrd == 0; @@ -234,6 +246,11 @@ public long size() { return lastGlobalOrd + 1; } + @Override + public long maxOwningBucketOrd() { + return owningOrdToBuckets.size() - 1; + } + @Override public BucketOrdsEnum ordsEnum(long owningBucketOrd) { if (owningBucketOrd >= owningOrdToBuckets.size()) { diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java index f6e28f7615fbc..98d80930fc43c 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java @@ -22,33 +22,40 @@ import org.apache.lucene.document.Document; import org.apache.lucene.document.LongPoint; import org.apache.lucene.document.SortedNumericDocValuesField; +import org.apache.lucene.document.SortedSetDocValuesField; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.common.CheckedBiConsumer; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.time.DateFormatter; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.DateFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.NumberFieldMapper; +import org.elasticsearch.search.aggregations.AggregationBuilder; import org.elasticsearch.search.aggregations.AggregationBuilders; -import org.elasticsearch.search.aggregations.AggregatorTestCase; import org.elasticsearch.search.aggregations.MultiBucketConsumerService; +import org.elasticsearch.search.aggregations.bucket.terms.StringTerms; +import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.InternalMax; import org.elasticsearch.search.aggregations.metrics.InternalStats; +import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder; import org.elasticsearch.search.aggregations.pipeline.DerivativePipelineAggregationBuilder; import org.elasticsearch.search.aggregations.pipeline.InternalSimpleValue; import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper; import org.hamcrest.Matchers; import java.io.IOException; +import java.time.Instant; import java.time.LocalDate; import java.time.YearMonth; import java.time.ZoneOffset; @@ -57,14 +64,20 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Locale; import java.util.Map; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; import java.util.function.Consumer; -import java.util.stream.Collectors; +import static org.hamcrest.Matchers.either; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasEntry; +import static org.hamcrest.Matchers.hasSize; -public class AutoDateHistogramAggregatorTests extends AggregatorTestCase { +public class AutoDateHistogramAggregatorTests extends DateHistogramAggregatorTestCase { private static final String DATE_FIELD = "date"; private static final String INSTANT_FIELD = "instant"; private static final String NUMERIC_FIELD = "numeric"; @@ -94,19 +107,22 @@ public void testMatchNoDocs() throws IOException { } public void testMatchAllDocs() throws IOException { + Map expectedDocCount = new TreeMap<>(); + expectedDocCount.put("2010-01-01T00:00:00.000Z", 2); + expectedDocCount.put("2012-01-01T00:00:00.000Z", 1); + expectedDocCount.put("2013-01-01T00:00:00.000Z", 2); + expectedDocCount.put("2015-01-01T00:00:00.000Z", 3); + expectedDocCount.put("2016-01-01T00:00:00.000Z", 1); + expectedDocCount.put("2017-01-01T00:00:00.000Z", 1); testSearchCase(DEFAULT_QUERY, DATES_WITH_TIME, - aggregation -> aggregation.setNumBuckets(6).field(DATE_FIELD), - histogram -> { - assertEquals(10, histogram.getBuckets().size()); - assertTrue(AggregationInspectionHelper.hasValue(histogram)); - } + aggregation -> aggregation.setNumBuckets(8).field(DATE_FIELD), + result -> assertThat(bucketCountsAsMap(result), equalTo(expectedDocCount)) ); + expectedDocCount.put("2011-01-01T00:00:00.000Z", 0); + expectedDocCount.put("2014-01-01T00:00:00.000Z", 0); testSearchAndReduceCase(DEFAULT_QUERY, DATES_WITH_TIME, aggregation -> aggregation.setNumBuckets(8).field(DATE_FIELD), - histogram -> { - assertEquals(8, histogram.getBuckets().size()); - assertTrue(AggregationInspectionHelper.hasValue(histogram)); - } + result -> assertThat(bucketCountsAsMap(result), equalTo(expectedDocCount)) ); } @@ -193,6 +209,177 @@ public void testSubAggregations() throws IOException { }); } + public void testAsSubAgg() throws IOException { + AggregationBuilder builder = new TermsAggregationBuilder("k1").field("k1").subAggregation( + new AutoDateHistogramAggregationBuilder("dh").field(AGGREGABLE_DATE).setNumBuckets(3).subAggregation( + new MaxAggregationBuilder("max").field("n"))); + asSubAggTestCase(builder, (StringTerms terms) -> { + StringTerms.Bucket a = terms.getBucketByKey("a"); + InternalAutoDateHistogram adh = a.getAggregations().get("dh"); + Map expectedDocCount = new TreeMap<>(); + expectedDocCount.put("2020-01-01T00:00:00.000Z", 2); + expectedDocCount.put("2021-01-01T00:00:00.000Z", 2); + assertThat(bucketCountsAsMap(adh), equalTo(expectedDocCount)); + Map expectedMax = new TreeMap<>(); + expectedMax.put("2020-01-01T00:00:00.000Z", 2.0); + expectedMax.put("2021-01-01T00:00:00.000Z", 4.0); + assertThat(maxAsMap(adh), equalTo(expectedMax)); + + StringTerms.Bucket b = terms.getBucketByKey("b"); + InternalAutoDateHistogram bdh = b.getAggregations().get("dh"); + expectedDocCount.clear(); + expectedDocCount.put("2020-02-01T00:00:00.000Z", 1); + assertThat(bucketCountsAsMap(bdh), equalTo(expectedDocCount)); + expectedMax.clear(); + expectedMax.put("2020-02-01T00:00:00.000Z", 5.0); + assertThat(maxAsMap(bdh), equalTo(expectedMax)); + }); + builder = new TermsAggregationBuilder("k2").field("k2").subAggregation(builder); + asSubAggTestCase(builder, (StringTerms terms) -> { + StringTerms.Bucket a = terms.getBucketByKey("a"); + StringTerms ak1 = a.getAggregations().get("k1"); + StringTerms.Bucket ak1a = ak1.getBucketByKey("a"); + InternalAutoDateHistogram ak1adh = ak1a.getAggregations().get("dh"); + Map expectedDocCount = new TreeMap<>(); + expectedDocCount.put("2020-01-01T00:00:00.000Z", 2); + expectedDocCount.put("2021-01-01T00:00:00.000Z", 1); + assertThat(bucketCountsAsMap(ak1adh), equalTo(expectedDocCount)); + Map expectedMax = new TreeMap<>(); + expectedMax.put("2020-01-01T00:00:00.000Z", 2.0); + expectedMax.put("2021-01-01T00:00:00.000Z", 3.0); + assertThat(maxAsMap(ak1adh), equalTo(expectedMax)); + + StringTerms.Bucket b = terms.getBucketByKey("b"); + StringTerms bk1 = b.getAggregations().get("k1"); + StringTerms.Bucket bk1a = bk1.getBucketByKey("a"); + InternalAutoDateHistogram bk1adh = bk1a.getAggregations().get("dh"); + expectedDocCount.clear(); + expectedDocCount.put("2021-03-01T00:00:00.000Z", 1); + assertThat(bucketCountsAsMap(bk1adh), equalTo(expectedDocCount)); + expectedMax.clear(); + expectedMax.put("2021-03-01T00:00:00.000Z", 4.0); + assertThat(maxAsMap(bk1adh), equalTo(expectedMax)); + StringTerms.Bucket bk1b = bk1.getBucketByKey("b"); + InternalAutoDateHistogram bk1bdh = bk1b.getAggregations().get("dh"); + expectedDocCount.clear(); + expectedDocCount.put("2020-02-01T00:00:00.000Z", 1); + assertThat(bucketCountsAsMap(bk1bdh), equalTo(expectedDocCount)); + expectedMax.clear(); + expectedMax.put("2020-02-01T00:00:00.000Z", 5.0); + assertThat(maxAsMap(bk1bdh), equalTo(expectedMax)); + }); + } + + public void testAsSubAggWithIncreasedRounding() throws IOException { + CheckedBiConsumer buildIndex = (iw, dft) -> { + long start = dft.parse("2020-01-01T00:00:00Z"); + long end = dft.parse("2021-01-01T00:00:00Z"); + long useC = dft.parse("2020-07-01T00:00Z"); + long anHour = dft.resolution().convert(Instant.ofEpochSecond(TimeUnit.HOURS.toSeconds(1))); + List> docs = new ArrayList<>(); + BytesRef aBytes = new BytesRef("a"); + BytesRef bBytes = new BytesRef("b"); + BytesRef cBytes = new BytesRef("c"); + int n = 0; + for (long d = start; d < end; d += anHour) { + docs.add(List.of( + new SortedNumericDocValuesField(AGGREGABLE_DATE, d), + new SortedSetDocValuesField("k1", aBytes), + new SortedSetDocValuesField("k1", d < useC ? bBytes : cBytes), + new SortedNumericDocValuesField("n", n++) + )); + } + /* + * Intentionally add all documents at once to put them on the + * same shard to make the reduce behavior consistent. + */ + iw.addDocuments(docs); + }; + AggregationBuilder builder = new TermsAggregationBuilder("k1").field("k1").subAggregation( + new AutoDateHistogramAggregationBuilder("dh").field(AGGREGABLE_DATE).setNumBuckets(4).subAggregation( + new MaxAggregationBuilder("max").field("n"))); + asSubAggTestCase(builder, buildIndex, (StringTerms terms) -> { + StringTerms.Bucket a = terms.getBucketByKey("a"); + InternalAutoDateHistogram adh = a.getAggregations().get("dh"); + Map expectedDocCount = new TreeMap<>(); + expectedDocCount.put("2020-01-01T00:00:00.000Z", 2184); + expectedDocCount.put("2020-04-01T00:00:00.000Z", 2184); + expectedDocCount.put("2020-07-01T00:00:00.000Z", 2208); + expectedDocCount.put("2020-10-01T00:00:00.000Z", 2208); + assertThat(bucketCountsAsMap(adh), equalTo(expectedDocCount)); + Map expectedMax = new TreeMap<>(); + expectedMax.put("2020-01-01T00:00:00.000Z", 2183.0); + expectedMax.put("2020-04-01T00:00:00.000Z", 4367.0); + expectedMax.put("2020-07-01T00:00:00.000Z", 6575.0); + expectedMax.put("2020-10-01T00:00:00.000Z", 8783.0); + assertThat(maxAsMap(adh), equalTo(expectedMax)); + + StringTerms.Bucket b = terms.getBucketByKey("b"); + InternalAutoDateHistogram bdh = b.getAggregations().get("dh"); + expectedDocCount.clear(); + expectedDocCount.put("2020-01-01T00:00:00.000Z", 2184); + expectedDocCount.put("2020-04-01T00:00:00.000Z", 2184); + assertThat(bucketCountsAsMap(bdh), equalTo(expectedDocCount)); + expectedMax.clear(); + expectedMax.put("2020-01-01T00:00:00.000Z", 2183.0); + expectedMax.put("2020-04-01T00:00:00.000Z", 4367.0); + assertThat(maxAsMap(bdh), equalTo(expectedMax)); + + StringTerms.Bucket c = terms.getBucketByKey("c"); + InternalAutoDateHistogram cdh = c.getAggregations().get("dh"); + expectedDocCount.clear(); + expectedDocCount.put("2020-07-01T00:00:00.000Z", 2208); + expectedDocCount.put("2020-10-01T00:00:00.000Z", 2208); + assertThat(bucketCountsAsMap(cdh), equalTo(expectedDocCount)); + expectedMax.clear(); + expectedMax.put("2020-07-01T00:00:00.000Z", 6575.0); + expectedMax.put("2020-10-01T00:00:00.000Z", 8783.0); + assertThat(maxAsMap(cdh), equalTo(expectedMax)); + }); + } + + public void testAsSubAggInManyBuckets() throws IOException { + CheckedBiConsumer buildIndex = (iw, dft) -> { + long start = dft.parse("2020-01-01T00:00:00Z"); + long end = dft.parse("2021-01-01T00:00:00Z"); + long anHour = dft.resolution().convert(Instant.ofEpochSecond(TimeUnit.HOURS.toSeconds(1))); + List> docs = new ArrayList<>(); + int n = 0; + for (long d = start; d < end; d += anHour) { + docs.add(List.of( + new SortedNumericDocValuesField(AGGREGABLE_DATE, d), + new SortedNumericDocValuesField("n", n % 100) + )); + n++; + } + /* + * Intentionally add all documents at once to put them on the + * same shard to make the reduce behavior consistent. + */ + iw.addDocuments(docs); + }; + AggregationBuilder builder = new HistogramAggregationBuilder("n").field("n").interval(1).subAggregation( + new AutoDateHistogramAggregationBuilder("dh").field(AGGREGABLE_DATE).setNumBuckets(4).subAggregation( + new MaxAggregationBuilder("max").field("n"))); + asSubAggTestCase(builder, buildIndex, (InternalHistogram histo) -> { + assertThat(histo.getBuckets(), hasSize(100)); + for (int n = 0; n < 100; n ++) { + InternalHistogram.Bucket b = histo.getBuckets().get(n); + InternalAutoDateHistogram dh = b.getAggregations().get("dh"); + assertThat(bucketCountsAsMap(dh), hasEntry(equalTo("2020-01-01T00:00:00.000Z"), either(equalTo(21)).or(equalTo(22)))); + assertThat(bucketCountsAsMap(dh), hasEntry(equalTo("2020-04-01T00:00:00.000Z"), either(equalTo(21)).or(equalTo(22)))); + assertThat(bucketCountsAsMap(dh), hasEntry(equalTo("2020-07-01T00:00:00.000Z"), either(equalTo(22)).or(equalTo(23)))); + assertThat(bucketCountsAsMap(dh), hasEntry(equalTo("2020-10-01T00:00:00.000Z"), either(equalTo(22)).or(equalTo(23)))); + Map expectedMax = new TreeMap<>(); + expectedMax.put("2020-01-01T00:00:00.000Z", (double) n); + expectedMax.put("2020-04-01T00:00:00.000Z", (double) n); + expectedMax.put("2020-07-01T00:00:00.000Z", (double) n); + expectedMax.put("2020-10-01T00:00:00.000Z", (double) n); + assertThat(maxAsMap(dh), equalTo(expectedMax)); + } + }); + } + public void testNoDocs() throws IOException { final List dates = Collections.emptyList(); final Consumer aggregation = agg -> agg.setNumBuckets(10).field(DATE_FIELD); @@ -249,20 +436,7 @@ public void testIntervalYear() throws IOException { final long start = LocalDate.of(2015, 1, 1).atStartOfDay(ZoneOffset.UTC).toInstant().toEpochMilli(); final long end = LocalDate.of(2017, 12, 31).atStartOfDay(ZoneOffset.UTC).toInstant().toEpochMilli(); final Query rangeQuery = LongPoint.newRangeQuery(INSTANT_FIELD, start, end); - testSearchCase(rangeQuery, DATES_WITH_TIME, - aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), - histogram -> { - final List buckets = histogram.getBuckets(); - assertEquals(5, buckets.size()); - for (int i = 0; i < buckets.size(); i++) { - final Histogram.Bucket bucket = buckets.get(i); - assertEquals(DATES_WITH_TIME.get(5 + i), bucket.getKey()); - assertEquals(1, bucket.getDocCount()); - } - assertTrue(AggregationInspectionHelper.hasValue(histogram)); - } - ); - testSearchAndReduceCase(rangeQuery, DATES_WITH_TIME, + testBothCases(rangeQuery, DATES_WITH_TIME, aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), histogram -> { final ZonedDateTime startDate = ZonedDateTime.of(2015, 1, 1, 0, 0, 0, 0, ZoneOffset.UTC); @@ -287,29 +461,13 @@ public void testIntervalMonth() throws IOException { ZonedDateTime.of(2017, 3, 4, 0, 0, 0, 0, ZoneOffset.UTC), ZonedDateTime.of(2017, 3, 5, 0, 0, 0, 0, ZoneOffset.UTC), ZonedDateTime.of(2017, 3, 6, 0, 0, 0, 0, ZoneOffset.UTC)); - testSearchCase(DEFAULT_QUERY, datesForMonthInterval, - aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), histogram -> { - final List buckets = histogram.getBuckets(); - assertEquals(datesForMonthInterval.size(), buckets.size()); - for (int i = 0; i < buckets.size(); i++) { - final Histogram.Bucket bucket = buckets.get(i); - assertEquals(datesForMonthInterval.get(i), bucket.getKey()); - assertEquals(1, bucket.getDocCount()); - } - }); - testSearchAndReduceCase(DEFAULT_QUERY, datesForMonthInterval, + Map expectedDocCount = new TreeMap<>(); + expectedDocCount.put("2017-01-01T00:00:00.000Z", 1); + expectedDocCount.put("2017-02-01T00:00:00.000Z", 2); + expectedDocCount.put("2017-03-01T00:00:00.000Z", 3); + testBothCases(DEFAULT_QUERY, datesForMonthInterval, aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), - histogram -> { - final Map expectedDocCount = new HashMap<>(); - expectedDocCount.put(datesForMonthInterval.get(0).withDayOfMonth(1), 1); - expectedDocCount.put(datesForMonthInterval.get(1).withDayOfMonth(1), 2); - expectedDocCount.put(datesForMonthInterval.get(3).withDayOfMonth(1), 3); - final List buckets = histogram.getBuckets(); - assertEquals(expectedDocCount.size(), buckets.size()); - buckets.forEach(bucket -> - assertEquals(expectedDocCount.getOrDefault(bucket.getKey(), 0).longValue(), bucket.getDocCount())); - assertTrue(AggregationInspectionHelper.hasValue(histogram)); - } + result -> assertThat(bucketCountsAsMap(result), equalTo(expectedDocCount)) ); } @@ -332,28 +490,19 @@ public void testIntervalDay() throws IOException { ZonedDateTime.of(2017, 2, 3, 0, 0, 0, 0, ZoneOffset.UTC), ZonedDateTime.of(2017, 2, 3, 0, 0, 0, 0, ZoneOffset.UTC), ZonedDateTime.of(2017, 2, 5, 0, 0, 0, 0, ZoneOffset.UTC)); - final Map expectedDocCount = new HashMap<>(); - expectedDocCount.put(datesForDayInterval.get(0), 1); - expectedDocCount.put(datesForDayInterval.get(1), 2); - expectedDocCount.put(datesForDayInterval.get(3), 3); - expectedDocCount.put(datesForDayInterval.get(6), 1); - + Map expectedDocCount = new TreeMap<>(); + expectedDocCount.put("2017-02-01T00:00:00.000Z", 1); + expectedDocCount.put("2017-02-02T00:00:00.000Z", 2); + expectedDocCount.put("2017-02-03T00:00:00.000Z", 3); + expectedDocCount.put("2017-02-05T00:00:00.000Z", 1); testSearchCase(DEFAULT_QUERY, datesForDayInterval, - aggregation -> aggregation.setNumBuckets(5).field(DATE_FIELD), histogram -> { - final List buckets = histogram.getBuckets(); - assertEquals(expectedDocCount.size(), buckets.size()); - buckets.forEach(bucket -> - assertEquals(expectedDocCount.getOrDefault(bucket.getKey(), 0).longValue(), bucket.getDocCount())); - }); + aggregation -> aggregation.setNumBuckets(5).field(DATE_FIELD), + result -> assertThat(bucketCountsAsMap(result), equalTo(expectedDocCount)) + ); + expectedDocCount.put("2017-02-04T00:00:00.000Z", 0); testSearchAndReduceCase(DEFAULT_QUERY, datesForDayInterval, aggregation -> aggregation.setNumBuckets(5).field(DATE_FIELD), - histogram -> { - final List buckets = histogram.getBuckets(); - assertEquals(5, buckets.size()); - buckets.forEach(bucket -> - assertEquals(expectedDocCount.getOrDefault(bucket.getKey(), 0).longValue(), bucket.getDocCount())); - assertTrue(AggregationInspectionHelper.hasValue(histogram)); - } + result -> assertThat(bucketCountsAsMap(result), equalTo(expectedDocCount)) ); } @@ -366,32 +515,20 @@ public void testIntervalDayWithTZ() throws IOException { ZonedDateTime.of(2017, 2, 3, 0, 0, 0, 0, ZoneOffset.UTC), ZonedDateTime.of(2017, 2, 3, 0, 0, 0, 0, ZoneOffset.UTC), ZonedDateTime.of(2017, 2, 5, 0, 0, 0, 0, ZoneOffset.UTC)); + Map expectedDocCount = new TreeMap<>(); + expectedDocCount.put("2017-01-31T00:00:00.000-01:00", 1); + expectedDocCount.put("2017-02-01T00:00:00.000-01:00", 2); + expectedDocCount.put("2017-02-02T00:00:00.000-01:00", 3); + expectedDocCount.put("2017-02-04T00:00:00.000-01:00", 1); testSearchCase(DEFAULT_QUERY, datesForDayInterval, - aggregation -> aggregation.setNumBuckets(5).field(DATE_FIELD).timeZone(ZoneOffset.ofHours(-1)), histogram -> { - final Map expectedDocCount = new HashMap<>(); - expectedDocCount.put("2017-01-31T23:00:00.000-01:00", 1); - expectedDocCount.put("2017-02-01T23:00:00.000-01:00", 2); - expectedDocCount.put("2017-02-02T23:00:00.000-01:00", 3); - expectedDocCount.put("2017-02-04T23:00:00.000-01:00", 1); - final List buckets = histogram.getBuckets(); - assertEquals(expectedDocCount.size(), buckets.size()); - buckets.forEach(bucket -> - assertEquals(expectedDocCount.getOrDefault(bucket.getKeyAsString(), 0).longValue(), bucket.getDocCount())); - assertTrue(AggregationInspectionHelper.hasValue(histogram)); - }); + aggregation -> aggregation.setNumBuckets(5).field(DATE_FIELD).timeZone(ZoneOffset.ofHours(-1)), + result -> assertThat(bucketCountsAsMap(result), equalTo(expectedDocCount)) + ); + expectedDocCount.put("2017-02-03T00:00:00.000-01:00", 0); testSearchAndReduceCase(DEFAULT_QUERY, datesForDayInterval, - aggregation -> aggregation.setNumBuckets(5).field(DATE_FIELD).timeZone(ZoneOffset.ofHours(-1)), histogram -> { - final Map expectedDocCount = new HashMap<>(); - expectedDocCount.put("2017-01-31T00:00:00.000-01:00", 1); - expectedDocCount.put("2017-02-01T00:00:00.000-01:00", 2); - expectedDocCount.put("2017-02-02T00:00:00.000-01:00", 3); - expectedDocCount.put("2017-02-04T00:00:00.000-01:00", 1); - final List buckets = histogram.getBuckets(); - assertEquals(5, buckets.size()); - buckets.forEach(bucket -> - assertEquals(expectedDocCount.getOrDefault(bucket.getKeyAsString(), 0).longValue(), bucket.getDocCount())); - assertTrue(AggregationInspectionHelper.hasValue(histogram)); - }); + aggregation -> aggregation.setNumBuckets(5).field(DATE_FIELD).timeZone(ZoneOffset.ofHours(-1)), + result -> assertThat(bucketCountsAsMap(result), equalTo(expectedDocCount)) + ); } public void testIntervalHour() throws IOException { @@ -406,51 +543,36 @@ public void testIntervalHour() throws IOException { ZonedDateTime.of(2017, 2, 1, 16, 6, 0, 0, ZoneOffset.UTC), ZonedDateTime.of(2017, 2, 1, 16, 48, 0, 0, ZoneOffset.UTC), ZonedDateTime.of(2017, 2, 1, 16, 59, 0, 0, ZoneOffset.UTC)); + Map expectedDocCount = new TreeMap<>(); + expectedDocCount.put("2017-02-01T09:00:00.000Z", 2); + expectedDocCount.put("2017-02-01T10:00:00.000Z", 1); + expectedDocCount.put("2017-02-01T13:00:00.000Z", 1); + expectedDocCount.put("2017-02-01T14:00:00.000Z", 2); + expectedDocCount.put("2017-02-01T15:00:00.000Z", 1); + expectedDocCount.put("2017-02-01T15:00:00.000Z", 1); + expectedDocCount.put("2017-02-01T16:00:00.000Z", 3); testSearchCase(DEFAULT_QUERY, datesForHourInterval, aggregation -> aggregation.setNumBuckets(8).field(DATE_FIELD), - histogram -> { - final List buckets = histogram.getBuckets(); - assertEquals(datesForHourInterval.size(), buckets.size()); - for (int i = 0; i < buckets.size(); i++) { - final Histogram.Bucket bucket = buckets.get(i); - assertEquals(datesForHourInterval.get(i), bucket.getKey()); - assertEquals(1, bucket.getDocCount()); - } - } + result -> assertThat(bucketCountsAsMap(result), equalTo(expectedDocCount)) ); + expectedDocCount.put("2017-02-01T11:00:00.000Z", 0); + expectedDocCount.put("2017-02-01T12:00:00.000Z", 0); testSearchAndReduceCase(DEFAULT_QUERY, datesForHourInterval, aggregation -> aggregation.setNumBuckets(10).field(DATE_FIELD), - histogram -> { - final Map expectedDocCount = new HashMap<>(); - expectedDocCount.put(datesForHourInterval.get(0).withMinute(0), 2); - expectedDocCount.put(datesForHourInterval.get(2).withMinute(0), 1); - expectedDocCount.put(datesForHourInterval.get(3).withMinute(0), 1); - expectedDocCount.put(datesForHourInterval.get(4).withMinute(0), 2); - expectedDocCount.put(datesForHourInterval.get(6).withMinute(0), 1); - expectedDocCount.put(datesForHourInterval.get(7).withMinute(0), 3); - final List buckets = histogram.getBuckets(); - assertEquals(8, buckets.size()); - buckets.forEach(bucket -> - assertEquals(expectedDocCount.getOrDefault(bucket.getKey(), 0).longValue(), bucket.getDocCount())); - } + result -> assertThat(bucketCountsAsMap(result), equalTo(expectedDocCount)) ); + expectedDocCount.clear(); + expectedDocCount.put("2017-02-01T09:00:00.000Z", 3); + expectedDocCount.put("2017-02-01T12:00:00.000Z", 3); + expectedDocCount.put("2017-02-01T15:00:00.000Z", 4); testSearchAndReduceCase(DEFAULT_QUERY, datesForHourInterval, aggregation -> aggregation.setNumBuckets(6).field(DATE_FIELD), - histogram -> { - final Map expectedDocCount = new HashMap<>(); - expectedDocCount.put(datesForHourInterval.get(0).withMinute(0), 3); - expectedDocCount.put(datesForHourInterval.get(0).plusHours(3).withMinute(0), 3); - expectedDocCount.put(datesForHourInterval.get(0).plusHours(6).withMinute(0), 4); - final List buckets = histogram.getBuckets(); - assertEquals(expectedDocCount.size(), buckets.size()); - buckets.forEach(bucket -> - assertEquals(expectedDocCount.getOrDefault(bucket.getKey(), 0).longValue(), bucket.getDocCount())); - } + result -> assertThat(bucketCountsAsMap(result), equalTo(expectedDocCount)) ); } public void testIntervalHourWithTZ() throws IOException { - final List datesForHourInterval = Arrays.asList( + List datesForHourInterval = Arrays.asList( ZonedDateTime.of(2017, 2, 1, 9, 2, 0, 0, ZoneOffset.UTC), ZonedDateTime.of(2017, 2, 1, 9, 35, 0, 0, ZoneOffset.UTC), ZonedDateTime.of(2017, 2, 1, 10, 15, 0, 0, ZoneOffset.UTC), @@ -461,36 +583,22 @@ public void testIntervalHourWithTZ() throws IOException { ZonedDateTime.of(2017, 2, 1, 16, 6, 0, 0, ZoneOffset.UTC), ZonedDateTime.of(2017, 2, 1, 16, 48, 0, 0, ZoneOffset.UTC), ZonedDateTime.of(2017, 2, 1, 16, 59, 0, 0, ZoneOffset.UTC)); + Map expectedDocCount = new TreeMap<>(); + expectedDocCount.put("2017-02-01T08:00:00.000-01:00", 2); + expectedDocCount.put("2017-02-01T09:00:00.000-01:00", 1); + expectedDocCount.put("2017-02-01T12:00:00.000-01:00", 1); + expectedDocCount.put("2017-02-01T13:00:00.000-01:00", 2); + expectedDocCount.put("2017-02-01T14:00:00.000-01:00", 1); + expectedDocCount.put("2017-02-01T15:00:00.000-01:00", 3); testSearchCase(DEFAULT_QUERY, datesForHourInterval, aggregation -> aggregation.setNumBuckets(8).field(DATE_FIELD).timeZone(ZoneOffset.ofHours(-1)), - histogram -> { - final List dateStrings = datesForHourInterval.stream() - .map(dateTime -> DateFormatter.forPattern("strict_date_time") - .format(dateTime.withZoneSameInstant(ZoneOffset.ofHours(-1)))).collect(Collectors.toList()); - final List buckets = histogram.getBuckets(); - assertEquals(datesForHourInterval.size(), buckets.size()); - for (int i = 0; i < buckets.size(); i++) { - final Histogram.Bucket bucket = buckets.get(i); - assertEquals(dateStrings.get(i), bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - } - } + result -> assertThat(bucketCountsAsMap(result), equalTo(expectedDocCount)) ); + expectedDocCount.put("2017-02-01T10:00:00.000-01:00", 0); + expectedDocCount.put("2017-02-01T11:00:00.000-01:00", 0); testSearchAndReduceCase(DEFAULT_QUERY, datesForHourInterval, aggregation -> aggregation.setNumBuckets(10).field(DATE_FIELD).timeZone(ZoneOffset.ofHours(-1)), - histogram -> { - final Map expectedDocCount = new HashMap<>(); - expectedDocCount.put("2017-02-01T08:00:00.000-01:00", 2); - expectedDocCount.put("2017-02-01T09:00:00.000-01:00", 1); - expectedDocCount.put("2017-02-01T12:00:00.000-01:00", 1); - expectedDocCount.put("2017-02-01T13:00:00.000-01:00", 2); - expectedDocCount.put("2017-02-01T14:00:00.000-01:00", 1); - expectedDocCount.put("2017-02-01T15:00:00.000-01:00", 3); - final List buckets = histogram.getBuckets(); - assertEquals(8, buckets.size()); - buckets.forEach(bucket -> - assertEquals(expectedDocCount.getOrDefault(bucket.getKeyAsString(), 0).longValue(), bucket.getDocCount())); - } + result -> assertThat(bucketCountsAsMap(result), equalTo(expectedDocCount)) ); } @@ -690,31 +798,35 @@ public void testIntervalMinute() throws IOException { ZonedDateTime.of(2017, 2, 1, 9, 15, 37, 0, ZoneOffset.UTC), ZonedDateTime.of(2017, 2, 1, 9, 16, 4, 0, ZoneOffset.UTC), ZonedDateTime.of(2017, 2, 1, 9, 16, 42, 0, ZoneOffset.UTC)); - + Map skeletonDocCount = new TreeMap<>(); + skeletonDocCount.put("2017-02-01T09:02:00.000Z", 2); + skeletonDocCount.put("2017-02-01T09:15:00.000Z", 1); + skeletonDocCount.put("2017-02-01T09:16:00.000Z", 2); testSearchCase(DEFAULT_QUERY, datesForMinuteInterval, aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), - histogram -> { - final List buckets = histogram.getBuckets(); - assertEquals(datesForMinuteInterval.size(), buckets.size()); - for (int i = 0; i < buckets.size(); i++) { - final Histogram.Bucket bucket = buckets.get(i); - assertEquals(datesForMinuteInterval.get(i), bucket.getKey()); - assertEquals(1, bucket.getDocCount()); - } - } + result -> assertThat(bucketCountsAsMap(result), equalTo(skeletonDocCount)) ); + Map fullDocCount = new TreeMap<>(); + fullDocCount.put("2017-02-01T09:02:00.000Z", 2); + fullDocCount.put("2017-02-01T09:07:00.000Z", 0); + fullDocCount.put("2017-02-01T09:12:00.000Z", 3); testSearchAndReduceCase(DEFAULT_QUERY, datesForMinuteInterval, + aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), + result -> assertThat(bucketCountsAsMap(result), equalTo(fullDocCount)) + ); + + testSearchCase(DEFAULT_QUERY, datesForMinuteInterval, aggregation -> aggregation.setNumBuckets(15).field(DATE_FIELD), - histogram -> { - final Map expectedDocCount = new HashMap<>(); - expectedDocCount.put(datesForMinuteInterval.get(0).withSecond(0), 2); - expectedDocCount.put(datesForMinuteInterval.get(2).withSecond(0), 1); - expectedDocCount.put(datesForMinuteInterval.get(3).withSecond(0), 2); - final List buckets = histogram.getBuckets(); - assertEquals(15, buckets.size()); - buckets.forEach(bucket -> - assertEquals(expectedDocCount.getOrDefault(bucket.getKey(), 0).longValue(), bucket.getDocCount())); - } + result -> assertThat(bucketCountsAsMap(result), equalTo(skeletonDocCount)) + ); + fullDocCount.clear(); + fullDocCount.putAll(skeletonDocCount); + for (int minute = 3; minute < 15; minute++) { + fullDocCount.put(String.format(Locale.ROOT, "2017-02-01T09:%02d:00.000Z", minute), 0); + } + testSearchAndReduceCase(DEFAULT_QUERY, datesForMinuteInterval, + aggregation -> aggregation.setNumBuckets(15).field(DATE_FIELD), + result -> assertThat(bucketCountsAsMap(result), equalTo(fullDocCount)) ); } @@ -726,27 +838,21 @@ public void testIntervalSecond() throws IOException { ZonedDateTime.of(2017, 2, 1, 0, 0, 11, 688, ZoneOffset.UTC), ZonedDateTime.of(2017, 2, 1, 0, 0, 11, 210, ZoneOffset.UTC), ZonedDateTime.of(2017, 2, 1, 0, 0, 11, 380, ZoneOffset.UTC)); - final ZonedDateTime startDate = datesForSecondInterval.get(0).withNano(0); - final Map expectedDocCount = new HashMap<>(); - expectedDocCount.put(startDate, 1); - expectedDocCount.put(startDate.plusSeconds(2), 2); - expectedDocCount.put(startDate.plusSeconds(6), 3); - + Map expectedDocCount = new TreeMap<>(); + expectedDocCount.put("2017-02-01T00:00:05.000Z", 1); + expectedDocCount.put("2017-02-01T00:00:07.000Z", 2); + expectedDocCount.put("2017-02-01T00:00:11.000Z", 3); testSearchCase(DEFAULT_QUERY, datesForSecondInterval, - aggregation -> aggregation.setNumBuckets(7).field(DATE_FIELD), histogram -> { - final List buckets = histogram.getBuckets(); - assertEquals(expectedDocCount.size(), buckets.size()); - buckets.forEach(bucket -> - assertEquals(expectedDocCount.getOrDefault(bucket.getKey(), 0).longValue(), bucket.getDocCount())); - }); + aggregation -> aggregation.setNumBuckets(7).field(DATE_FIELD), + result -> assertThat(bucketCountsAsMap(result), equalTo(expectedDocCount)) + ); + expectedDocCount.put("2017-02-01T00:00:06.000Z", 0); + expectedDocCount.put("2017-02-01T00:00:08.000Z", 0); + expectedDocCount.put("2017-02-01T00:00:09.000Z", 0); + expectedDocCount.put("2017-02-01T00:00:10.000Z", 0); testSearchAndReduceCase(DEFAULT_QUERY, datesForSecondInterval, aggregation -> aggregation.setNumBuckets(7).field(DATE_FIELD), - histogram -> { - final List buckets = histogram.getBuckets(); - assertEquals(7, buckets.size()); - buckets.forEach(bucket -> - assertEquals(expectedDocCount.getOrDefault(bucket.getKey(), 0).longValue(), bucket.getDocCount())); - } + result -> assertThat(bucketCountsAsMap(result), equalTo(expectedDocCount)) ); } @@ -869,4 +975,23 @@ private void indexSampleData(List dataset, RandomIndexWriter inde i += 1; } } + + private Map bucketCountsAsMap(InternalAutoDateHistogram result) { + LinkedHashMap map = new LinkedHashMap<>(result.getBuckets().size()); + result.getBuckets().stream().forEach(b -> { + Object old = map.put(b.getKeyAsString(), Math.toIntExact(b.getDocCount())); + assertNull(old); + }); + return map; + } + + private Map maxAsMap(InternalAutoDateHistogram result) { + LinkedHashMap map = new LinkedHashMap<>(result.getBuckets().size()); + result.getBuckets().stream().forEach(b -> { + InternalMax max = b.getAggregations().get("max"); + Object old = map.put(b.getKeyAsString(), max.getValue()); + assertNull(old); + }); + return map; + } } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorTestCase.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorTestCase.java new file mode 100644 index 0000000000000..bfc3c03f7f970 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorTestCase.java @@ -0,0 +1,114 @@ +/* + * 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.histogram; + +import org.apache.lucene.document.SortedNumericDocValuesField; +import org.apache.lucene.document.SortedSetDocValuesField; +import org.apache.lucene.index.IndexOptions; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.CheckedBiConsumer; +import org.elasticsearch.index.mapper.DateFieldMapper; +import org.elasticsearch.index.mapper.KeywordFieldMapper; +import org.elasticsearch.index.mapper.NumberFieldMapper; +import org.elasticsearch.index.mapper.NumberFieldMapper.NumberType; +import org.elasticsearch.search.aggregations.AggregationBuilder; +import org.elasticsearch.search.aggregations.AggregatorTestCase; +import org.elasticsearch.search.aggregations.InternalAggregation; + +import java.io.IOException; +import java.util.List; +import java.util.function.Consumer; + +public abstract class DateHistogramAggregatorTestCase extends AggregatorTestCase { + /** + * A date that is always "aggregable" because it has doc values but may or + * may not have a search index. If it doesn't then we can't use our fancy + * date rounding mechanism that needs to know the minimum and maximum dates + * it is going to round because it ready *that* out of the search index. + */ + protected static final String AGGREGABLE_DATE = "aggregable_date"; + + protected final void asSubAggTestCase(AggregationBuilder builder, Consumer verify) + throws IOException { + CheckedBiConsumer buildIndex = (iw, dft) -> { + iw.addDocument(List.of( + new SortedNumericDocValuesField(AGGREGABLE_DATE, dft.parse("2020-02-01T00:00:00Z")), + new SortedSetDocValuesField("k1", new BytesRef("a")), + new SortedSetDocValuesField("k2", new BytesRef("a")), + new SortedNumericDocValuesField("n", 1) + )); + iw.addDocument(List.of( + new SortedNumericDocValuesField(AGGREGABLE_DATE, dft.parse("2020-03-01T00:00:00Z")), + new SortedSetDocValuesField("k1", new BytesRef("a")), + new SortedSetDocValuesField("k2", new BytesRef("a")), + new SortedNumericDocValuesField("n", 2) + )); + iw.addDocument(List.of( + new SortedNumericDocValuesField(AGGREGABLE_DATE, dft.parse("2021-02-01T00:00:00Z")), + new SortedSetDocValuesField("k1", new BytesRef("a")), + new SortedSetDocValuesField("k2", new BytesRef("a")), + new SortedNumericDocValuesField("n", 3) + )); + iw.addDocument(List.of( + new SortedNumericDocValuesField(AGGREGABLE_DATE, dft.parse("2021-03-01T00:00:00Z")), + new SortedSetDocValuesField("k1", new BytesRef("a")), + new SortedSetDocValuesField("k2", new BytesRef("b")), + new SortedNumericDocValuesField("n", 4) + )); + iw.addDocument(List.of( + new SortedNumericDocValuesField(AGGREGABLE_DATE, dft.parse("2020-02-01T00:00:00Z")), + new SortedSetDocValuesField("k1", new BytesRef("b")), + new SortedSetDocValuesField("k2", new BytesRef("b")), + new SortedNumericDocValuesField("n", 5) + )); + }; + asSubAggTestCase(builder, buildIndex, verify); + } + + protected final void asSubAggTestCase( + AggregationBuilder builder, + CheckedBiConsumer buildIndex, + Consumer verify + ) throws IOException { + KeywordFieldMapper.KeywordFieldType k1ft = new KeywordFieldMapper.KeywordFieldType(); + k1ft.setName("k1"); + k1ft.setHasDocValues(true); + KeywordFieldMapper.KeywordFieldType k2ft = new KeywordFieldMapper.KeywordFieldType(); + k2ft.setName("k2"); + k2ft.setHasDocValues(true); + NumberFieldMapper.NumberFieldType nft = new NumberFieldMapper.NumberFieldType(NumberType.LONG); + nft.setName("n"); + DateFieldMapper.DateFieldType dft = aggregableDateFieldType(false, randomBoolean()); + testCase(builder, new MatchAllDocsQuery(), iw -> buildIndex.accept(iw, dft), verify, k1ft, k2ft, nft, dft); + } + + protected final DateFieldMapper.DateFieldType aggregableDateFieldType(boolean useNanosecondResolution, boolean isSearchable) { + DateFieldMapper.Builder builder = new DateFieldMapper.Builder(AGGREGABLE_DATE); + if (useNanosecondResolution) { + builder.withResolution(DateFieldMapper.Resolution.NANOSECONDS); + } + DateFieldMapper.DateFieldType fieldType = builder.fieldType(); + fieldType.setIndexOptions(isSearchable ? IndexOptions.DOCS : IndexOptions.NONE); + fieldType.setName(AGGREGABLE_DATE); + return fieldType; + } +} diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorTests.java index a605d92cc90b3..55c1af57da298 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorTests.java @@ -22,9 +22,7 @@ import org.apache.lucene.document.Document; import org.apache.lucene.document.LongPoint; import org.apache.lucene.document.SortedNumericDocValuesField; -import org.apache.lucene.document.SortedSetDocValuesField; import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.search.IndexSearcher; @@ -32,14 +30,10 @@ import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.store.Directory; -import org.apache.lucene.util.BytesRef; -import org.elasticsearch.common.CheckedConsumer; import org.elasticsearch.common.time.DateFormatters; import org.elasticsearch.index.mapper.DateFieldMapper; -import org.elasticsearch.index.mapper.KeywordFieldMapper; import org.elasticsearch.search.aggregations.AggregationBuilder; import org.elasticsearch.search.aggregations.AggregationBuilders; -import org.elasticsearch.search.aggregations.AggregatorTestCase; import org.elasticsearch.search.aggregations.BucketOrder; import org.elasticsearch.search.aggregations.MultiBucketConsumerService.TooManyBucketsException; import org.elasticsearch.search.aggregations.bucket.terms.StringTerms; @@ -56,15 +50,7 @@ import static java.util.stream.Collectors.toList; import static org.hamcrest.Matchers.equalTo; -public class DateHistogramAggregatorTests extends AggregatorTestCase { - - /** - * A date that is always "aggregable" because it has doc values but may or - * may not have a search index. If it doesn't then we can't use our fancy - * date rounding mechanism that needs to know the minimum and maximum dates - * it is going to round because it ready *that* out of the search index. - */ - private static final String AGGREGABLE_DATE = "aggregable_date"; +public class DateHistogramAggregatorTests extends DateHistogramAggregatorTestCase { /** * A date that is always "searchable" because it is indexed. */ @@ -172,43 +158,9 @@ public void testMatchAllDocs() throws IOException { } public void testAsSubAgg() throws IOException { - KeywordFieldMapper.KeywordFieldType k1ft = new KeywordFieldMapper.KeywordFieldType(); - k1ft.setName("k1"); - k1ft.setHasDocValues(true); - KeywordFieldMapper.KeywordFieldType k2ft = new KeywordFieldMapper.KeywordFieldType(); - k2ft.setName("k2"); - k2ft.setHasDocValues(true); - DateFieldMapper.DateFieldType dft = aggregableDateFieldType(false, randomBoolean()); - CheckedConsumer buildIndex = iw -> { - iw.addDocument(List.of( - new SortedNumericDocValuesField(AGGREGABLE_DATE, dft.parse("2020-02-01T00:00:00Z")), - new SortedSetDocValuesField("k1", new BytesRef("a")), - new SortedSetDocValuesField("k2", new BytesRef("a")) - )); - iw.addDocument(List.of( - new SortedNumericDocValuesField(AGGREGABLE_DATE, dft.parse("2020-03-01T00:00:00Z")), - new SortedSetDocValuesField("k1", new BytesRef("a")), - new SortedSetDocValuesField("k2", new BytesRef("a")) - )); - iw.addDocument(List.of( - new SortedNumericDocValuesField(AGGREGABLE_DATE, dft.parse("2021-02-01T00:00:00Z")), - new SortedSetDocValuesField("k1", new BytesRef("a")), - new SortedSetDocValuesField("k2", new BytesRef("a")) - )); - iw.addDocument(List.of( - new SortedNumericDocValuesField(AGGREGABLE_DATE, dft.parse("2021-03-01T00:00:00Z")), - new SortedSetDocValuesField("k1", new BytesRef("a")), - new SortedSetDocValuesField("k2", new BytesRef("b")) - )); - iw.addDocument(List.of( - new SortedNumericDocValuesField(AGGREGABLE_DATE, dft.parse("2020-02-01T00:00:00Z")), - new SortedSetDocValuesField("k1", new BytesRef("b")), - new SortedSetDocValuesField("k2", new BytesRef("b")) - )); - }; AggregationBuilder builder = new TermsAggregationBuilder("k1").field("k1").subAggregation( - new DateHistogramAggregationBuilder("dh").field(AGGREGABLE_DATE).calendarInterval(DateHistogramInterval.YEAR)); - testCase(builder, new MatchAllDocsQuery(), buildIndex, (StringTerms terms) -> { + new DateHistogramAggregationBuilder("dh").field(AGGREGABLE_DATE).calendarInterval(DateHistogramInterval.YEAR)); + asSubAggTestCase(builder, (StringTerms terms) -> { StringTerms.Bucket a = terms.getBucketByKey("a"); InternalDateHistogram adh = a.getAggregations().get("dh"); assertThat(adh.getBuckets().stream().map(bucket -> bucket.getKey().toString()).collect(toList()), equalTo(List.of( @@ -220,9 +172,9 @@ public void testAsSubAgg() throws IOException { assertThat(bdh.getBuckets().stream().map(bucket -> bucket.getKey().toString()).collect(toList()), equalTo(List.of( "2020-01-01T00:00Z" ))); - }, k1ft, dft); + }); builder = new TermsAggregationBuilder("k2").field("k2").subAggregation(builder); - testCase(builder, new MatchAllDocsQuery(), buildIndex, (StringTerms terms) -> { + asSubAggTestCase(builder, (StringTerms terms) -> { StringTerms.Bucket a = terms.getBucketByKey("a"); StringTerms ak1 = a.getAggregations().get("k1"); StringTerms.Bucket ak1a = ak1.getBucketByKey("a"); @@ -243,7 +195,7 @@ public void testAsSubAgg() throws IOException { assertThat(bk1bdh.getBuckets().stream().map(bucket -> bucket.getKey().toString()).collect(toList()), equalTo(List.of( "2020-01-01T00:00Z" ))); - }, k1ft, k2ft, dft); + }); } public void testNoDocsDeprecatedInterval() throws IOException { @@ -1346,17 +1298,6 @@ private void executeTestCase(boolean reduced, } } - private DateFieldMapper.DateFieldType aggregableDateFieldType(boolean useNanosecondResolution, boolean isSearchable) { - DateFieldMapper.Builder builder = new DateFieldMapper.Builder(AGGREGABLE_DATE); - if (useNanosecondResolution) { - builder.withResolution(DateFieldMapper.Resolution.NANOSECONDS); - } - DateFieldMapper.DateFieldType fieldType = builder.fieldType(); - fieldType.setIndexOptions(isSearchable ? IndexOptions.DOCS : IndexOptions.NONE); - fieldType.setName(AGGREGABLE_DATE); - return fieldType; - } - private static long asLong(String dateTime) { return DateFormatters.from(DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.parse(dateTime)).toInstant().toEpochMilli(); } 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 99cab4dda80c9..fe0a2895053c0 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 @@ -30,6 +30,7 @@ import java.util.Set; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; public class LongKeyedBucketOrdsTests extends ESTestCase { private final MockBigArrays bigArrays = new MockBigArrays(new MockPageCacheRecycler(Settings.EMPTY), new NoneCircuitBreakerService()); @@ -92,6 +93,8 @@ private void collectsFromSingleBucketCase(LongKeyedBucketOrds ords) { assertThat(ordEnum.value(), equalTo(values[i])); } assertFalse(ordEnum.next()); + + assertThat(ords.maxOwningBucketOrd(), equalTo(0L)); } finally { ords.close(); } @@ -156,6 +159,8 @@ public void testCollectsFromManyBuckets() { } assertFalse(ords.ordsEnum(randomLongBetween(maxOwningBucketOrd + 1, Long.MAX_VALUE)).next()); assertThat(ords.bucketsInOrd(randomLongBetween(maxOwningBucketOrd + 1, Long.MAX_VALUE)), equalTo(0L)); + + assertThat(ords.maxOwningBucketOrd(), greaterThanOrEqualTo(maxOwningBucketOrd)); } } From 8b67b27560f4aa84fa947b49bb012ffe964d5661 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 29 May 2020 16:53:56 -0400 Subject: [PATCH 02/18] What happens if we, like, never correct the rounding? --- .../bucket/histogram/AutoDateHistogramAggregator.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) 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 2a982bfc8f801..a68dd2917b94c 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 @@ -323,7 +323,7 @@ private void rebucket() { @Override public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { - correctRounding(owningBucketOrds); +// correctRounding(owningBucketOrds); /* * Now that we have the perfect rounding rebucket everything to merge * all of the buckets together that we were too lazy to merge while @@ -355,7 +355,7 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I } /** - * Pick the correct rounding for the specifies {@code owningBucketOrds}. + * Pick the correct rounding for the specified {@code owningBucketOrds}. */ private void correctRounding(long[] owningBucketOrds) { for (long owningBucketOrd : owningBucketOrds) { @@ -377,13 +377,14 @@ private void correctRounding(long[] owningBucketOrds) { long newKey = preparedRounding.round(oldKey); min = Math.min(min, newKey); max = Math.max(max, newKey); - bucketOrds.add(owningBucketOrd, newKey); + perfect.add(newKey); } count = perfect.size(); } } while (newRounding < roundingInfos.length - 1 && ( count > targetBuckets * roundingInfos[newRounding].getMaximumInnerInterval() || max - min > targetBuckets * roundingInfos[newRounding].getMaximumRoughEstimateDurationMillis())); + assert newRounding == oldRounding; setRounding(owningBucketOrd, newRounding); wastedBucketsOverestimate += bucketOrds.bucketsInOrd(owningBucketOrd) - count; } From 962bb33610a81e2c8a36da1d5951e68033d10850 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 1 Jun 2020 13:44:27 -0400 Subject: [PATCH 03/18] Fixup --- .../AbstractHistogramAggregator.java | 2 +- .../AutoDateHistogramAggregator.java | 97 +++++++------------ 2 files changed, 35 insertions(+), 64 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AbstractHistogramAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AbstractHistogramAggregator.java index f026bcdc01f41..4f639368ea745 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AbstractHistogramAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AbstractHistogramAggregator.java @@ -90,7 +90,7 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I double roundKey = Double.longBitsToDouble(bucketValue); double key = roundKey * interval + offset; return new InternalHistogram.Bucket(key, docCount, keyed, formatter, subAggregationResults); - }, buckets -> { + }, (owningBucketOrd, 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()); 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 a68dd2917b94c..5e7d355174b0e 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 @@ -28,7 +28,6 @@ import org.elasticsearch.common.util.ByteArray; import org.elasticsearch.common.util.IntArray; import org.elasticsearch.common.util.LongArray; -import org.elasticsearch.common.util.LongHash; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; @@ -57,19 +56,19 @@ *

* Initially it uses the most fine grained rounding configuration possible but * as more data arrives it uses two heuristics to shift to coarser and coarser - * rounding. The first heuristic is the number of buckets, specifically, it + * rounding. The first heuristic is the number of buckets, specifically, * when there are more buckets than can "fit" in the current rounding it shifts * to the next rounding. Instead of redoing the rounding, it estimates the * number of buckets that will "survive" at the new rounding and uses * that as the initial value for the bucket count that it * increments in order to trigger another promotion to another coarser * rounding. This works fairly well at containing the number of buckets, but - * it the estimate of the number of buckets will be wrong if the buckets are + * the estimate of the number of buckets will be wrong if the buckets are * quite a spread out compared to the rounding. *

* The second heuristic it uses to trigger promotion to a coarser rounding is * the distance between the min and max bucket. When that distance is greater - * than what the current rounding supports it promotes. This is heuristic + * than what the current rounding supports it promotes. This heuristic * isn't good at limiting the number of buckets but is great when the buckets * are spread out compared to the rounding. So it should complement the first * heuristic. @@ -122,10 +121,13 @@ class AutoDateHistogramAggregator extends DeferableBucketAggregator { */ private ByteArray roundingIndices; /** - * The min and max of each bucket's keys. min lives in indices of the form - * {@code 2n} and max in {@code 2n + 1}. + * The minimum key per {@code owningBucketOrd}. */ - private LongArray bounds; + private LongArray mins; + /** + * The max key per {@code owningBucketOrd}. + */ + private LongArray maxes; /** * A reference to the collector so we can * {@link MergingBucketsDeferringCollector#mergeBuckets(long[])}. @@ -138,7 +140,7 @@ class AutoDateHistogramAggregator extends DeferableBucketAggregator { private IntArray liveBucketCountUnderestimate; /** * An over estimate of the number of wasted buckets. When this gets - * too high we {@link #rebucket()} which sets it to 0. + * too high we {@link #rebucket} which sets it to 0. */ private long wastedBucketsOverestimate = 0; /** @@ -176,9 +178,10 @@ class AutoDateHistogramAggregator extends DeferableBucketAggregator { this.collectsFromSingleBucket = collectsFromSingleBucket; assert roundingInfos.length < 127 : "Rounding must fit in a signed byte"; roundingIndices = context.bigArrays().newByteArray(1, true); - bounds = context.bigArrays().newLongArray(2, true); - bounds.set(0, Long.MAX_VALUE); - bounds.set(1, Long.MIN_VALUE); + mins = context.bigArrays().newLongArray(1, false); + mins.set(0, Long.MAX_VALUE); + maxes = context.bigArrays().newLongArray(1, false); + maxes.set(1, Long.MIN_VALUE); preparedRoundings = new Rounding.Prepared[roundingInfos.length]; // Prepare the first rounding because we know we'll need it. preparedRoundings[0] = roundingPreparer.apply(roundingInfos[0].rounding); @@ -243,7 +246,7 @@ private byte collectValue(LeafBucketCollector sub, long owningBucketOrd, byte ro collectBucket(sub, doc, bucketOrd); liveBucketCountUnderestimate = context.bigArrays().grow(liveBucketCountUnderestimate, owningBucketOrd + 1); int estimatedBucketCount = liveBucketCountUnderestimate.increment(owningBucketOrd, 1); - return increaseRounding(owningBucketOrd, estimatedBucketCount, rounded, roundingIdx); + return increaseRoundingIfNeeded(owningBucketOrd, estimatedBucketCount, rounded, roundingIdx); } /** @@ -251,21 +254,25 @@ private byte collectValue(LeafBucketCollector sub, long owningBucketOrd, byte ro * estimated, bucket counts, {@link #rebucket() rebucketing} the all * buckets if the estimated number of wasted buckets is too high. */ - private byte increaseRounding(long owningBucketOrd, int oldEstimatedBucketCount, long newKey, byte oldRounding) { + private byte increaseRoundingIfNeeded(long owningBucketOrd, int oldEstimatedBucketCount, long newKey, byte oldRounding) { if (oldRounding >= roundingInfos.length - 1) { return oldRounding; } - if (bounds.size() < owningBucketOrd * 2 + 2) { - long oldSize = bounds.size(); - bounds = context.bigArrays().grow(bounds, owningBucketOrd * 2 + 2); - for (long b = oldSize; b < bounds.size(); b++) { - bounds.set(b, (b & 1L) == 0L ? Long.MAX_VALUE : Long.MIN_VALUE); - } + if (mins.size() < owningBucketOrd + 1) { + long oldSize = mins.size(); + mins = context.bigArrays().grow(mins, owningBucketOrd + 1); + mins.fill(oldSize, mins.size(), Long.MAX_VALUE); + } + if (maxes.size() < owningBucketOrd + 1) { + long oldSize = maxes.size(); + maxes = context.bigArrays().grow(maxes, owningBucketOrd + 1); + maxes.fill(oldSize, maxes.size(), Long.MIN_VALUE); } - long min = Math.min(bounds.get(owningBucketOrd * 2), newKey); - bounds.set(owningBucketOrd * 2, min); - long max = Math.max(bounds.get(owningBucketOrd * 2 + 1), newKey); - bounds.set(owningBucketOrd * 2 + 1, max); + + long min = Math.min(mins.get(owningBucketOrd), newKey); + mins.set(owningBucketOrd, min); + long max = Math.max(maxes.get(owningBucketOrd * 2 + 1), newKey); + maxes.set(owningBucketOrd, max); if (oldEstimatedBucketCount <= targetBuckets * roundingInfos[oldRounding].getMaximumInnerInterval() && max - min <= targetBuckets * roundingInfos[oldRounding].getMaximumRoughEstimateDurationMillis()) { return oldRounding; @@ -281,8 +288,8 @@ private byte increaseRounding(long owningBucketOrd, int oldEstimatedBucketCount, newEstimatedBucketCount > targetBuckets * roundingInfos[newRounding].getMaximumInnerInterval() || max - min > targetBuckets * roundingInfos[newRounding].getMaximumRoughEstimateDurationMillis())); setRounding(owningBucketOrd, newRounding); - bounds.set(owningBucketOrd * 2, preparedRoundings[newRounding].round(bounds.get(owningBucketOrd * 2))); - bounds.set(owningBucketOrd * 2 + 1, preparedRoundings[newRounding].round(bounds.get(owningBucketOrd * 2 + 1))); + mins.set(owningBucketOrd, preparedRoundings[newRounding].round(mins.get(owningBucketOrd))); + maxes.set(owningBucketOrd, preparedRoundings[newRounding].round(maxes.get(owningBucketOrd))); wastedBucketsOverestimate += oldEstimatedBucketCount - newEstimatedBucketCount; if (wastedBucketsOverestimate > nextRebucketAt) { rebucket(); @@ -323,7 +330,7 @@ private void rebucket() { @Override public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { -// correctRounding(owningBucketOrds); + // NOCOMMIT fix the comment /* * Now that we have the perfect rounding rebucket everything to merge * all of the buckets together that we were too lazy to merge while @@ -354,42 +361,6 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I }); } - /** - * Pick the correct rounding for the specified {@code owningBucketOrds}. - */ - private void correctRounding(long[] owningBucketOrds) { - for (long owningBucketOrd : owningBucketOrds) { - byte oldRounding = roundingIndexFor(owningBucketOrd); - if (oldRounding >= roundingInfos.length - 1) { - continue; - } - byte newRounding = (byte)(oldRounding - 1); - long count; - long min = Long.MAX_VALUE; - long max = Long.MIN_VALUE; - do { - newRounding++; - try (LongHash perfect = new LongHash(liveBucketCountUnderestimate.get(owningBucketOrd), context.bigArrays())) { - LongKeyedBucketOrds.BucketOrdsEnum ordsEnum = bucketOrds.ordsEnum(owningBucketOrd); - Rounding.Prepared preparedRounding = preparedRoundings[roundingIndexFor(owningBucketOrd)]; - while (ordsEnum.next()) { - long oldKey = ordsEnum.value(); - long newKey = preparedRounding.round(oldKey); - min = Math.min(min, newKey); - max = Math.max(max, newKey); - perfect.add(newKey); - } - count = perfect.size(); - } - } while (newRounding < roundingInfos.length - 1 && ( - count > targetBuckets * roundingInfos[newRounding].getMaximumInnerInterval() - || max - min > targetBuckets * roundingInfos[newRounding].getMaximumRoughEstimateDurationMillis())); - assert newRounding == oldRounding; - setRounding(owningBucketOrd, newRounding); - wastedBucketsOverestimate += bucketOrds.bucketsInOrd(owningBucketOrd) - count; - } - } - private void setRounding(long owningBucketOrd, byte newRounding) { roundingIndices = context.bigArrays().grow(roundingIndices, owningBucketOrd + 1); roundingIndices.set(owningBucketOrd, newRounding); @@ -420,6 +391,6 @@ private byte roundingIndexFor(long owningBucketOrd) { @Override public void doClose() { - Releasables.close(bucketOrds, roundingIndices, bounds, liveBucketCountUnderestimate); + Releasables.close(bucketOrds, roundingIndices, mins, maxes, liveBucketCountUnderestimate); } } From fd49345e6e27edf98e155ac05425d315e3f16f78 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 1 Jun 2020 14:20:09 -0400 Subject: [PATCH 04/18] Well that was totally wrong --- .../bucket/histogram/AutoDateHistogramAggregator.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 5e7d355174b0e..51574e343dd3c 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 @@ -181,7 +181,7 @@ class AutoDateHistogramAggregator extends DeferableBucketAggregator { mins = context.bigArrays().newLongArray(1, false); mins.set(0, Long.MAX_VALUE); maxes = context.bigArrays().newLongArray(1, false); - maxes.set(1, Long.MIN_VALUE); + maxes.set(0, Long.MIN_VALUE); preparedRoundings = new Rounding.Prepared[roundingInfos.length]; // Prepare the first rounding because we know we'll need it. preparedRoundings[0] = roundingPreparer.apply(roundingInfos[0].rounding); @@ -271,7 +271,7 @@ private byte increaseRoundingIfNeeded(long owningBucketOrd, int oldEstimatedBuck long min = Math.min(mins.get(owningBucketOrd), newKey); mins.set(owningBucketOrd, min); - long max = Math.max(maxes.get(owningBucketOrd * 2 + 1), newKey); + long max = Math.max(maxes.get(owningBucketOrd), newKey); maxes.set(owningBucketOrd, max); if (oldEstimatedBucketCount <= targetBuckets * roundingInfos[oldRounding].getMaximumInnerInterval() && max - min <= targetBuckets * roundingInfos[oldRounding].getMaximumRoughEstimateDurationMillis()) { From ba36e41812c240576b68893c6927e9eb2f4bc6ed Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 1 Jun 2020 15:11:44 -0400 Subject: [PATCH 05/18] Remove nocommit --- .../bucket/histogram/AutoDateHistogramAggregator.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) 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 51574e343dd3c..b4996bf6492cc 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 @@ -330,11 +330,10 @@ private void rebucket() { @Override public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { - // NOCOMMIT fix the comment /* - * Now that we have the perfect rounding rebucket everything to merge - * all of the buckets together that we were too lazy to merge while - * collecting. + * Rebucket the aggregation so we have don't send send "wasted" buckets + * back to the coordinating node because it'll just merge them back + * together. * * TODO it'd be faster if we could apply the merging on the fly as we * replay the hits and build the buckets. How much faster is From 4895af7c95b333b18b71275626644ab40fb88c88 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 2 Jun 2020 10:09:36 -0400 Subject: [PATCH 06/18] Maybe skip rebucketing --- .../histogram/AutoDateHistogramAggregator.java | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) 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 b4996bf6492cc..5beaace46a51c 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 @@ -330,10 +330,13 @@ private void rebucket() { @Override public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + double fractionWasted = ((double) wastedBucketsOverestimate) / ((double) bucketOrds.size()); /* - * Rebucket the aggregation so we have don't send send "wasted" buckets - * back to the coordinating node because it'll just merge them back - * together. + * If there are many incorrect buckets then rebucket the aggregation + * so we have don't send them to the coordinating now. It can turn + * pretty much everything that we can throw at it into right answers + * but it'd be rude to send it huge results just so it can merge them. + * On the other hand, rebucketing is fairly slow. * * TODO it'd be faster if we could apply the merging on the fly as we * replay the hits and build the buckets. How much faster is @@ -343,7 +346,9 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I * But if there is a non-delaying but selectivate aggregation "above" * this one then the performance gain could be substantial. */ - rebucket(); + if (fractionWasted > .2) { + rebucket(); + } return buildAggregationsForVariableBuckets(owningBucketOrds, bucketOrds, (bucketValue, docCount, subAggregationResults) -> new InternalAutoDateHistogram.Bucket(bucketValue, docCount, formatter, subAggregationResults), From 2bf4618f370325866f9953bb8843805272a75b1d Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 2 Jun 2020 10:38:16 -0400 Subject: [PATCH 07/18] Update comment --- .../bucket/histogram/AutoDateHistogramAggregator.java | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) 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 5beaace46a51c..2eaab503ffb7b 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 @@ -339,12 +339,9 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I * On the other hand, rebucketing is fairly slow. * * TODO it'd be faster if we could apply the merging on the fly as we - * replay the hits and build the buckets. How much faster is - * *interesting*. Performance tests with a couple of sub-`stats` aggs - * show `date_histogram` to have about the same performance as - * `auto_date_histogram` so there isn't really much to be gained here. - * But if there is a non-delaying but selectivate aggregation "above" - * this one then the performance gain could be substantial. + * replay the hits and build the buckets. How much faster is not clear, + * but it does have the advantage of only touching the buckets that we + * want to collect. */ if (fractionWasted > .2) { rebucket(); From 43b3512fa17a565c369ddc46b87fc95a3f945710 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 2 Jun 2020 19:14:57 -0400 Subject: [PATCH 08/18] Use int --- .../histogram/AutoDateHistogramAggregator.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) 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 2eaab503ffb7b..c5671f64f6ea4 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 @@ -221,7 +221,7 @@ public void collect(int doc, long owningBucketOrd) throws IOException { int valuesCount = values.docValueCount(); long previousRounded = Long.MIN_VALUE; - byte roundingIdx = roundingIndexFor(owningBucketOrd); + int roundingIdx = roundingIndexFor(owningBucketOrd); for (int i = 0; i < valuesCount; ++i) { long value = values.nextValue(); long rounded = preparedRoundings[roundingIdx].round(value); @@ -235,7 +235,7 @@ public void collect(int doc, long owningBucketOrd) throws IOException { } } - private byte collectValue(LeafBucketCollector sub, long owningBucketOrd, byte roundingIdx, int doc, long rounded) + private int collectValue(LeafBucketCollector sub, long owningBucketOrd, int roundingIdx, int doc, long rounded) throws IOException { long bucketOrd = bucketOrds.add(owningBucketOrd, rounded); if (bucketOrd < 0) { // already seen @@ -254,7 +254,7 @@ private byte collectValue(LeafBucketCollector sub, long owningBucketOrd, byte ro * estimated, bucket counts, {@link #rebucket() rebucketing} the all * buckets if the estimated number of wasted buckets is too high. */ - private byte increaseRoundingIfNeeded(long owningBucketOrd, int oldEstimatedBucketCount, long newKey, byte oldRounding) { + private int increaseRoundingIfNeeded(long owningBucketOrd, int oldEstimatedBucketCount, long newKey, int oldRounding) { if (oldRounding >= roundingInfos.length - 1) { return oldRounding; } @@ -278,7 +278,7 @@ private byte increaseRoundingIfNeeded(long owningBucketOrd, int oldEstimatedBuck return oldRounding; } long oldRoughDuration = roundingInfos[oldRounding].roughEstimateDurationMillis; - byte newRounding = oldRounding; + int newRounding = oldRounding; int newEstimatedBucketCount; do { newRounding++; @@ -362,9 +362,9 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I }); } - private void setRounding(long owningBucketOrd, byte newRounding) { + private void setRounding(long owningBucketOrd, int newRounding) { roundingIndices = context.bigArrays().grow(roundingIndices, owningBucketOrd + 1); - roundingIndices.set(owningBucketOrd, newRounding); + roundingIndices.set(owningBucketOrd, (byte) newRounding); if (preparedRoundings[newRounding] == null) { preparedRoundings[newRounding] = roundingPreparer.apply(roundingInfos[newRounding].rounding); } @@ -386,7 +386,7 @@ public void collectDebugInfo(BiConsumer add) { add.accept("rebucket_count", rebucketCount); } - private byte roundingIndexFor(long owningBucketOrd) { + private int roundingIndexFor(long owningBucketOrd) { return owningBucketOrd < roundingIndices.size() ? roundingIndices.get(owningBucketOrd) : 0; } From ec12da27fa5f31b08ec938abf5469716f1e201ec Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 3 Jun 2020 09:58:54 -0400 Subject: [PATCH 09/18] Rounding strat --- .../AutoDateHistogramAggregator.java | 239 +++++++++++++----- 1 file changed, 169 insertions(+), 70 deletions(-) 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 c5671f64f6ea4..bd9f8b5d667a6 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 @@ -24,7 +24,9 @@ import org.apache.lucene.util.CollectionUtil; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Rounding; +import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.ByteArray; import org.elasticsearch.common.util.IntArray; import org.elasticsearch.common.util.LongArray; @@ -92,34 +94,16 @@ class AutoDateHistogramAggregator extends DeferableBucketAggregator { private final ValuesSource.Numeric valuesSource; private final DocValueFormat formatter; - private final RoundingInfo[] roundingInfos; - private final Function roundingPreparer; private final int targetBuckets; private final boolean collectsFromSingleBucket; - /** - * An array of prepared roundings in the same order as - * {@link #roundingInfos}. The 0th entry is prepared initially, - * and other entries are null until first needed. - */ - private final Rounding.Prepared[] preparedRoundings; + + private final PreparedRoundings roundings; + /** * Map from {@code owningBucketOrd, roundedDate} to {@code bucketOrdinal}. */ private LongKeyedBucketOrds bucketOrds; - /** - * The index of the rounding that each {@code owningBucketOrd} is - * currently using. - *

- * During collection we use overestimates for how much buckets are save - * by bumping to the next rounding index. So we end up bumping less - * aggressively than a "perfect" algorithm. That is fine because we - * correct the error when we merge the buckets together all the way - * up in {@link InternalAutoDateHistogram#reduceBucket}. In particular, - * on final reduce we bump the rounding until it we appropriately - * cover the date range across all of the results returned by all of - * the {@link AutoDateHistogramAggregator}s. - */ - private ByteArray roundingIndices; + /** * The minimum key per {@code owningBucketOrd}. */ @@ -173,18 +157,15 @@ class AutoDateHistogramAggregator extends DeferableBucketAggregator { this.targetBuckets = numBuckets; this.valuesSource = (ValuesSource.Numeric) valuesSource; this.formatter = formatter; - this.roundingInfos = roundingInfos; - this.roundingPreparer = roundingPreparer; this.collectsFromSingleBucket = collectsFromSingleBucket; assert roundingInfos.length < 127 : "Rounding must fit in a signed byte"; - roundingIndices = context.bigArrays().newByteArray(1, true); + this.roundings = collectsFromSingleBucket + ? new SingleBucketPreparedRoundings(roundingPreparer, roundingInfos) + : new ManyBucketsPreparedRoundings(context.bigArrays(), roundingPreparer, roundingInfos); mins = context.bigArrays().newLongArray(1, false); mins.set(0, Long.MAX_VALUE); maxes = context.bigArrays().newLongArray(1, false); maxes.set(0, Long.MIN_VALUE); - preparedRoundings = new Rounding.Prepared[roundingInfos.length]; - // Prepare the first rounding because we know we'll need it. - preparedRoundings[0] = roundingPreparer.apply(roundingInfos[0].rounding); bucketOrds = LongKeyedBucketOrds.build(context.bigArrays(), collectsFromSingleBucket); liveBucketCountUnderestimate = context.bigArrays().newIntArray(1, true); } @@ -221,41 +202,51 @@ public void collect(int doc, long owningBucketOrd) throws IOException { int valuesCount = values.docValueCount(); long previousRounded = Long.MIN_VALUE; - int roundingIdx = roundingIndexFor(owningBucketOrd); + IndexedRounding rounding = roundings.get(owningBucketOrd); for (int i = 0; i < valuesCount; ++i) { long value = values.nextValue(); - long rounded = preparedRoundings[roundingIdx].round(value); + long rounded = rounding.prepared.round(value); assert rounded >= previousRounded; if (rounded == previousRounded) { continue; } - roundingIdx = collectValue(sub, owningBucketOrd, roundingIdx, doc, rounded); + rounding = collectValue(sub, owningBucketOrd, doc, rounded, rounding); previousRounded = rounded; } } } - private int collectValue(LeafBucketCollector sub, long owningBucketOrd, int roundingIdx, int doc, long rounded) - throws IOException { + private IndexedRounding collectValue( + LeafBucketCollector sub, + long owningBucketOrd, + int doc, + long rounded, + IndexedRounding rounding + ) throws IOException { long bucketOrd = bucketOrds.add(owningBucketOrd, rounded); if (bucketOrd < 0) { // already seen bucketOrd = -1 - bucketOrd; collectExistingBucket(sub, doc, bucketOrd); - return roundingIdx; + return rounding; } collectBucket(sub, doc, bucketOrd); liveBucketCountUnderestimate = context.bigArrays().grow(liveBucketCountUnderestimate, owningBucketOrd + 1); int estimatedBucketCount = liveBucketCountUnderestimate.increment(owningBucketOrd, 1); - return increaseRoundingIfNeeded(owningBucketOrd, estimatedBucketCount, rounded, roundingIdx); + return increaseRoundingIfNeeded(owningBucketOrd, estimatedBucketCount, rounded, rounding); } /** - * Increase the rounding of {@code owningBucketOrd} using - * estimated, bucket counts, {@link #rebucket() rebucketing} the all - * buckets if the estimated number of wasted buckets is too high. + * Check if we need increase the rounding of {@code owningBucketOrd} using + * estimated bucket counts and the interval between the min and max buckets. + * {@link #rebucket()} if the new estimated number of wasted buckets is too high. */ - private int increaseRoundingIfNeeded(long owningBucketOrd, int oldEstimatedBucketCount, long newKey, int oldRounding) { - if (oldRounding >= roundingInfos.length - 1) { + private IndexedRounding increaseRoundingIfNeeded( + long owningBucketOrd, + int oldEstimatedBucketCount, + long newKey, + IndexedRounding oldRounding + ) { + if (false == oldRounding.canIncrease) { return oldRounding; } if (mins.size() < owningBucketOrd + 1) { @@ -273,23 +264,23 @@ private int increaseRoundingIfNeeded(long owningBucketOrd, int oldEstimatedBucke mins.set(owningBucketOrd, min); long max = Math.max(maxes.get(owningBucketOrd), newKey); maxes.set(owningBucketOrd, max); - if (oldEstimatedBucketCount <= targetBuckets * roundingInfos[oldRounding].getMaximumInnerInterval() - && max - min <= targetBuckets * roundingInfos[oldRounding].getMaximumRoughEstimateDurationMillis()) { + if (oldEstimatedBucketCount <= targetBuckets * roundings.infos[oldRounding.index].getMaximumInnerInterval() + && max - min <= targetBuckets * roundings.infos[oldRounding.index].getMaximumRoughEstimateDurationMillis()) { return oldRounding; } - long oldRoughDuration = roundingInfos[oldRounding].roughEstimateDurationMillis; - int newRounding = oldRounding; + long oldRoughDuration = roundings.infos[oldRounding.index].roughEstimateDurationMillis; + int newRounding = oldRounding.index; int newEstimatedBucketCount; do { newRounding++; - double ratio = (double) oldRoughDuration / (double) roundingInfos[newRounding].getRoughEstimateDurationMillis(); + double ratio = (double) oldRoughDuration / (double) roundings.infos[newRounding].getRoughEstimateDurationMillis(); newEstimatedBucketCount = (int) Math.ceil(oldEstimatedBucketCount * ratio); - } while (newRounding < roundingInfos.length - 1 && ( - newEstimatedBucketCount > targetBuckets * roundingInfos[newRounding].getMaximumInnerInterval() - || max - min > targetBuckets * roundingInfos[newRounding].getMaximumRoughEstimateDurationMillis())); - setRounding(owningBucketOrd, newRounding); - mins.set(owningBucketOrd, preparedRoundings[newRounding].round(mins.get(owningBucketOrd))); - maxes.set(owningBucketOrd, preparedRoundings[newRounding].round(maxes.get(owningBucketOrd))); + } while (newRounding < roundings.infos.length - 1 && ( + newEstimatedBucketCount > targetBuckets * roundings.infos[newRounding].getMaximumInnerInterval() + || max - min > targetBuckets * roundings.infos[newRounding].getMaximumRoughEstimateDurationMillis())); + IndexedRounding newIndexedRounding = roundings.set(owningBucketOrd, newRounding); + mins.set(owningBucketOrd, newIndexedRounding.prepared.round(mins.get(owningBucketOrd))); + maxes.set(owningBucketOrd, newIndexedRounding.prepared.round(maxes.get(owningBucketOrd))); wastedBucketsOverestimate += oldEstimatedBucketCount - newEstimatedBucketCount; if (wastedBucketsOverestimate > nextRebucketAt) { rebucket(); @@ -299,7 +290,7 @@ private int increaseRoundingIfNeeded(long owningBucketOrd, int oldEstimatedBucke } else { liveBucketCountUnderestimate.set(owningBucketOrd, newEstimatedBucketCount); } - return newRounding; + return newIndexedRounding; } }; } @@ -311,10 +302,10 @@ private void rebucket() { bucketOrds = LongKeyedBucketOrds.build(context.bigArrays(), collectsFromSingleBucket); for (long owningBucketOrd = 0; owningBucketOrd <= oldOrds.maxOwningBucketOrd(); owningBucketOrd++) { LongKeyedBucketOrds.BucketOrdsEnum ordsEnum = oldOrds.ordsEnum(owningBucketOrd); - Rounding.Prepared preparedRounding = preparedRoundings[roundingIndexFor(owningBucketOrd)]; + IndexedRounding preparedRounding = roundings.get(owningBucketOrd); while (ordsEnum.next()) { long oldKey = ordsEnum.value(); - long newKey = preparedRounding.round(oldKey); + long newKey = preparedRounding.prepared.round(oldKey); long newBucketOrd = bucketOrds.add(owningBucketOrd, newKey); mergeMap[(int) ordsEnum.ord()] = newBucketOrd >= 0 ? newBucketOrd : -1 - newBucketOrd; } @@ -355,24 +346,16 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I CollectionUtil.introSort(buckets, BucketOrder.key(true).comparator()); // value source will be null for unmapped fields - InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundingInfos, - roundingIndexFor(owningBucketOrd), buildEmptySubAggregations()); + InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundings.infos, + roundings.get(owningBucketOrd).index, buildEmptySubAggregations()); return new InternalAutoDateHistogram(name, buckets, targetBuckets, emptyBucketInfo, formatter, metadata(), 1); }); } - private void setRounding(long owningBucketOrd, int newRounding) { - roundingIndices = context.bigArrays().grow(roundingIndices, owningBucketOrd + 1); - roundingIndices.set(owningBucketOrd, (byte) newRounding); - if (preparedRoundings[newRounding] == null) { - preparedRoundings[newRounding] = roundingPreparer.apply(roundingInfos[newRounding].rounding); - } - } - @Override public InternalAggregation buildEmptyAggregation() { - InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundingInfos, 0, + InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundings.infos, 0, buildEmptySubAggregations()); return new InternalAutoDateHistogram(name, Collections.emptyList(), targetBuckets, emptyBucketInfo, formatter, metadata(), 1); } @@ -386,12 +369,128 @@ public void collectDebugInfo(BiConsumer add) { add.accept("rebucket_count", rebucketCount); } - private int roundingIndexFor(long owningBucketOrd) { - return owningBucketOrd < roundingIndices.size() ? roundingIndices.get(owningBucketOrd) : 0; - } - @Override public void doClose() { - Releasables.close(bucketOrds, roundingIndices, mins, maxes, liveBucketCountUnderestimate); + Releasables.close(bucketOrds, roundings, mins, maxes, liveBucketCountUnderestimate); + } + + protected abstract static class PreparedRoundings implements Releasable { + private final Function preparer; + private final RoundingInfo[] infos; + + public PreparedRoundings(Function preparer, RoundingInfo[] infos) { + this.preparer = preparer; + this.infos = infos; + } + + protected abstract IndexedRounding get(long owningBucketOrd); + + protected abstract IndexedRounding set(long owningBucketOrd, int newRounding); + + protected final IndexedRounding prepare(int index) { + return new IndexedRounding(index, index < infos.length - 1, preparer.apply(infos[index].rounding)); + } + } + + protected static class SingleBucketPreparedRoundings extends PreparedRoundings { + private IndexedRounding current; + + public SingleBucketPreparedRoundings(Function roundingPreparer, RoundingInfo[] roundingInfos) { + super(roundingPreparer, roundingInfos); + current = prepare(0); + } + + @Override + protected IndexedRounding get(long owningBucketOrd) { + assert owningBucketOrd == 0; + return current; + } + + @Override + protected IndexedRounding set(long owningBucketOrd, int newRounding) { + assert owningBucketOrd == 0; + current = prepare(newRounding); + return current; + } + + @Override + public void close() {} + } + + protected static class ManyBucketsPreparedRoundings extends PreparedRoundings { + private final BigArrays bigArrays; + /** + * An array of prepared roundings in the same order as + * {@link #infos}. The 0th entry is prepared initially, + * and other entries are null until first needed. + */ + private final IndexedRounding[] prepared; + /** + * The index of the rounding that each {@code owningBucketOrd} is + * currently using. + *

+ * During collection we use overestimates for how much buckets are save + * by bumping to the next rounding index. So we end up bumping less + * aggressively than a "perfect" algorithm. That is fine because we + * correct the error when we merge the buckets together all the way + * up in {@link InternalAutoDateHistogram#reduceBucket}. In particular, + * on final reduce we bump the rounding until it we appropriately + * cover the date range across all of the results returned by all of + * the {@link AutoDateHistogramAggregator}s. + */ + private ByteArray indices; + + public ManyBucketsPreparedRoundings( + BigArrays bigArrays, + Function roundingPreparer, + RoundingInfo[] roundingInfos + ) { + super(roundingPreparer, roundingInfos); + this.bigArrays = bigArrays; + this.prepared = new IndexedRounding[roundingInfos.length]; + indices = bigArrays.newByteArray(1, true); + // Prepare the first rounding because we know we'll need it. + prepared[0] = prepare(0); + } + + @Override + protected IndexedRounding get(long owningBucketOrd) { + if (owningBucketOrd >= indices.size()) { + return prepared[0]; + } + /* + * This will never return null because we always prepare a rounding + * at the index when we set the index. + */ + return prepared[indices.get(owningBucketOrd)]; + } + + @Override + protected IndexedRounding set(long owningBucketOrd, int newRounding) { + indices = bigArrays.grow(indices, owningBucketOrd + 1); + indices.set(owningBucketOrd, (byte) newRounding); + if (prepared[newRounding] == null) { + prepared[newRounding] = prepare(newRounding); + } + return prepared[newRounding]; + } + + @Override + public void close() { + indices.close(); + } + } + + private static class IndexedRounding { + private final int index; + private final boolean canIncrease; + private final Rounding.Prepared prepared; + + public IndexedRounding(int index, boolean canIncrease, Rounding.Prepared prepared) { + this.index = index; + this.canIncrease = canIncrease; + this.prepared = prepared; + } + } } From 6aca7a5eec6bdf9d949c04d68d96baa4408c8d84 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 3 Jun 2020 13:53:22 -0400 Subject: [PATCH 10/18] Time --- .../AutoDateHistogramAggregator.java | 35 ++++++++++++++++++- 1 file changed, 34 insertions(+), 1 deletion(-) 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 bd9f8b5d667a6..a39e49e15fd07 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 @@ -44,6 +44,7 @@ import org.elasticsearch.search.aggregations.bucket.terms.LongKeyedBucketOrds; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.search.profile.Timer; import java.io.IOException; import java.util.Collections; @@ -138,6 +139,13 @@ class AutoDateHistogramAggregator extends DeferableBucketAggregator { */ private int rebucketCount = 0; + private final Timer valueCountTimer = new Timer(); + private final Timer nextValueTimer = new Timer(); + private final Timer roundTimer = new Timer(); + private final Timer ordLookupTimer = new Timer(); + private final Timer newOrdTimer = new Timer(); + private final Timer oldOrdTimer = new Timer(); + AutoDateHistogramAggregator( String name, AggregatorFactories factories, @@ -199,13 +207,19 @@ public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCol @Override public void collect(int doc, long owningBucketOrd) throws IOException { if (values.advanceExact(doc)) { + valueCountTimer.start(); int valuesCount = values.docValueCount(); + valueCountTimer.stop(); long previousRounded = Long.MIN_VALUE; IndexedRounding rounding = roundings.get(owningBucketOrd); for (int i = 0; i < valuesCount; ++i) { + nextValueTimer.start(); long value = values.nextValue(); + nextValueTimer.stop(); + roundTimer.start(); long rounded = rounding.prepared.round(value); + roundTimer.stop(); assert rounded >= previousRounded; if (rounded == previousRounded) { continue; @@ -223,16 +237,23 @@ private IndexedRounding collectValue( long rounded, IndexedRounding rounding ) throws IOException { + ordLookupTimer.start(); long bucketOrd = bucketOrds.add(owningBucketOrd, rounded); + ordLookupTimer.stop(); if (bucketOrd < 0) { // already seen + oldOrdTimer.start(); bucketOrd = -1 - bucketOrd; collectExistingBucket(sub, doc, bucketOrd); + oldOrdTimer.stop(); return rounding; } + newOrdTimer.start(); collectBucket(sub, doc, bucketOrd); liveBucketCountUnderestimate = context.bigArrays().grow(liveBucketCountUnderestimate, owningBucketOrd + 1); int estimatedBucketCount = liveBucketCountUnderestimate.increment(owningBucketOrd, 1); - return increaseRoundingIfNeeded(owningBucketOrd, estimatedBucketCount, rounded, rounding); + IndexedRounding r = increaseRoundingIfNeeded(owningBucketOrd, estimatedBucketCount, rounded, rounding); + newOrdTimer.stop(); + return r; } /** @@ -367,6 +388,18 @@ public void collectDebugInfo(BiConsumer add) { add.accept("wasted_buckets_overestimate", wastedBucketsOverestimate); add.accept("next_rebucket_at", nextRebucketAt); add.accept("rebucket_count", rebucketCount); + add.accept("value_count_time", valueCountTimer.getApproximateTiming()); + add.accept("value_count_count", valueCountTimer.getCount()); + add.accept("next_value_time", nextValueTimer.getApproximateTiming()); + add.accept("next_value_count", nextValueTimer.getCount()); + add.accept("round_time", roundTimer.getApproximateTiming()); + add.accept("round_count", roundTimer.getCount()); + add.accept("ord_lookup_time", ordLookupTimer.getApproximateTiming()); + add.accept("ord_lookup_count", ordLookupTimer.getCount()); + add.accept("new_ord_time", newOrdTimer.getApproximateTiming()); + add.accept("new_ord_count", newOrdTimer.getCount()); + add.accept("old_ord_time", oldOrdTimer.getApproximateTiming()); + add.accept("old_ord_count", oldOrdTimer.getCount()); } @Override From 94f84732648cbad65f3fc09bcb50bd41f9366352 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 3 Jun 2020 15:01:55 -0400 Subject: [PATCH 11/18] Single values --- .../AutoDateHistogramAggregator.java | 234 +++++++++--------- 1 file changed, 115 insertions(+), 119 deletions(-) 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 a39e49e15fd07..0197e6e4930ef 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 @@ -18,7 +18,9 @@ */ package org.elasticsearch.search.aggregations.bucket.histogram; +import org.apache.lucene.index.DocValues; import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.SortedNumericDocValues; import org.apache.lucene.search.ScoreMode; import org.apache.lucene.util.CollectionUtil; @@ -138,13 +140,8 @@ class AutoDateHistogramAggregator extends DeferableBucketAggregator { * results. We keep this just to report to the profiler. */ private int rebucketCount = 0; - - private final Timer valueCountTimer = new Timer(); - private final Timer nextValueTimer = new Timer(); - private final Timer roundTimer = new Timer(); - private final Timer ordLookupTimer = new Timer(); - private final Timer newOrdTimer = new Timer(); - private final Timer oldOrdTimer = new Timer(); + private int segmentsWithSingleValuedOrds = 0; + private int segmentsWithMultiValuedOrds = 0; AutoDateHistogramAggregator( String name, @@ -203,117 +200,126 @@ public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCol return LeafBucketCollector.NO_OP_COLLECTOR; } SortedNumericDocValues values = valuesSource.longValues(ctx); - return new LeafBucketCollectorBase(sub, values) { + NumericDocValues singleValues = DocValues.unwrapSingleton(values); + if (singleValues != null) { + segmentsWithSingleValuedOrds++; + return new Collector(sub, values) { + @Override + public void collect(int doc, long owningBucketOrd) throws IOException { + if (false == singleValues.advanceExact(doc)) { + return; + } + IndexedRounding rounding = roundings.get(owningBucketOrd); + long rounded = roundings.get(owningBucketOrd).prepared.round(singleValues.longValue()); + collectValue(sub, owningBucketOrd, doc, rounded, rounding); + } + }; + } + segmentsWithMultiValuedOrds++; + return new Collector(sub, values) { @Override public void collect(int doc, long owningBucketOrd) throws IOException { if (values.advanceExact(doc)) { - valueCountTimer.start(); - int valuesCount = values.docValueCount(); - valueCountTimer.stop(); - - long previousRounded = Long.MIN_VALUE; - IndexedRounding rounding = roundings.get(owningBucketOrd); - for (int i = 0; i < valuesCount; ++i) { - nextValueTimer.start(); - long value = values.nextValue(); - nextValueTimer.stop(); - roundTimer.start(); - long rounded = rounding.prepared.round(value); - roundTimer.stop(); - assert rounded >= previousRounded; - if (rounded == previousRounded) { - continue; - } - rounding = collectValue(sub, owningBucketOrd, doc, rounded, rounding); - previousRounded = rounded; + return; + } + int valuesCount = values.docValueCount(); + + long previousRounded = Long.MIN_VALUE; + IndexedRounding rounding = roundings.get(owningBucketOrd); + for (int i = 0; i < valuesCount; ++i) { + long rounded = rounding.prepared.round(values.nextValue()); + if (rounded == previousRounded) { + continue; } + rounding = collectValue(sub, owningBucketOrd, doc, rounded, rounding); + previousRounded = rounded; } } + }; + } - private IndexedRounding collectValue( - LeafBucketCollector sub, - long owningBucketOrd, - int doc, - long rounded, - IndexedRounding rounding - ) throws IOException { - ordLookupTimer.start(); - long bucketOrd = bucketOrds.add(owningBucketOrd, rounded); - ordLookupTimer.stop(); - if (bucketOrd < 0) { // already seen - oldOrdTimer.start(); - bucketOrd = -1 - bucketOrd; - collectExistingBucket(sub, doc, bucketOrd); - oldOrdTimer.stop(); - return rounding; - } - newOrdTimer.start(); - collectBucket(sub, doc, bucketOrd); - liveBucketCountUnderestimate = context.bigArrays().grow(liveBucketCountUnderestimate, owningBucketOrd + 1); - int estimatedBucketCount = liveBucketCountUnderestimate.increment(owningBucketOrd, 1); - IndexedRounding r = increaseRoundingIfNeeded(owningBucketOrd, estimatedBucketCount, rounded, rounding); - newOrdTimer.stop(); - return r; + protected abstract class Collector extends LeafBucketCollectorBase { + public Collector(LeafBucketCollector sub, Object values) { + super(sub, values); + } + + protected IndexedRounding collectValue( + LeafBucketCollector sub, + long owningBucketOrd, + int doc, + long rounded, + IndexedRounding rounding + ) throws IOException { + long bucketOrd = bucketOrds.add(owningBucketOrd, rounded); + if (bucketOrd < 0) { // already seen + bucketOrd = -1 - bucketOrd; + collectExistingBucket(sub, doc, bucketOrd); + return rounding; } + collectBucket(sub, doc, bucketOrd); + liveBucketCountUnderestimate = context.bigArrays().grow(liveBucketCountUnderestimate, owningBucketOrd + 1); + int estimatedBucketCount = liveBucketCountUnderestimate.increment(owningBucketOrd, 1); + IndexedRounding r = increaseRoundingIfNeeded(owningBucketOrd, estimatedBucketCount, rounded, rounding); + return r; + } - /** - * Check if we need increase the rounding of {@code owningBucketOrd} using - * estimated bucket counts and the interval between the min and max buckets. - * {@link #rebucket()} if the new estimated number of wasted buckets is too high. - */ - private IndexedRounding increaseRoundingIfNeeded( - long owningBucketOrd, - int oldEstimatedBucketCount, - long newKey, - IndexedRounding oldRounding - ) { - if (false == oldRounding.canIncrease) { - return oldRounding; - } - if (mins.size() < owningBucketOrd + 1) { - long oldSize = mins.size(); - mins = context.bigArrays().grow(mins, owningBucketOrd + 1); - mins.fill(oldSize, mins.size(), Long.MAX_VALUE); - } - if (maxes.size() < owningBucketOrd + 1) { - long oldSize = maxes.size(); - maxes = context.bigArrays().grow(maxes, owningBucketOrd + 1); - maxes.fill(oldSize, maxes.size(), Long.MIN_VALUE); - } + /** + * Check if we need increase the rounding of {@code owningBucketOrd} using + * estimated bucket counts and the interval between the min and max buckets. + * {@link #rebucket()} if the new estimated number of wasted buckets is too high. + */ + private IndexedRounding increaseRoundingIfNeeded( + long owningBucketOrd, + int oldEstimatedBucketCount, + long newKey, + IndexedRounding oldRounding + ) { + if (false == oldRounding.canIncrease) { + return oldRounding; + } + if (mins.size() < owningBucketOrd + 1) { + long oldSize = mins.size(); + mins = context.bigArrays().grow(mins, owningBucketOrd + 1); + mins.fill(oldSize, mins.size(), Long.MAX_VALUE); + } + if (maxes.size() < owningBucketOrd + 1) { + long oldSize = maxes.size(); + maxes = context.bigArrays().grow(maxes, owningBucketOrd + 1); + maxes.fill(oldSize, maxes.size(), Long.MIN_VALUE); + } - long min = Math.min(mins.get(owningBucketOrd), newKey); - mins.set(owningBucketOrd, min); - long max = Math.max(maxes.get(owningBucketOrd), newKey); - maxes.set(owningBucketOrd, max); - if (oldEstimatedBucketCount <= targetBuckets * roundings.infos[oldRounding.index].getMaximumInnerInterval() - && max - min <= targetBuckets * roundings.infos[oldRounding.index].getMaximumRoughEstimateDurationMillis()) { - return oldRounding; - } - long oldRoughDuration = roundings.infos[oldRounding.index].roughEstimateDurationMillis; - int newRounding = oldRounding.index; - int newEstimatedBucketCount; - do { - newRounding++; - double ratio = (double) oldRoughDuration / (double) roundings.infos[newRounding].getRoughEstimateDurationMillis(); - newEstimatedBucketCount = (int) Math.ceil(oldEstimatedBucketCount * ratio); - } while (newRounding < roundings.infos.length - 1 && ( - newEstimatedBucketCount > targetBuckets * roundings.infos[newRounding].getMaximumInnerInterval() - || max - min > targetBuckets * roundings.infos[newRounding].getMaximumRoughEstimateDurationMillis())); - IndexedRounding newIndexedRounding = roundings.set(owningBucketOrd, newRounding); - mins.set(owningBucketOrd, newIndexedRounding.prepared.round(mins.get(owningBucketOrd))); - maxes.set(owningBucketOrd, newIndexedRounding.prepared.round(maxes.get(owningBucketOrd))); - wastedBucketsOverestimate += oldEstimatedBucketCount - newEstimatedBucketCount; - if (wastedBucketsOverestimate > nextRebucketAt) { - rebucket(); - // Bump the threshold for the next rebucketing - wastedBucketsOverestimate = 0; - nextRebucketAt *= 2; - } else { - liveBucketCountUnderestimate.set(owningBucketOrd, newEstimatedBucketCount); - } - return newIndexedRounding; + long min = Math.min(mins.get(owningBucketOrd), newKey); + mins.set(owningBucketOrd, min); + long max = Math.max(maxes.get(owningBucketOrd), newKey); + maxes.set(owningBucketOrd, max); + if (oldEstimatedBucketCount <= targetBuckets * roundings.infos[oldRounding.index].getMaximumInnerInterval() + && max - min <= targetBuckets * roundings.infos[oldRounding.index].getMaximumRoughEstimateDurationMillis()) { + return oldRounding; } - }; + long oldRoughDuration = roundings.infos[oldRounding.index].roughEstimateDurationMillis; + int newRounding = oldRounding.index; + int newEstimatedBucketCount; + do { + newRounding++; + double ratio = (double) oldRoughDuration / (double) roundings.infos[newRounding].getRoughEstimateDurationMillis(); + newEstimatedBucketCount = (int) Math.ceil(oldEstimatedBucketCount * ratio); + } while (newRounding < roundings.infos.length - 1 && ( + newEstimatedBucketCount > targetBuckets * roundings.infos[newRounding].getMaximumInnerInterval() + || max - min > targetBuckets * roundings.infos[newRounding].getMaximumRoughEstimateDurationMillis())); + IndexedRounding newIndexedRounding = roundings.set(owningBucketOrd, newRounding); + mins.set(owningBucketOrd, newIndexedRounding.prepared.round(mins.get(owningBucketOrd))); + maxes.set(owningBucketOrd, newIndexedRounding.prepared.round(maxes.get(owningBucketOrd))); + wastedBucketsOverestimate += oldEstimatedBucketCount - newEstimatedBucketCount; + if (wastedBucketsOverestimate > nextRebucketAt) { + rebucket(); + // Bump the threshold for the next rebucketing + wastedBucketsOverestimate = 0; + nextRebucketAt *= 2; + } else { + liveBucketCountUnderestimate.set(owningBucketOrd, newEstimatedBucketCount); + } + return newIndexedRounding; + } } private void rebucket() { @@ -388,18 +394,8 @@ public void collectDebugInfo(BiConsumer add) { add.accept("wasted_buckets_overestimate", wastedBucketsOverestimate); add.accept("next_rebucket_at", nextRebucketAt); add.accept("rebucket_count", rebucketCount); - add.accept("value_count_time", valueCountTimer.getApproximateTiming()); - add.accept("value_count_count", valueCountTimer.getCount()); - add.accept("next_value_time", nextValueTimer.getApproximateTiming()); - add.accept("next_value_count", nextValueTimer.getCount()); - add.accept("round_time", roundTimer.getApproximateTiming()); - add.accept("round_count", roundTimer.getCount()); - add.accept("ord_lookup_time", ordLookupTimer.getApproximateTiming()); - add.accept("ord_lookup_count", ordLookupTimer.getCount()); - add.accept("new_ord_time", newOrdTimer.getApproximateTiming()); - add.accept("new_ord_count", newOrdTimer.getCount()); - add.accept("old_ord_time", oldOrdTimer.getApproximateTiming()); - add.accept("old_ord_count", oldOrdTimer.getCount()); + add.accept("segments_with_single_valued_ords", segmentsWithSingleValuedOrds); + add.accept("segments_with_multi_valued_ords", segmentsWithMultiValuedOrds); } @Override From 77ca03a06fe18b14e008dad20cfdf3ad3aa87600 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Thu, 4 Jun 2020 07:58:41 -0400 Subject: [PATCH 12/18] Revert "Single values" This reverts commit 94f84732648cbad65f3fc09bcb50bd41f9366352. --- .../AutoDateHistogramAggregator.java | 234 +++++++++--------- 1 file changed, 119 insertions(+), 115 deletions(-) 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 0197e6e4930ef..a39e49e15fd07 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 @@ -18,9 +18,7 @@ */ package org.elasticsearch.search.aggregations.bucket.histogram; -import org.apache.lucene.index.DocValues; import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.SortedNumericDocValues; import org.apache.lucene.search.ScoreMode; import org.apache.lucene.util.CollectionUtil; @@ -140,8 +138,13 @@ class AutoDateHistogramAggregator extends DeferableBucketAggregator { * results. We keep this just to report to the profiler. */ private int rebucketCount = 0; - private int segmentsWithSingleValuedOrds = 0; - private int segmentsWithMultiValuedOrds = 0; + + private final Timer valueCountTimer = new Timer(); + private final Timer nextValueTimer = new Timer(); + private final Timer roundTimer = new Timer(); + private final Timer ordLookupTimer = new Timer(); + private final Timer newOrdTimer = new Timer(); + private final Timer oldOrdTimer = new Timer(); AutoDateHistogramAggregator( String name, @@ -200,126 +203,117 @@ public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCol return LeafBucketCollector.NO_OP_COLLECTOR; } SortedNumericDocValues values = valuesSource.longValues(ctx); - NumericDocValues singleValues = DocValues.unwrapSingleton(values); - if (singleValues != null) { - segmentsWithSingleValuedOrds++; - return new Collector(sub, values) { - @Override - public void collect(int doc, long owningBucketOrd) throws IOException { - if (false == singleValues.advanceExact(doc)) { - return; - } - IndexedRounding rounding = roundings.get(owningBucketOrd); - long rounded = roundings.get(owningBucketOrd).prepared.round(singleValues.longValue()); - collectValue(sub, owningBucketOrd, doc, rounded, rounding); - } - }; - } - segmentsWithMultiValuedOrds++; - return new Collector(sub, values) { + return new LeafBucketCollectorBase(sub, values) { @Override public void collect(int doc, long owningBucketOrd) throws IOException { if (values.advanceExact(doc)) { - return; - } - int valuesCount = values.docValueCount(); - - long previousRounded = Long.MIN_VALUE; - IndexedRounding rounding = roundings.get(owningBucketOrd); - for (int i = 0; i < valuesCount; ++i) { - long rounded = rounding.prepared.round(values.nextValue()); - if (rounded == previousRounded) { - continue; + valueCountTimer.start(); + int valuesCount = values.docValueCount(); + valueCountTimer.stop(); + + long previousRounded = Long.MIN_VALUE; + IndexedRounding rounding = roundings.get(owningBucketOrd); + for (int i = 0; i < valuesCount; ++i) { + nextValueTimer.start(); + long value = values.nextValue(); + nextValueTimer.stop(); + roundTimer.start(); + long rounded = rounding.prepared.round(value); + roundTimer.stop(); + assert rounded >= previousRounded; + if (rounded == previousRounded) { + continue; + } + rounding = collectValue(sub, owningBucketOrd, doc, rounded, rounding); + previousRounded = rounded; } - rounding = collectValue(sub, owningBucketOrd, doc, rounded, rounding); - previousRounded = rounded; } } - }; - } - - protected abstract class Collector extends LeafBucketCollectorBase { - public Collector(LeafBucketCollector sub, Object values) { - super(sub, values); - } - protected IndexedRounding collectValue( - LeafBucketCollector sub, - long owningBucketOrd, - int doc, - long rounded, - IndexedRounding rounding - ) throws IOException { - long bucketOrd = bucketOrds.add(owningBucketOrd, rounded); - if (bucketOrd < 0) { // already seen - bucketOrd = -1 - bucketOrd; - collectExistingBucket(sub, doc, bucketOrd); - return rounding; + private IndexedRounding collectValue( + LeafBucketCollector sub, + long owningBucketOrd, + int doc, + long rounded, + IndexedRounding rounding + ) throws IOException { + ordLookupTimer.start(); + long bucketOrd = bucketOrds.add(owningBucketOrd, rounded); + ordLookupTimer.stop(); + if (bucketOrd < 0) { // already seen + oldOrdTimer.start(); + bucketOrd = -1 - bucketOrd; + collectExistingBucket(sub, doc, bucketOrd); + oldOrdTimer.stop(); + return rounding; + } + newOrdTimer.start(); + collectBucket(sub, doc, bucketOrd); + liveBucketCountUnderestimate = context.bigArrays().grow(liveBucketCountUnderestimate, owningBucketOrd + 1); + int estimatedBucketCount = liveBucketCountUnderestimate.increment(owningBucketOrd, 1); + IndexedRounding r = increaseRoundingIfNeeded(owningBucketOrd, estimatedBucketCount, rounded, rounding); + newOrdTimer.stop(); + return r; } - collectBucket(sub, doc, bucketOrd); - liveBucketCountUnderestimate = context.bigArrays().grow(liveBucketCountUnderestimate, owningBucketOrd + 1); - int estimatedBucketCount = liveBucketCountUnderestimate.increment(owningBucketOrd, 1); - IndexedRounding r = increaseRoundingIfNeeded(owningBucketOrd, estimatedBucketCount, rounded, rounding); - return r; - } - /** - * Check if we need increase the rounding of {@code owningBucketOrd} using - * estimated bucket counts and the interval between the min and max buckets. - * {@link #rebucket()} if the new estimated number of wasted buckets is too high. - */ - private IndexedRounding increaseRoundingIfNeeded( - long owningBucketOrd, - int oldEstimatedBucketCount, - long newKey, - IndexedRounding oldRounding - ) { - if (false == oldRounding.canIncrease) { - return oldRounding; - } - if (mins.size() < owningBucketOrd + 1) { - long oldSize = mins.size(); - mins = context.bigArrays().grow(mins, owningBucketOrd + 1); - mins.fill(oldSize, mins.size(), Long.MAX_VALUE); - } - if (maxes.size() < owningBucketOrd + 1) { - long oldSize = maxes.size(); - maxes = context.bigArrays().grow(maxes, owningBucketOrd + 1); - maxes.fill(oldSize, maxes.size(), Long.MIN_VALUE); - } + /** + * Check if we need increase the rounding of {@code owningBucketOrd} using + * estimated bucket counts and the interval between the min and max buckets. + * {@link #rebucket()} if the new estimated number of wasted buckets is too high. + */ + private IndexedRounding increaseRoundingIfNeeded( + long owningBucketOrd, + int oldEstimatedBucketCount, + long newKey, + IndexedRounding oldRounding + ) { + if (false == oldRounding.canIncrease) { + return oldRounding; + } + if (mins.size() < owningBucketOrd + 1) { + long oldSize = mins.size(); + mins = context.bigArrays().grow(mins, owningBucketOrd + 1); + mins.fill(oldSize, mins.size(), Long.MAX_VALUE); + } + if (maxes.size() < owningBucketOrd + 1) { + long oldSize = maxes.size(); + maxes = context.bigArrays().grow(maxes, owningBucketOrd + 1); + maxes.fill(oldSize, maxes.size(), Long.MIN_VALUE); + } - long min = Math.min(mins.get(owningBucketOrd), newKey); - mins.set(owningBucketOrd, min); - long max = Math.max(maxes.get(owningBucketOrd), newKey); - maxes.set(owningBucketOrd, max); - if (oldEstimatedBucketCount <= targetBuckets * roundings.infos[oldRounding.index].getMaximumInnerInterval() - && max - min <= targetBuckets * roundings.infos[oldRounding.index].getMaximumRoughEstimateDurationMillis()) { - return oldRounding; - } - long oldRoughDuration = roundings.infos[oldRounding.index].roughEstimateDurationMillis; - int newRounding = oldRounding.index; - int newEstimatedBucketCount; - do { - newRounding++; - double ratio = (double) oldRoughDuration / (double) roundings.infos[newRounding].getRoughEstimateDurationMillis(); - newEstimatedBucketCount = (int) Math.ceil(oldEstimatedBucketCount * ratio); - } while (newRounding < roundings.infos.length - 1 && ( - newEstimatedBucketCount > targetBuckets * roundings.infos[newRounding].getMaximumInnerInterval() - || max - min > targetBuckets * roundings.infos[newRounding].getMaximumRoughEstimateDurationMillis())); - IndexedRounding newIndexedRounding = roundings.set(owningBucketOrd, newRounding); - mins.set(owningBucketOrd, newIndexedRounding.prepared.round(mins.get(owningBucketOrd))); - maxes.set(owningBucketOrd, newIndexedRounding.prepared.round(maxes.get(owningBucketOrd))); - wastedBucketsOverestimate += oldEstimatedBucketCount - newEstimatedBucketCount; - if (wastedBucketsOverestimate > nextRebucketAt) { - rebucket(); - // Bump the threshold for the next rebucketing - wastedBucketsOverestimate = 0; - nextRebucketAt *= 2; - } else { - liveBucketCountUnderestimate.set(owningBucketOrd, newEstimatedBucketCount); + long min = Math.min(mins.get(owningBucketOrd), newKey); + mins.set(owningBucketOrd, min); + long max = Math.max(maxes.get(owningBucketOrd), newKey); + maxes.set(owningBucketOrd, max); + if (oldEstimatedBucketCount <= targetBuckets * roundings.infos[oldRounding.index].getMaximumInnerInterval() + && max - min <= targetBuckets * roundings.infos[oldRounding.index].getMaximumRoughEstimateDurationMillis()) { + return oldRounding; + } + long oldRoughDuration = roundings.infos[oldRounding.index].roughEstimateDurationMillis; + int newRounding = oldRounding.index; + int newEstimatedBucketCount; + do { + newRounding++; + double ratio = (double) oldRoughDuration / (double) roundings.infos[newRounding].getRoughEstimateDurationMillis(); + newEstimatedBucketCount = (int) Math.ceil(oldEstimatedBucketCount * ratio); + } while (newRounding < roundings.infos.length - 1 && ( + newEstimatedBucketCount > targetBuckets * roundings.infos[newRounding].getMaximumInnerInterval() + || max - min > targetBuckets * roundings.infos[newRounding].getMaximumRoughEstimateDurationMillis())); + IndexedRounding newIndexedRounding = roundings.set(owningBucketOrd, newRounding); + mins.set(owningBucketOrd, newIndexedRounding.prepared.round(mins.get(owningBucketOrd))); + maxes.set(owningBucketOrd, newIndexedRounding.prepared.round(maxes.get(owningBucketOrd))); + wastedBucketsOverestimate += oldEstimatedBucketCount - newEstimatedBucketCount; + if (wastedBucketsOverestimate > nextRebucketAt) { + rebucket(); + // Bump the threshold for the next rebucketing + wastedBucketsOverestimate = 0; + nextRebucketAt *= 2; + } else { + liveBucketCountUnderestimate.set(owningBucketOrd, newEstimatedBucketCount); + } + return newIndexedRounding; } - return newIndexedRounding; - } + }; } private void rebucket() { @@ -394,8 +388,18 @@ public void collectDebugInfo(BiConsumer add) { add.accept("wasted_buckets_overestimate", wastedBucketsOverestimate); add.accept("next_rebucket_at", nextRebucketAt); add.accept("rebucket_count", rebucketCount); - add.accept("segments_with_single_valued_ords", segmentsWithSingleValuedOrds); - add.accept("segments_with_multi_valued_ords", segmentsWithMultiValuedOrds); + add.accept("value_count_time", valueCountTimer.getApproximateTiming()); + add.accept("value_count_count", valueCountTimer.getCount()); + add.accept("next_value_time", nextValueTimer.getApproximateTiming()); + add.accept("next_value_count", nextValueTimer.getCount()); + add.accept("round_time", roundTimer.getApproximateTiming()); + add.accept("round_count", roundTimer.getCount()); + add.accept("ord_lookup_time", ordLookupTimer.getApproximateTiming()); + add.accept("ord_lookup_count", ordLookupTimer.getCount()); + add.accept("new_ord_time", newOrdTimer.getApproximateTiming()); + add.accept("new_ord_count", newOrdTimer.getCount()); + add.accept("old_ord_time", oldOrdTimer.getApproximateTiming()); + add.accept("old_ord_count", oldOrdTimer.getCount()); } @Override From 6411589cfb8fd4be25bc8c2d4656a6a269b2aea9 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Thu, 4 Jun 2020 07:58:43 -0400 Subject: [PATCH 13/18] Revert "Time" This reverts commit 6aca7a5eec6bdf9d949c04d68d96baa4408c8d84. --- .../AutoDateHistogramAggregator.java | 35 +------------------ 1 file changed, 1 insertion(+), 34 deletions(-) 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 a39e49e15fd07..bd9f8b5d667a6 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 @@ -44,7 +44,6 @@ import org.elasticsearch.search.aggregations.bucket.terms.LongKeyedBucketOrds; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.internal.SearchContext; -import org.elasticsearch.search.profile.Timer; import java.io.IOException; import java.util.Collections; @@ -139,13 +138,6 @@ class AutoDateHistogramAggregator extends DeferableBucketAggregator { */ private int rebucketCount = 0; - private final Timer valueCountTimer = new Timer(); - private final Timer nextValueTimer = new Timer(); - private final Timer roundTimer = new Timer(); - private final Timer ordLookupTimer = new Timer(); - private final Timer newOrdTimer = new Timer(); - private final Timer oldOrdTimer = new Timer(); - AutoDateHistogramAggregator( String name, AggregatorFactories factories, @@ -207,19 +199,13 @@ public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCol @Override public void collect(int doc, long owningBucketOrd) throws IOException { if (values.advanceExact(doc)) { - valueCountTimer.start(); int valuesCount = values.docValueCount(); - valueCountTimer.stop(); long previousRounded = Long.MIN_VALUE; IndexedRounding rounding = roundings.get(owningBucketOrd); for (int i = 0; i < valuesCount; ++i) { - nextValueTimer.start(); long value = values.nextValue(); - nextValueTimer.stop(); - roundTimer.start(); long rounded = rounding.prepared.round(value); - roundTimer.stop(); assert rounded >= previousRounded; if (rounded == previousRounded) { continue; @@ -237,23 +223,16 @@ private IndexedRounding collectValue( long rounded, IndexedRounding rounding ) throws IOException { - ordLookupTimer.start(); long bucketOrd = bucketOrds.add(owningBucketOrd, rounded); - ordLookupTimer.stop(); if (bucketOrd < 0) { // already seen - oldOrdTimer.start(); bucketOrd = -1 - bucketOrd; collectExistingBucket(sub, doc, bucketOrd); - oldOrdTimer.stop(); return rounding; } - newOrdTimer.start(); collectBucket(sub, doc, bucketOrd); liveBucketCountUnderestimate = context.bigArrays().grow(liveBucketCountUnderestimate, owningBucketOrd + 1); int estimatedBucketCount = liveBucketCountUnderestimate.increment(owningBucketOrd, 1); - IndexedRounding r = increaseRoundingIfNeeded(owningBucketOrd, estimatedBucketCount, rounded, rounding); - newOrdTimer.stop(); - return r; + return increaseRoundingIfNeeded(owningBucketOrd, estimatedBucketCount, rounded, rounding); } /** @@ -388,18 +367,6 @@ public void collectDebugInfo(BiConsumer add) { add.accept("wasted_buckets_overestimate", wastedBucketsOverestimate); add.accept("next_rebucket_at", nextRebucketAt); add.accept("rebucket_count", rebucketCount); - add.accept("value_count_time", valueCountTimer.getApproximateTiming()); - add.accept("value_count_count", valueCountTimer.getCount()); - add.accept("next_value_time", nextValueTimer.getApproximateTiming()); - add.accept("next_value_count", nextValueTimer.getCount()); - add.accept("round_time", roundTimer.getApproximateTiming()); - add.accept("round_count", roundTimer.getCount()); - add.accept("ord_lookup_time", ordLookupTimer.getApproximateTiming()); - add.accept("ord_lookup_count", ordLookupTimer.getCount()); - add.accept("new_ord_time", newOrdTimer.getApproximateTiming()); - add.accept("new_ord_count", newOrdTimer.getCount()); - add.accept("old_ord_time", oldOrdTimer.getApproximateTiming()); - add.accept("old_ord_count", oldOrdTimer.getCount()); } @Override From 2e67acd7b1c7d9bbea748fef05346210a320aae1 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Thu, 4 Jun 2020 07:58:44 -0400 Subject: [PATCH 14/18] Revert "Rounding strat" This reverts commit ec12da27fa5f31b08ec938abf5469716f1e201ec. --- .../AutoDateHistogramAggregator.java | 239 +++++------------- 1 file changed, 70 insertions(+), 169 deletions(-) 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 bd9f8b5d667a6..c5671f64f6ea4 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 @@ -24,9 +24,7 @@ import org.apache.lucene.util.CollectionUtil; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Rounding; -import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; -import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.ByteArray; import org.elasticsearch.common.util.IntArray; import org.elasticsearch.common.util.LongArray; @@ -94,16 +92,34 @@ class AutoDateHistogramAggregator extends DeferableBucketAggregator { private final ValuesSource.Numeric valuesSource; private final DocValueFormat formatter; + private final RoundingInfo[] roundingInfos; + private final Function roundingPreparer; private final int targetBuckets; private final boolean collectsFromSingleBucket; - - private final PreparedRoundings roundings; - + /** + * An array of prepared roundings in the same order as + * {@link #roundingInfos}. The 0th entry is prepared initially, + * and other entries are null until first needed. + */ + private final Rounding.Prepared[] preparedRoundings; /** * Map from {@code owningBucketOrd, roundedDate} to {@code bucketOrdinal}. */ private LongKeyedBucketOrds bucketOrds; - + /** + * The index of the rounding that each {@code owningBucketOrd} is + * currently using. + *

+ * During collection we use overestimates for how much buckets are save + * by bumping to the next rounding index. So we end up bumping less + * aggressively than a "perfect" algorithm. That is fine because we + * correct the error when we merge the buckets together all the way + * up in {@link InternalAutoDateHistogram#reduceBucket}. In particular, + * on final reduce we bump the rounding until it we appropriately + * cover the date range across all of the results returned by all of + * the {@link AutoDateHistogramAggregator}s. + */ + private ByteArray roundingIndices; /** * The minimum key per {@code owningBucketOrd}. */ @@ -157,15 +173,18 @@ class AutoDateHistogramAggregator extends DeferableBucketAggregator { this.targetBuckets = numBuckets; this.valuesSource = (ValuesSource.Numeric) valuesSource; this.formatter = formatter; + this.roundingInfos = roundingInfos; + this.roundingPreparer = roundingPreparer; this.collectsFromSingleBucket = collectsFromSingleBucket; assert roundingInfos.length < 127 : "Rounding must fit in a signed byte"; - this.roundings = collectsFromSingleBucket - ? new SingleBucketPreparedRoundings(roundingPreparer, roundingInfos) - : new ManyBucketsPreparedRoundings(context.bigArrays(), roundingPreparer, roundingInfos); + roundingIndices = context.bigArrays().newByteArray(1, true); mins = context.bigArrays().newLongArray(1, false); mins.set(0, Long.MAX_VALUE); maxes = context.bigArrays().newLongArray(1, false); maxes.set(0, Long.MIN_VALUE); + preparedRoundings = new Rounding.Prepared[roundingInfos.length]; + // Prepare the first rounding because we know we'll need it. + preparedRoundings[0] = roundingPreparer.apply(roundingInfos[0].rounding); bucketOrds = LongKeyedBucketOrds.build(context.bigArrays(), collectsFromSingleBucket); liveBucketCountUnderestimate = context.bigArrays().newIntArray(1, true); } @@ -202,51 +221,41 @@ public void collect(int doc, long owningBucketOrd) throws IOException { int valuesCount = values.docValueCount(); long previousRounded = Long.MIN_VALUE; - IndexedRounding rounding = roundings.get(owningBucketOrd); + int roundingIdx = roundingIndexFor(owningBucketOrd); for (int i = 0; i < valuesCount; ++i) { long value = values.nextValue(); - long rounded = rounding.prepared.round(value); + long rounded = preparedRoundings[roundingIdx].round(value); assert rounded >= previousRounded; if (rounded == previousRounded) { continue; } - rounding = collectValue(sub, owningBucketOrd, doc, rounded, rounding); + roundingIdx = collectValue(sub, owningBucketOrd, roundingIdx, doc, rounded); previousRounded = rounded; } } } - private IndexedRounding collectValue( - LeafBucketCollector sub, - long owningBucketOrd, - int doc, - long rounded, - IndexedRounding rounding - ) throws IOException { + private int collectValue(LeafBucketCollector sub, long owningBucketOrd, int roundingIdx, int doc, long rounded) + throws IOException { long bucketOrd = bucketOrds.add(owningBucketOrd, rounded); if (bucketOrd < 0) { // already seen bucketOrd = -1 - bucketOrd; collectExistingBucket(sub, doc, bucketOrd); - return rounding; + return roundingIdx; } collectBucket(sub, doc, bucketOrd); liveBucketCountUnderestimate = context.bigArrays().grow(liveBucketCountUnderestimate, owningBucketOrd + 1); int estimatedBucketCount = liveBucketCountUnderestimate.increment(owningBucketOrd, 1); - return increaseRoundingIfNeeded(owningBucketOrd, estimatedBucketCount, rounded, rounding); + return increaseRoundingIfNeeded(owningBucketOrd, estimatedBucketCount, rounded, roundingIdx); } /** - * Check if we need increase the rounding of {@code owningBucketOrd} using - * estimated bucket counts and the interval between the min and max buckets. - * {@link #rebucket()} if the new estimated number of wasted buckets is too high. + * Increase the rounding of {@code owningBucketOrd} using + * estimated, bucket counts, {@link #rebucket() rebucketing} the all + * buckets if the estimated number of wasted buckets is too high. */ - private IndexedRounding increaseRoundingIfNeeded( - long owningBucketOrd, - int oldEstimatedBucketCount, - long newKey, - IndexedRounding oldRounding - ) { - if (false == oldRounding.canIncrease) { + private int increaseRoundingIfNeeded(long owningBucketOrd, int oldEstimatedBucketCount, long newKey, int oldRounding) { + if (oldRounding >= roundingInfos.length - 1) { return oldRounding; } if (mins.size() < owningBucketOrd + 1) { @@ -264,23 +273,23 @@ private IndexedRounding increaseRoundingIfNeeded( mins.set(owningBucketOrd, min); long max = Math.max(maxes.get(owningBucketOrd), newKey); maxes.set(owningBucketOrd, max); - if (oldEstimatedBucketCount <= targetBuckets * roundings.infos[oldRounding.index].getMaximumInnerInterval() - && max - min <= targetBuckets * roundings.infos[oldRounding.index].getMaximumRoughEstimateDurationMillis()) { + if (oldEstimatedBucketCount <= targetBuckets * roundingInfos[oldRounding].getMaximumInnerInterval() + && max - min <= targetBuckets * roundingInfos[oldRounding].getMaximumRoughEstimateDurationMillis()) { return oldRounding; } - long oldRoughDuration = roundings.infos[oldRounding.index].roughEstimateDurationMillis; - int newRounding = oldRounding.index; + long oldRoughDuration = roundingInfos[oldRounding].roughEstimateDurationMillis; + int newRounding = oldRounding; int newEstimatedBucketCount; do { newRounding++; - double ratio = (double) oldRoughDuration / (double) roundings.infos[newRounding].getRoughEstimateDurationMillis(); + double ratio = (double) oldRoughDuration / (double) roundingInfos[newRounding].getRoughEstimateDurationMillis(); newEstimatedBucketCount = (int) Math.ceil(oldEstimatedBucketCount * ratio); - } while (newRounding < roundings.infos.length - 1 && ( - newEstimatedBucketCount > targetBuckets * roundings.infos[newRounding].getMaximumInnerInterval() - || max - min > targetBuckets * roundings.infos[newRounding].getMaximumRoughEstimateDurationMillis())); - IndexedRounding newIndexedRounding = roundings.set(owningBucketOrd, newRounding); - mins.set(owningBucketOrd, newIndexedRounding.prepared.round(mins.get(owningBucketOrd))); - maxes.set(owningBucketOrd, newIndexedRounding.prepared.round(maxes.get(owningBucketOrd))); + } while (newRounding < roundingInfos.length - 1 && ( + newEstimatedBucketCount > targetBuckets * roundingInfos[newRounding].getMaximumInnerInterval() + || max - min > targetBuckets * roundingInfos[newRounding].getMaximumRoughEstimateDurationMillis())); + setRounding(owningBucketOrd, newRounding); + mins.set(owningBucketOrd, preparedRoundings[newRounding].round(mins.get(owningBucketOrd))); + maxes.set(owningBucketOrd, preparedRoundings[newRounding].round(maxes.get(owningBucketOrd))); wastedBucketsOverestimate += oldEstimatedBucketCount - newEstimatedBucketCount; if (wastedBucketsOverestimate > nextRebucketAt) { rebucket(); @@ -290,7 +299,7 @@ private IndexedRounding increaseRoundingIfNeeded( } else { liveBucketCountUnderestimate.set(owningBucketOrd, newEstimatedBucketCount); } - return newIndexedRounding; + return newRounding; } }; } @@ -302,10 +311,10 @@ private void rebucket() { bucketOrds = LongKeyedBucketOrds.build(context.bigArrays(), collectsFromSingleBucket); for (long owningBucketOrd = 0; owningBucketOrd <= oldOrds.maxOwningBucketOrd(); owningBucketOrd++) { LongKeyedBucketOrds.BucketOrdsEnum ordsEnum = oldOrds.ordsEnum(owningBucketOrd); - IndexedRounding preparedRounding = roundings.get(owningBucketOrd); + Rounding.Prepared preparedRounding = preparedRoundings[roundingIndexFor(owningBucketOrd)]; while (ordsEnum.next()) { long oldKey = ordsEnum.value(); - long newKey = preparedRounding.prepared.round(oldKey); + long newKey = preparedRounding.round(oldKey); long newBucketOrd = bucketOrds.add(owningBucketOrd, newKey); mergeMap[(int) ordsEnum.ord()] = newBucketOrd >= 0 ? newBucketOrd : -1 - newBucketOrd; } @@ -346,16 +355,24 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I CollectionUtil.introSort(buckets, BucketOrder.key(true).comparator()); // value source will be null for unmapped fields - InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundings.infos, - roundings.get(owningBucketOrd).index, buildEmptySubAggregations()); + InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundingInfos, + roundingIndexFor(owningBucketOrd), buildEmptySubAggregations()); return new InternalAutoDateHistogram(name, buckets, targetBuckets, emptyBucketInfo, formatter, metadata(), 1); }); } + private void setRounding(long owningBucketOrd, int newRounding) { + roundingIndices = context.bigArrays().grow(roundingIndices, owningBucketOrd + 1); + roundingIndices.set(owningBucketOrd, (byte) newRounding); + if (preparedRoundings[newRounding] == null) { + preparedRoundings[newRounding] = roundingPreparer.apply(roundingInfos[newRounding].rounding); + } + } + @Override public InternalAggregation buildEmptyAggregation() { - InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundings.infos, 0, + InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundingInfos, 0, buildEmptySubAggregations()); return new InternalAutoDateHistogram(name, Collections.emptyList(), targetBuckets, emptyBucketInfo, formatter, metadata(), 1); } @@ -369,128 +386,12 @@ public void collectDebugInfo(BiConsumer add) { add.accept("rebucket_count", rebucketCount); } - @Override - public void doClose() { - Releasables.close(bucketOrds, roundings, mins, maxes, liveBucketCountUnderestimate); - } - - protected abstract static class PreparedRoundings implements Releasable { - private final Function preparer; - private final RoundingInfo[] infos; - - public PreparedRoundings(Function preparer, RoundingInfo[] infos) { - this.preparer = preparer; - this.infos = infos; - } - - protected abstract IndexedRounding get(long owningBucketOrd); - - protected abstract IndexedRounding set(long owningBucketOrd, int newRounding); - - protected final IndexedRounding prepare(int index) { - return new IndexedRounding(index, index < infos.length - 1, preparer.apply(infos[index].rounding)); - } - } - - protected static class SingleBucketPreparedRoundings extends PreparedRoundings { - private IndexedRounding current; - - public SingleBucketPreparedRoundings(Function roundingPreparer, RoundingInfo[] roundingInfos) { - super(roundingPreparer, roundingInfos); - current = prepare(0); - } - - @Override - protected IndexedRounding get(long owningBucketOrd) { - assert owningBucketOrd == 0; - return current; - } - - @Override - protected IndexedRounding set(long owningBucketOrd, int newRounding) { - assert owningBucketOrd == 0; - current = prepare(newRounding); - return current; - } - - @Override - public void close() {} - } - - protected static class ManyBucketsPreparedRoundings extends PreparedRoundings { - private final BigArrays bigArrays; - /** - * An array of prepared roundings in the same order as - * {@link #infos}. The 0th entry is prepared initially, - * and other entries are null until first needed. - */ - private final IndexedRounding[] prepared; - /** - * The index of the rounding that each {@code owningBucketOrd} is - * currently using. - *

- * During collection we use overestimates for how much buckets are save - * by bumping to the next rounding index. So we end up bumping less - * aggressively than a "perfect" algorithm. That is fine because we - * correct the error when we merge the buckets together all the way - * up in {@link InternalAutoDateHistogram#reduceBucket}. In particular, - * on final reduce we bump the rounding until it we appropriately - * cover the date range across all of the results returned by all of - * the {@link AutoDateHistogramAggregator}s. - */ - private ByteArray indices; - - public ManyBucketsPreparedRoundings( - BigArrays bigArrays, - Function roundingPreparer, - RoundingInfo[] roundingInfos - ) { - super(roundingPreparer, roundingInfos); - this.bigArrays = bigArrays; - this.prepared = new IndexedRounding[roundingInfos.length]; - indices = bigArrays.newByteArray(1, true); - // Prepare the first rounding because we know we'll need it. - prepared[0] = prepare(0); - } - - @Override - protected IndexedRounding get(long owningBucketOrd) { - if (owningBucketOrd >= indices.size()) { - return prepared[0]; - } - /* - * This will never return null because we always prepare a rounding - * at the index when we set the index. - */ - return prepared[indices.get(owningBucketOrd)]; - } - - @Override - protected IndexedRounding set(long owningBucketOrd, int newRounding) { - indices = bigArrays.grow(indices, owningBucketOrd + 1); - indices.set(owningBucketOrd, (byte) newRounding); - if (prepared[newRounding] == null) { - prepared[newRounding] = prepare(newRounding); - } - return prepared[newRounding]; - } - - @Override - public void close() { - indices.close(); - } + private int roundingIndexFor(long owningBucketOrd) { + return owningBucketOrd < roundingIndices.size() ? roundingIndices.get(owningBucketOrd) : 0; } - private static class IndexedRounding { - private final int index; - private final boolean canIncrease; - private final Rounding.Prepared prepared; - - public IndexedRounding(int index, boolean canIncrease, Rounding.Prepared prepared) { - this.index = index; - this.canIncrease = canIncrease; - this.prepared = prepared; - } - + @Override + public void doClose() { + Releasables.close(bucketOrds, roundingIndices, mins, maxes, liveBucketCountUnderestimate); } } From 068f028e77f250f4e5dbeb58fbf7a3f164fffc8a Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Thu, 4 Jun 2020 08:49:48 -0400 Subject: [PATCH 15/18] How about this? --- .../330_auto_date_histogram.yml | 23 + .../AutoDateHistogramAggregator.java | 756 ++++++++++++------ .../AutoDateHistogramAggregatorFactory.java | 4 +- .../InternalAggregationProfileTree.java | 4 + 4 files changed, 527 insertions(+), 260 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/330_auto_date_histogram.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/330_auto_date_histogram.yml index 93285a802ab90..68b4d8538a6d2 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/330_auto_date_histogram.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/330_auto_date_histogram.yml @@ -4,6 +4,7 @@ setup: index: test body: settings: + number_of_shards: 1 number_of_replicas: 0 mappings: properties: @@ -75,3 +76,25 @@ setup: - match: { aggregations.histo.buckets.1.doc_count: 2 } - match: { aggregations.histo.buckets.1.v.value: 7 } - match: { aggregations.histo_avg_v.value: 5 } + +--- +"profile at top level": + - skip: + version: " - 7.99.99" + reason: Debug information added in 8.0.0 (to be backported to 7.9.0) + + - do: + search: + body: + profile: true + size: 0 + aggs: + histo: + auto_date_histogram: + field: date + buckets: 2 + + - match: { hits.total.value: 4 } + - length: { aggregations.histo.buckets: 2 } + - match: { profile.shards.0.aggregations.0.type: AutoDateHistogramAggregator.FromSingle } + - match: { profile.shards.0.aggregations.0.debug.surviving_buckets: 4 } 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 c5671f64f6ea4..4afa51e72c455 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 @@ -24,6 +24,7 @@ import org.apache.lucene.util.CollectionUtil; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Rounding; +import org.elasticsearch.common.Rounding.Prepared; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.util.ByteArray; import org.elasticsearch.common.util.IntArray; @@ -48,149 +49,96 @@ import java.util.Map; import java.util.function.BiConsumer; import java.util.function.Function; +import java.util.function.LongToIntFunction; /** * An aggregator for date values that attempts to return a specific number of * buckets, reconfiguring how it rounds dates to buckets on the fly as new * data arrives. *

- * Initially it uses the most fine grained rounding configuration possible but - * as more data arrives it uses two heuristics to shift to coarser and coarser - * rounding. The first heuristic is the number of buckets, specifically, - * when there are more buckets than can "fit" in the current rounding it shifts - * to the next rounding. Instead of redoing the rounding, it estimates the - * number of buckets that will "survive" at the new rounding and uses - * that as the initial value for the bucket count that it - * increments in order to trigger another promotion to another coarser - * rounding. This works fairly well at containing the number of buckets, but - * the estimate of the number of buckets will be wrong if the buckets are - * quite a spread out compared to the rounding. - *

- * The second heuristic it uses to trigger promotion to a coarser rounding is - * the distance between the min and max bucket. When that distance is greater - * than what the current rounding supports it promotes. This heuristic - * isn't good at limiting the number of buckets but is great when the buckets - * are spread out compared to the rounding. So it should complement the first - * heuristic. - *

- * When promoting a rounding we keep the old buckets around because it is - * expensive to call {@link MergingBucketsDeferringCollector#mergeBuckets}. - * In particular it is {@code O(number_of_hits_collected_so_far)}. So if we - * called it frequently we'd end up in {@code O(n^2)} territory. Bad news for - * aggregations! Instead, we keep a "budget" of buckets that we're ok - * "wasting". When we promote the rounding and our estimate of the number of - * "dead" buckets that have data but have yet to be merged into the buckets - * that are valid for the current rounding exceeds the budget then we rebucket - * the entire aggregation and double the budget. - *

- * Once we're done collecting and we know exactly which buckets we'll be - * returning we finally perform a "real", "perfect bucketing", - * rounding all of the keys for {@code owningBucketOrd} that we're going to - * collect and picking the rounding based on a real, accurate count and the - * min and max. + * This class is abstract because there is a simple implementation for when the + * aggregator only collects from a single bucket and a more complex + * implementation when it doesn't. This ain't great from a test coverage + * standpoint but the simpler implementation is between 7% and 15% faster + * when you can use it. This is an important aggregation and we need that + * performance. */ -class AutoDateHistogramAggregator extends DeferableBucketAggregator { +abstract class AutoDateHistogramAggregator extends DeferableBucketAggregator { + static AutoDateHistogramAggregator build( + String name, + AggregatorFactories factories, + int targetBuckets, + RoundingInfo[] roundingInfos, + Function roundingPreparer, + @Nullable ValuesSource valuesSource, + DocValueFormat formatter, + SearchContext aggregationContext, + Aggregator parent, + boolean collectsFromSingleBucket, + Map metadata + ) throws IOException { + return collectsFromSingleBucket + ? new FromSingle( + name, + factories, + targetBuckets, + roundingInfos, + roundingPreparer, + valuesSource, + formatter, + aggregationContext, + parent, + metadata + ) + : new FromMany( + name, + factories, + targetBuckets, + roundingInfos, + roundingPreparer, + valuesSource, + formatter, + aggregationContext, + parent, + metadata + ); + } + private final ValuesSource.Numeric valuesSource; private final DocValueFormat formatter; - private final RoundingInfo[] roundingInfos; private final Function roundingPreparer; - private final int targetBuckets; - private final boolean collectsFromSingleBucket; - /** - * An array of prepared roundings in the same order as - * {@link #roundingInfos}. The 0th entry is prepared initially, - * and other entries are null until first needed. - */ - private final Rounding.Prepared[] preparedRoundings; - /** - * Map from {@code owningBucketOrd, roundedDate} to {@code bucketOrdinal}. - */ - private LongKeyedBucketOrds bucketOrds; - /** - * The index of the rounding that each {@code owningBucketOrd} is - * currently using. - *

- * During collection we use overestimates for how much buckets are save - * by bumping to the next rounding index. So we end up bumping less - * aggressively than a "perfect" algorithm. That is fine because we - * correct the error when we merge the buckets together all the way - * up in {@link InternalAutoDateHistogram#reduceBucket}. In particular, - * on final reduce we bump the rounding until it we appropriately - * cover the date range across all of the results returned by all of - * the {@link AutoDateHistogramAggregator}s. - */ - private ByteArray roundingIndices; - /** - * The minimum key per {@code owningBucketOrd}. - */ - private LongArray mins; - /** - * The max key per {@code owningBucketOrd}. - */ - private LongArray maxes; /** * A reference to the collector so we can * {@link MergingBucketsDeferringCollector#mergeBuckets(long[])}. */ private MergingBucketsDeferringCollector deferringCollector; - /** - * An underestimate of the number of buckets that are "live" in the - * current rounding for each {@code owningBucketOrdinal}. - */ - private IntArray liveBucketCountUnderestimate; - /** - * An over estimate of the number of wasted buckets. When this gets - * too high we {@link #rebucket} which sets it to 0. - */ - private long wastedBucketsOverestimate = 0; - /** - * The next {@link #wastedBucketsOverestimate} that will trigger a - * {@link #rebucket() rebucketing}. - */ - private long nextRebucketAt = 1000; // TODO this could almost certainly start higher when asMultiBucketAggregator is gone - /** - * The number of times the aggregator had to {@link #rebucket()} the - * results. We keep this just to report to the profiler. - */ - private int rebucketCount = 0; - AutoDateHistogramAggregator( + protected final RoundingInfo[] roundingInfos; + protected final int targetBuckets; + + private AutoDateHistogramAggregator( String name, AggregatorFactories factories, - int numBuckets, + int targetBuckets, RoundingInfo[] roundingInfos, Function roundingPreparer, @Nullable ValuesSource valuesSource, DocValueFormat formatter, SearchContext aggregationContext, Aggregator parent, - boolean collectsFromSingleBucket, Map metadata - ) - throws IOException { + ) throws IOException { super(name, factories, aggregationContext, parent, metadata); - this.targetBuckets = numBuckets; + this.targetBuckets = targetBuckets; this.valuesSource = (ValuesSource.Numeric) valuesSource; this.formatter = formatter; this.roundingInfos = roundingInfos; this.roundingPreparer = roundingPreparer; - this.collectsFromSingleBucket = collectsFromSingleBucket; - assert roundingInfos.length < 127 : "Rounding must fit in a signed byte"; - roundingIndices = context.bigArrays().newByteArray(1, true); - mins = context.bigArrays().newLongArray(1, false); - mins.set(0, Long.MAX_VALUE); - maxes = context.bigArrays().newLongArray(1, false); - maxes.set(0, Long.MIN_VALUE); - preparedRoundings = new Rounding.Prepared[roundingInfos.length]; - // Prepare the first rounding because we know we'll need it. - preparedRoundings[0] = roundingPreparer.apply(roundingInfos[0].rounding); - bucketOrds = LongKeyedBucketOrds.build(context.bigArrays(), collectsFromSingleBucket); - liveBucketCountUnderestimate = context.bigArrays().newIntArray(1, true); } @Override - public ScoreMode scoreMode() { + public final ScoreMode scoreMode() { if (valuesSource != null && valuesSource.needsScores()) { return ScoreMode.COMPLETE; } @@ -198,200 +146,492 @@ public ScoreMode scoreMode() { } @Override - protected boolean shouldDefer(Aggregator aggregator) { + protected final boolean shouldDefer(Aggregator aggregator) { return true; } @Override - public DeferringBucketCollector getDeferringCollector() { + public final DeferringBucketCollector getDeferringCollector() { deferringCollector = new MergingBucketsDeferringCollector(context, descendsFromGlobalAggregator(parent())); return deferringCollector; } + protected abstract LeafBucketCollector getLeafCollector(SortedNumericDocValues values, LeafBucketCollector sub) throws IOException; + @Override - public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException { + public final LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException { if (valuesSource == null) { return LeafBucketCollector.NO_OP_COLLECTOR; } - SortedNumericDocValues values = valuesSource.longValues(ctx); - return new LeafBucketCollectorBase(sub, values) { - @Override - public void collect(int doc, long owningBucketOrd) throws IOException { - if (values.advanceExact(doc)) { + return getLeafCollector(valuesSource.longValues(ctx), sub); + } + + protected final InternalAggregation[] buildAggregations( + LongKeyedBucketOrds bucketOrds, + LongToIntFunction roundingIndexFor, + long[] owningBucketOrds + ) throws IOException { + return buildAggregationsForVariableBuckets( + owningBucketOrds, + bucketOrds, + (bucketValue, docCount, subAggregationResults) -> new InternalAutoDateHistogram.Bucket( + bucketValue, + docCount, + formatter, + subAggregationResults + ), + (owningBucketOrd, 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, + roundingIndexFor.applyAsInt(owningBucketOrd), + buildEmptySubAggregations() + ); + + return new InternalAutoDateHistogram(name, buckets, targetBuckets, emptyBucketInfo, formatter, metadata(), 1); + } + ); + } + + @Override + public final InternalAggregation buildEmptyAggregation() { + InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo( + roundingInfos, + 0, + buildEmptySubAggregations() + ); + return new InternalAutoDateHistogram(name, Collections.emptyList(), targetBuckets, emptyBucketInfo, formatter, metadata(), 1); + } + + protected final Rounding.Prepared prepareRounding(int index) { + return roundingPreparer.apply(roundingInfos[index].rounding); + } + + protected final void merge(long[] mergeMap, long newNumBuckets) { + mergeBuckets(mergeMap, newNumBuckets); + if (deferringCollector != null) { + deferringCollector.mergeBuckets(mergeMap); + } + } + + /** + * Initially it uses the most fine grained rounding configuration possible + * but as more data arrives it rebuckets the data until it "fits" in the + * aggregation rounding. Similar to {@link FromMany} this checks both the + * bucket count and range of the aggregation, but unlike + * {@linkplain FromMany} it keeps an accurate count of the buckets and it + * doesn't delay rebucketing. + *

+ * Rebucketing is roughly {@code O(number_of_hits_collected_so_far)} but we + * rebucket roughly {@code O(log number_of_hits_collected_so_far)} because + * the "shape" of the roundings is roughly + * logarithmically increasing. + */ + private static class FromSingle extends AutoDateHistogramAggregator { + private int roundingIdx; + private Rounding.Prepared preparedRounding; + private LongKeyedBucketOrds bucketOrds; + private long min = Long.MAX_VALUE; + private long max = Long.MIN_VALUE; + + public FromSingle( + String name, + AggregatorFactories factories, + int targetBuckets, + RoundingInfo[] roundingInfos, + Function roundingPreparer, + ValuesSource valuesSource, + DocValueFormat formatter, + SearchContext aggregationContext, + Aggregator parent, + Map metadata + ) throws IOException { + super( + name, + factories, + targetBuckets, + roundingInfos, + roundingPreparer, + valuesSource, + formatter, + aggregationContext, + parent, + metadata + ); + + preparedRounding = prepareRounding(0); + bucketOrds = LongKeyedBucketOrds.build(context.bigArrays(), true); + } + + @Override + protected LeafBucketCollector getLeafCollector(SortedNumericDocValues values, LeafBucketCollector sub) throws IOException { + return new LeafBucketCollectorBase(sub, values) { + @Override + public void collect(int doc, long owningBucketOrd) throws IOException { + assert owningBucketOrd == 0; + if (false == values.advanceExact(doc)) { + return; + } int valuesCount = values.docValueCount(); long previousRounded = Long.MIN_VALUE; - int roundingIdx = roundingIndexFor(owningBucketOrd); for (int i = 0; i < valuesCount; ++i) { long value = values.nextValue(); - long rounded = preparedRoundings[roundingIdx].round(value); + long rounded = preparedRounding.round(value); assert rounded >= previousRounded; if (rounded == previousRounded) { continue; } - roundingIdx = collectValue(sub, owningBucketOrd, roundingIdx, doc, rounded); + collectValue(doc, rounded); previousRounded = rounded; } } - } - private int collectValue(LeafBucketCollector sub, long owningBucketOrd, int roundingIdx, int doc, long rounded) - throws IOException { - long bucketOrd = bucketOrds.add(owningBucketOrd, rounded); - if (bucketOrd < 0) { // already seen - bucketOrd = -1 - bucketOrd; - collectExistingBucket(sub, doc, bucketOrd); - return roundingIdx; + private void collectValue(int doc, long rounded) throws IOException { + long bucketOrd = bucketOrds.add(0, rounded); + if (bucketOrd < 0) { // already seen + bucketOrd = -1 - bucketOrd; + collectExistingBucket(sub, doc, bucketOrd); + return; + } + collectBucket(sub, doc, bucketOrd); + increaseRoundingIfNeeded(rounded); } - collectBucket(sub, doc, bucketOrd); - liveBucketCountUnderestimate = context.bigArrays().grow(liveBucketCountUnderestimate, owningBucketOrd + 1); - int estimatedBucketCount = liveBucketCountUnderestimate.increment(owningBucketOrd, 1); - return increaseRoundingIfNeeded(owningBucketOrd, estimatedBucketCount, rounded, roundingIdx); - } - /** - * Increase the rounding of {@code owningBucketOrd} using - * estimated, bucket counts, {@link #rebucket() rebucketing} the all - * buckets if the estimated number of wasted buckets is too high. - */ - private int increaseRoundingIfNeeded(long owningBucketOrd, int oldEstimatedBucketCount, long newKey, int oldRounding) { - if (oldRounding >= roundingInfos.length - 1) { - return oldRounding; + private void increaseRoundingIfNeeded(long rounded) { + if (roundingIdx >= roundingInfos.length - 1) { + return; + } + min = Math.min(min, rounded); + max = Math.max(max, rounded); + if (bucketOrds.size() <= targetBuckets * roundingInfos[roundingIdx].getMaximumInnerInterval() + && max - min <= targetBuckets * roundingInfos[roundingIdx].getMaximumRoughEstimateDurationMillis()) { + return; + } + do { + try (LongKeyedBucketOrds oldOrds = bucketOrds) { + preparedRounding = prepareRounding(++roundingIdx); + long[] mergeMap = new long[Math.toIntExact(oldOrds.size())]; + bucketOrds = LongKeyedBucketOrds.build(context.bigArrays(), true); + LongKeyedBucketOrds.BucketOrdsEnum ordsEnum = oldOrds.ordsEnum(0); + while (ordsEnum.next()) { + long oldKey = ordsEnum.value(); + long newKey = preparedRounding.round(oldKey); + long newBucketOrd = bucketOrds.add(0, newKey); + mergeMap[(int) ordsEnum.ord()] = newBucketOrd >= 0 ? newBucketOrd : -1 - newBucketOrd; + } + merge(mergeMap, bucketOrds.size()); + } + } while (roundingIdx < roundingInfos.length - 1 + && (bucketOrds.size() > targetBuckets * roundingInfos[roundingIdx].getMaximumInnerInterval() + || max - min > targetBuckets * roundingInfos[roundingIdx].getMaximumRoughEstimateDurationMillis())); } - if (mins.size() < owningBucketOrd + 1) { - long oldSize = mins.size(); - mins = context.bigArrays().grow(mins, owningBucketOrd + 1); - mins.fill(oldSize, mins.size(), Long.MAX_VALUE); + }; + } + + @Override + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + return buildAggregations(bucketOrds, l -> roundingIdx, owningBucketOrds); + } + + @Override + public void collectDebugInfo(BiConsumer add) { + super.collectDebugInfo(add); + add.accept("surviving_buckets", bucketOrds.size()); + } + + @Override + protected void doClose() { + Releasables.close(bucketOrds); + } + } + + /** + * Initially it uses the most fine grained rounding configuration possible but + * as more data arrives it uses two heuristics to shift to coarser and coarser + * rounding. The first heuristic is the number of buckets, specifically, + * when there are more buckets than can "fit" in the current rounding it shifts + * to the next rounding. Instead of redoing the rounding, it estimates the + * number of buckets that will "survive" at the new rounding and uses + * that as the initial value for the bucket count that it + * increments in order to trigger another promotion to another coarser + * rounding. This works fairly well at containing the number of buckets, but + * the estimate of the number of buckets will be wrong if the buckets are + * quite a spread out compared to the rounding. + *

+ * The second heuristic it uses to trigger promotion to a coarser rounding is + * the distance between the min and max bucket. When that distance is greater + * than what the current rounding supports it promotes. This heuristic + * isn't good at limiting the number of buckets but is great when the buckets + * are spread out compared to the rounding. So it should complement the first + * heuristic. + *

+ * When promoting a rounding we keep the old buckets around because it is + * expensive to call {@link MergingBucketsDeferringCollector#mergeBuckets}. + * In particular it is {@code O(number_of_hits_collected_so_far)}. So if we + * called it frequently we'd end up in {@code O(n^2)} territory. Bad news for + * aggregations! Instead, we keep a "budget" of buckets that we're ok + * "wasting". When we promote the rounding and our estimate of the number of + * "dead" buckets that have data but have yet to be merged into the buckets + * that are valid for the current rounding exceeds the budget then we rebucket + * the entire aggregation and double the budget. + *

+ * Once we're done collecting and we know exactly which buckets we'll be + * returning we finally perform a "real", "perfect bucketing", + * rounding all of the keys for {@code owningBucketOrd} that we're going to + * collect and picking the rounding based on a real, accurate count and the + * min and max. + */ + private static class FromMany extends AutoDateHistogramAggregator { + /** + * An array of prepared roundings in the same order as + * {@link #roundingInfos}. The 0th entry is prepared initially, + * and other entries are null until first needed. + */ + private final Rounding.Prepared[] preparedRoundings; + /** + * Map from {@code owningBucketOrd, roundedDate} to {@code bucketOrdinal}. + */ + private LongKeyedBucketOrds bucketOrds; + /** + * The index of the rounding that each {@code owningBucketOrd} is + * currently using. + *

+ * During collection we use overestimates for how much buckets are save + * by bumping to the next rounding index. So we end up bumping less + * aggressively than a "perfect" algorithm. That is fine because we + * correct the error when we merge the buckets together all the way + * up in {@link InternalAutoDateHistogram#reduceBucket}. In particular, + * on final reduce we bump the rounding until it we appropriately + * cover the date range across all of the results returned by all of + * the {@link AutoDateHistogramAggregator}s. + */ + private ByteArray roundingIndices; + /** + * The minimum key per {@code owningBucketOrd}. + */ + private LongArray mins; + /** + * The max key per {@code owningBucketOrd}. + */ + private LongArray maxes; + + /** + * An underestimate of the number of buckets that are "live" in the + * current rounding for each {@code owningBucketOrdinal}. + */ + private IntArray liveBucketCountUnderestimate; + /** + * An over estimate of the number of wasted buckets. When this gets + * too high we {@link #rebucket} which sets it to 0. + */ + private long wastedBucketsOverestimate = 0; + /** + * The next {@link #wastedBucketsOverestimate} that will trigger a + * {@link #rebucket() rebucketing}. + */ + private long nextRebucketAt = 1000; // TODO this could almost certainly start higher when asMultiBucketAggregator is gone + /** + * The number of times the aggregator had to {@link #rebucket()} the + * results. We keep this just to report to the profiler. + */ + private int rebucketCount = 0; + + FromMany( + String name, + AggregatorFactories factories, + int targetBuckets, + RoundingInfo[] roundingInfos, + Function roundingPreparer, + @Nullable ValuesSource valuesSource, + DocValueFormat formatter, + SearchContext aggregationContext, + Aggregator parent, + Map metadata + ) throws IOException { + + super( + name, + factories, + targetBuckets, + roundingInfos, + roundingPreparer, + valuesSource, + formatter, + aggregationContext, + parent, + metadata + ); + assert roundingInfos.length < 127 : "Rounding must fit in a signed byte"; + roundingIndices = context.bigArrays().newByteArray(1, true); + mins = context.bigArrays().newLongArray(1, false); + mins.set(0, Long.MAX_VALUE); + maxes = context.bigArrays().newLongArray(1, false); + maxes.set(0, Long.MIN_VALUE); + preparedRoundings = new Rounding.Prepared[roundingInfos.length]; + // Prepare the first rounding because we know we'll need it. + preparedRoundings[0] = roundingPreparer.apply(roundingInfos[0].rounding); + bucketOrds = LongKeyedBucketOrds.build(context.bigArrays(), false); + liveBucketCountUnderestimate = context.bigArrays().newIntArray(1, true); + } + + @Override + protected LeafBucketCollector getLeafCollector(SortedNumericDocValues values, LeafBucketCollector sub) throws IOException { + return new LeafBucketCollectorBase(sub, values) { + @Override + public void collect(int doc, long owningBucketOrd) throws IOException { + if (false == values.advanceExact(doc)) { + return; + } + int valuesCount = values.docValueCount(); + + long previousRounded = Long.MIN_VALUE; + int roundingIdx = roundingIndexFor(owningBucketOrd); + for (int i = 0; i < valuesCount; ++i) { + long value = values.nextValue(); + long rounded = preparedRoundings[roundingIdx].round(value); + assert rounded >= previousRounded; + if (rounded == previousRounded) { + continue; + } + roundingIdx = collectValue(owningBucketOrd, roundingIdx, doc, rounded); + previousRounded = rounded; + } } - if (maxes.size() < owningBucketOrd + 1) { - long oldSize = maxes.size(); - maxes = context.bigArrays().grow(maxes, owningBucketOrd + 1); - maxes.fill(oldSize, maxes.size(), Long.MIN_VALUE); + + private int collectValue(long owningBucketOrd, int roundingIdx, int doc, long rounded) throws IOException { + long bucketOrd = bucketOrds.add(owningBucketOrd, rounded); + if (bucketOrd < 0) { // already seen + bucketOrd = -1 - bucketOrd; + collectExistingBucket(sub, doc, bucketOrd); + return roundingIdx; + } + collectBucket(sub, doc, bucketOrd); + liveBucketCountUnderestimate = context.bigArrays().grow(liveBucketCountUnderestimate, owningBucketOrd + 1); + int estimatedBucketCount = liveBucketCountUnderestimate.increment(owningBucketOrd, 1); + return increaseRoundingIfNeeded(owningBucketOrd, estimatedBucketCount, rounded, roundingIdx); } - long min = Math.min(mins.get(owningBucketOrd), newKey); - mins.set(owningBucketOrd, min); - long max = Math.max(maxes.get(owningBucketOrd), newKey); - maxes.set(owningBucketOrd, max); - if (oldEstimatedBucketCount <= targetBuckets * roundingInfos[oldRounding].getMaximumInnerInterval() + /** + * Increase the rounding of {@code owningBucketOrd} using + * estimated, bucket counts, {@link #rebucket() rebucketing} the all + * buckets if the estimated number of wasted buckets is too high. + */ + private int increaseRoundingIfNeeded(long owningBucketOrd, int oldEstimatedBucketCount, long newKey, int oldRounding) { + if (oldRounding >= roundingInfos.length - 1) { + return oldRounding; + } + if (mins.size() < owningBucketOrd + 1) { + long oldSize = mins.size(); + mins = context.bigArrays().grow(mins, owningBucketOrd + 1); + mins.fill(oldSize, mins.size(), Long.MAX_VALUE); + } + if (maxes.size() < owningBucketOrd + 1) { + long oldSize = maxes.size(); + maxes = context.bigArrays().grow(maxes, owningBucketOrd + 1); + maxes.fill(oldSize, maxes.size(), Long.MIN_VALUE); + } + + long min = Math.min(mins.get(owningBucketOrd), newKey); + mins.set(owningBucketOrd, min); + long max = Math.max(maxes.get(owningBucketOrd), newKey); + maxes.set(owningBucketOrd, max); + if (oldEstimatedBucketCount <= targetBuckets * roundingInfos[oldRounding].getMaximumInnerInterval() && max - min <= targetBuckets * roundingInfos[oldRounding].getMaximumRoughEstimateDurationMillis()) { - return oldRounding; - } - long oldRoughDuration = roundingInfos[oldRounding].roughEstimateDurationMillis; - int newRounding = oldRounding; - int newEstimatedBucketCount; - do { - newRounding++; - double ratio = (double) oldRoughDuration / (double) roundingInfos[newRounding].getRoughEstimateDurationMillis(); - newEstimatedBucketCount = (int) Math.ceil(oldEstimatedBucketCount * ratio); - } while (newRounding < roundingInfos.length - 1 && ( - newEstimatedBucketCount > targetBuckets * roundingInfos[newRounding].getMaximumInnerInterval() - || max - min > targetBuckets * roundingInfos[newRounding].getMaximumRoughEstimateDurationMillis())); - setRounding(owningBucketOrd, newRounding); - mins.set(owningBucketOrd, preparedRoundings[newRounding].round(mins.get(owningBucketOrd))); - maxes.set(owningBucketOrd, preparedRoundings[newRounding].round(maxes.get(owningBucketOrd))); - wastedBucketsOverestimate += oldEstimatedBucketCount - newEstimatedBucketCount; - if (wastedBucketsOverestimate > nextRebucketAt) { - rebucket(); - // Bump the threshold for the next rebucketing - wastedBucketsOverestimate = 0; - nextRebucketAt *= 2; - } else { - liveBucketCountUnderestimate.set(owningBucketOrd, newEstimatedBucketCount); + return oldRounding; + } + long oldRoughDuration = roundingInfos[oldRounding].roughEstimateDurationMillis; + int newRounding = oldRounding; + int newEstimatedBucketCount; + do { + newRounding++; + double ratio = (double) oldRoughDuration / (double) roundingInfos[newRounding].getRoughEstimateDurationMillis(); + newEstimatedBucketCount = (int) Math.ceil(oldEstimatedBucketCount * ratio); + } while (newRounding < roundingInfos.length - 1 + && (newEstimatedBucketCount > targetBuckets * roundingInfos[newRounding].getMaximumInnerInterval() + || max - min > targetBuckets * roundingInfos[newRounding].getMaximumRoughEstimateDurationMillis())); + setRounding(owningBucketOrd, newRounding); + mins.set(owningBucketOrd, preparedRoundings[newRounding].round(mins.get(owningBucketOrd))); + maxes.set(owningBucketOrd, preparedRoundings[newRounding].round(maxes.get(owningBucketOrd))); + wastedBucketsOverestimate += oldEstimatedBucketCount - newEstimatedBucketCount; + if (wastedBucketsOverestimate > nextRebucketAt) { + rebucket(); + // Bump the threshold for the next rebucketing + wastedBucketsOverestimate = 0; + nextRebucketAt *= 2; + } else { + liveBucketCountUnderestimate.set(owningBucketOrd, newEstimatedBucketCount); + } + return newRounding; } - return newRounding; - } - }; - } + }; + } - private void rebucket() { - rebucketCount++; - try (LongKeyedBucketOrds oldOrds = bucketOrds) { - long[] mergeMap = new long[Math.toIntExact(oldOrds.size())]; - bucketOrds = LongKeyedBucketOrds.build(context.bigArrays(), collectsFromSingleBucket); - for (long owningBucketOrd = 0; owningBucketOrd <= oldOrds.maxOwningBucketOrd(); owningBucketOrd++) { - LongKeyedBucketOrds.BucketOrdsEnum ordsEnum = oldOrds.ordsEnum(owningBucketOrd); - Rounding.Prepared preparedRounding = preparedRoundings[roundingIndexFor(owningBucketOrd)]; - while (ordsEnum.next()) { - long oldKey = ordsEnum.value(); - long newKey = preparedRounding.round(oldKey); - long newBucketOrd = bucketOrds.add(owningBucketOrd, newKey); - mergeMap[(int) ordsEnum.ord()] = newBucketOrd >= 0 ? newBucketOrd : -1 - newBucketOrd; + private void rebucket() { + rebucketCount++; + try (LongKeyedBucketOrds oldOrds = bucketOrds) { + long[] mergeMap = new long[Math.toIntExact(oldOrds.size())]; + bucketOrds = LongKeyedBucketOrds.build(context.bigArrays(), false); + for (long owningBucketOrd = 0; owningBucketOrd <= oldOrds.maxOwningBucketOrd(); owningBucketOrd++) { + LongKeyedBucketOrds.BucketOrdsEnum ordsEnum = oldOrds.ordsEnum(owningBucketOrd); + Rounding.Prepared preparedRounding = preparedRoundings[roundingIndexFor(owningBucketOrd)]; + while (ordsEnum.next()) { + long oldKey = ordsEnum.value(); + long newKey = preparedRounding.round(oldKey); + long newBucketOrd = bucketOrds.add(owningBucketOrd, newKey); + mergeMap[(int) ordsEnum.ord()] = newBucketOrd >= 0 ? newBucketOrd : -1 - newBucketOrd; + } + liveBucketCountUnderestimate = context.bigArrays().grow(liveBucketCountUnderestimate, owningBucketOrd + 1); + liveBucketCountUnderestimate.set(owningBucketOrd, Math.toIntExact(bucketOrds.bucketsInOrd(owningBucketOrd))); } - liveBucketCountUnderestimate = context.bigArrays().grow(liveBucketCountUnderestimate, owningBucketOrd + 1); - liveBucketCountUnderestimate.set(owningBucketOrd, Math.toIntExact(bucketOrds.bucketsInOrd(owningBucketOrd))); - } - mergeBuckets(mergeMap, bucketOrds.size()); - if (deferringCollector != null) { - deferringCollector.mergeBuckets(mergeMap); + merge(mergeMap, bucketOrds.size()); } } - } - @Override - public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { - double fractionWasted = ((double) wastedBucketsOverestimate) / ((double) bucketOrds.size()); - /* - * If there are many incorrect buckets then rebucket the aggregation - * so we have don't send them to the coordinating now. It can turn - * pretty much everything that we can throw at it into right answers - * but it'd be rude to send it huge results just so it can merge them. - * On the other hand, rebucketing is fairly slow. - * - * TODO it'd be faster if we could apply the merging on the fly as we - * replay the hits and build the buckets. How much faster is not clear, - * but it does have the advantage of only touching the buckets that we - * want to collect. - */ - if (fractionWasted > .2) { + @Override + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + /* + * Rebucket before building the aggregation to build as small as result + * as possible. + * + * TODO it'd be faster if we could apply the merging on the fly as we + * replay the hits and build the buckets. How much faster is not clear, + * but it does have the advantage of only touching the buckets that we + * want to collect. + */ rebucket(); + return buildAggregations(bucketOrds, this::roundingIndexFor, owningBucketOrds); } - return buildAggregationsForVariableBuckets(owningBucketOrds, bucketOrds, - (bucketValue, docCount, subAggregationResults) -> - new InternalAutoDateHistogram.Bucket(bucketValue, docCount, formatter, subAggregationResults), - (owningBucketOrd, 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, - roundingIndexFor(owningBucketOrd), buildEmptySubAggregations()); - - return new InternalAutoDateHistogram(name, buckets, targetBuckets, emptyBucketInfo, formatter, metadata(), 1); - }); - } - private void setRounding(long owningBucketOrd, int newRounding) { - roundingIndices = context.bigArrays().grow(roundingIndices, owningBucketOrd + 1); - roundingIndices.set(owningBucketOrd, (byte) newRounding); - if (preparedRoundings[newRounding] == null) { - preparedRoundings[newRounding] = roundingPreparer.apply(roundingInfos[newRounding].rounding); + @Override + public void collectDebugInfo(BiConsumer add) { + super.collectDebugInfo(add); + add.accept("surviving_buckets", bucketOrds.size()); + add.accept("wasted_buckets_overestimate", wastedBucketsOverestimate); + add.accept("next_rebucket_at", nextRebucketAt); + add.accept("rebucket_count", rebucketCount); } - } - @Override - public InternalAggregation buildEmptyAggregation() { - InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundingInfos, 0, - buildEmptySubAggregations()); - return new InternalAutoDateHistogram(name, Collections.emptyList(), targetBuckets, emptyBucketInfo, formatter, metadata(), 1); - } + private void setRounding(long owningBucketOrd, int newRounding) { + roundingIndices = context.bigArrays().grow(roundingIndices, owningBucketOrd + 1); + roundingIndices.set(owningBucketOrd, (byte) newRounding); + if (preparedRoundings[newRounding] == null) { + preparedRoundings[newRounding] = prepareRounding(newRounding); + } + } - @Override - public void collectDebugInfo(BiConsumer add) { - super.collectDebugInfo(add); - add.accept("surviving_buckets", bucketOrds.size()); - add.accept("wasted_buckets_overestimate", wastedBucketsOverestimate); - add.accept("next_rebucket_at", nextRebucketAt); - add.accept("rebucket_count", rebucketCount); - } + private int roundingIndexFor(long owningBucketOrd) { + return owningBucketOrd < roundingIndices.size() ? roundingIndices.get(owningBucketOrd) : 0; + } - private int roundingIndexFor(long owningBucketOrd) { - return owningBucketOrd < roundingIndices.size() ? roundingIndices.get(owningBucketOrd) : 0; + @Override + public void doClose() { + Releasables.close(bucketOrds, roundingIndices, mins, maxes, liveBucketCountUnderestimate); + } } - @Override - public void doClose() { - Releasables.close(bucketOrds, roundingIndices, mins, maxes, liveBucketCountUnderestimate); - } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java index 8e425a2d209ff..6cb65646f6dd1 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java @@ -44,7 +44,7 @@ public final class AutoDateHistogramAggregatorFactory extends ValuesSourceAggreg public static void registerAggregators(ValuesSourceRegistry.Builder builder) { builder.register(AutoDateHistogramAggregationBuilder.NAME, List.of(CoreValuesSourceType.DATE, CoreValuesSourceType.NUMERIC, CoreValuesSourceType.BOOLEAN), - (AutoDateHistogramAggregatorSupplier) AutoDateHistogramAggregator::new); + (AutoDateHistogramAggregatorSupplier) AutoDateHistogramAggregator::build); } private final int numBuckets; @@ -85,7 +85,7 @@ protected Aggregator doCreateInternal(ValuesSource valuesSource, protected Aggregator createUnmapped(SearchContext searchContext, Aggregator parent, Map metadata) throws IOException { - return new AutoDateHistogramAggregator(name, factories, numBuckets, roundingInfos, Rounding::prepareForUnknown, null, + return AutoDateHistogramAggregator.build(name, factories, numBuckets, roundingInfos, Rounding::prepareForUnknown, null, config.format(), searchContext, parent, false, metadata); } } diff --git a/server/src/main/java/org/elasticsearch/search/profile/aggregation/InternalAggregationProfileTree.java b/server/src/main/java/org/elasticsearch/search/profile/aggregation/InternalAggregationProfileTree.java index 4670d73cdc724..38b9fec2f4704 100644 --- a/server/src/main/java/org/elasticsearch/search/profile/aggregation/InternalAggregationProfileTree.java +++ b/server/src/main/java/org/elasticsearch/search/profile/aggregation/InternalAggregationProfileTree.java @@ -41,6 +41,10 @@ protected String getTypeFromElement(Aggregator element) { if (element instanceof MultiBucketAggregatorWrapper) { return ((MultiBucketAggregatorWrapper) element).getWrappedClass().getSimpleName(); } + Class enclosing = element.getClass().getEnclosingClass(); + if (enclosing != null) { + return enclosing.getSimpleName() + "." + element.getClass().getSimpleName(); + } return element.getClass().getSimpleName(); } From d3efe68b02d332fbc341e3d78dddd9d5c49e4466 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Thu, 4 Jun 2020 11:19:35 -0400 Subject: [PATCH 16/18] Monomorphic --- .../histogram/AutoDateHistogramAggregator.java | 12 ++++++------ .../bucket/terms/LongKeyedBucketOrds.java | 8 ++++---- 2 files changed, 10 insertions(+), 10 deletions(-) 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 4afa51e72c455..7dc7e5a3bf87a 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 @@ -234,7 +234,7 @@ protected final void merge(long[] mergeMap, long newNumBuckets) { private static class FromSingle extends AutoDateHistogramAggregator { private int roundingIdx; private Rounding.Prepared preparedRounding; - private LongKeyedBucketOrds bucketOrds; + private LongKeyedBucketOrds.FromSingle bucketOrds; private long min = Long.MAX_VALUE; private long max = Long.MIN_VALUE; @@ -264,7 +264,7 @@ public FromSingle( ); preparedRounding = prepareRounding(0); - bucketOrds = LongKeyedBucketOrds.build(context.bigArrays(), true); + bucketOrds = new LongKeyedBucketOrds.FromSingle(context.bigArrays()); } @Override @@ -316,7 +316,7 @@ private void increaseRoundingIfNeeded(long rounded) { try (LongKeyedBucketOrds oldOrds = bucketOrds) { preparedRounding = prepareRounding(++roundingIdx); long[] mergeMap = new long[Math.toIntExact(oldOrds.size())]; - bucketOrds = LongKeyedBucketOrds.build(context.bigArrays(), true); + bucketOrds = new LongKeyedBucketOrds.FromSingle(context.bigArrays()); LongKeyedBucketOrds.BucketOrdsEnum ordsEnum = oldOrds.ordsEnum(0); while (ordsEnum.next()) { long oldKey = ordsEnum.value(); @@ -396,7 +396,7 @@ private static class FromMany extends AutoDateHistogramAggregator { /** * Map from {@code owningBucketOrd, roundedDate} to {@code bucketOrdinal}. */ - private LongKeyedBucketOrds bucketOrds; + private LongKeyedBucketOrds.FromMany bucketOrds; /** * The index of the rounding that each {@code owningBucketOrd} is * currently using. @@ -475,7 +475,7 @@ private static class FromMany extends AutoDateHistogramAggregator { preparedRoundings = new Rounding.Prepared[roundingInfos.length]; // Prepare the first rounding because we know we'll need it. preparedRoundings[0] = roundingPreparer.apply(roundingInfos[0].rounding); - bucketOrds = LongKeyedBucketOrds.build(context.bigArrays(), false); + bucketOrds = new LongKeyedBucketOrds.FromMany(context.bigArrays()); liveBucketCountUnderestimate = context.bigArrays().newIntArray(1, true); } @@ -575,7 +575,7 @@ private void rebucket() { rebucketCount++; try (LongKeyedBucketOrds oldOrds = bucketOrds) { long[] mergeMap = new long[Math.toIntExact(oldOrds.size())]; - bucketOrds = LongKeyedBucketOrds.build(context.bigArrays(), false); + bucketOrds = new LongKeyedBucketOrds.FromMany(context.bigArrays()); for (long owningBucketOrd = 0; owningBucketOrd <= oldOrds.maxOwningBucketOrd(); owningBucketOrd++) { LongKeyedBucketOrds.BucketOrdsEnum ordsEnum = oldOrds.ordsEnum(owningBucketOrd); Rounding.Prepared preparedRounding = preparedRoundings[roundingIndexFor(owningBucketOrd)]; 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 da3525085e937..3b2e73f739ff7 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 @@ -107,10 +107,10 @@ public interface BucketOrdsEnum { /** * Implementation that only works if it is collecting from a single bucket. */ - private static class FromSingle extends LongKeyedBucketOrds { + public static class FromSingle extends LongKeyedBucketOrds { private final LongHash ords; - FromSingle(BigArrays bigArrays) { + public FromSingle(BigArrays bigArrays) { ords = new LongHash(1, bigArrays); } @@ -174,7 +174,7 @@ public void close() { /** * Implementation that works properly when collecting from many buckets. */ - private static class FromMany extends LongKeyedBucketOrds { + public 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; @@ -194,7 +194,7 @@ public void close() { private ObjectArray owningOrdToBuckets; private long lastGlobalOrd = -1; - FromMany(BigArrays bigArrays) { + public FromMany(BigArrays bigArrays) { this.bigArrays = bigArrays; owningOrdToBuckets = bigArrays.newObjectArray(1); } From 54f8c6612746057cfa80e4211016e4247a8eba6a Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Thu, 4 Jun 2020 15:37:42 -0400 Subject: [PATCH 17/18] Javadoc --- .../histogram/AutoDateHistogramAggregator.java | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) 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 7dc7e5a3bf87a..daf37bc502187 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 @@ -234,6 +234,16 @@ protected final void merge(long[] mergeMap, long newNumBuckets) { private static class FromSingle extends AutoDateHistogramAggregator { private int roundingIdx; private Rounding.Prepared preparedRounding; + /** + * Map from value to bucket ordinals. + *

+ * It is important that this is the exact subtype of + * {@link LongKeyedBucketOrds} so that the JVM can make a monomorphic + * call to {@link LongKeyedBucketOrds#add(long, long)} in the tight + * inner loop of {@link LeafBucketCollector#collect(int, long)}. You'd + * think that it wouldn't matter, but its seriously 7%-15% performance + * difference for the aggregation. Yikes. + */ private LongKeyedBucketOrds.FromSingle bucketOrds; private long min = Long.MAX_VALUE; private long max = Long.MIN_VALUE; @@ -394,7 +404,12 @@ private static class FromMany extends AutoDateHistogramAggregator { */ private final Rounding.Prepared[] preparedRoundings; /** - * Map from {@code owningBucketOrd, roundedDate} to {@code bucketOrdinal}. + * Map from value to bucket ordinals. + *

+ * It is important that this is the exact subtype of + * {@link LongKeyedBucketOrds} so that the JVM can make a monomorphic + * call to {@link LongKeyedBucketOrds#add(long, long)} in the tight + * inner loop of {@link LeafBucketCollector#collect(int, long)}. */ private LongKeyedBucketOrds.FromMany bucketOrds; /** From 0dcbd2597d1ef73bc12fef0b340ed9d06c35da98 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Thu, 4 Jun 2020 18:04:48 -0400 Subject: [PATCH 18/18] Oh checkstyle, you loveable scamp --- .../bucket/histogram/AutoDateHistogramAggregator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 daf37bc502187..2bd91835429c8 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 @@ -248,7 +248,7 @@ private static class FromSingle extends AutoDateHistogramAggregator { private long min = Long.MAX_VALUE; private long max = Long.MIN_VALUE; - public FromSingle( + FromSingle( String name, AggregatorFactories factories, int targetBuckets,