From 14cd560178bd9f0ba8994dc5fd38677aedbb16a5 Mon Sep 17 00:00:00 2001 From: weizijun Date: Thu, 7 Apr 2022 12:22:51 +0800 Subject: [PATCH 01/53] add time_series_aggregation --- .../TimeSeriesAggregationsIT.java | 34 +- .../elasticsearch/search/SearchModule.java | 12 + .../aggregations/AggregationBuilders.java | 4 + .../AggregationExecutionContext.java | 12 +- .../search/aggregations/BucketCollector.java | 2 +- .../timeseries/TimeSeriesIndexSearcher.java | 4 +- .../InternalTimeSeriesAggregation.java | 496 +++++++++++++++++ .../ParsedTimeSeriesAggregation.java | 206 +++++++ .../aggregation/TimeSeriesAggregation.java | 58 ++ ...meSeriesAggregationAggregationBuilder.java | 510 ++++++++++++++++++ ...meSeriesAggregationAggregationFactory.java | 152 ++++++ .../TimeSeriesAggregationAggregator.java | 412 ++++++++++++++ ...meSeriesAggregationAggregatorSupplier.java | 45 ++ .../aggregation/TimeSeriesAggregations.java | 59 ++ .../AggregatorBucketFunction.java | 24 + .../bucketfunction/AvgBucketFunction.java | 69 +++ .../bucketfunction/MaxBucketFunction.java | 56 ++ .../bucketfunction/MinBucketFunction.java | 56 ++ .../NoAggregatorBucketFunction.java | 39 ++ .../bucketfunction/SumBucketFunction.java | 56 ++ .../bucketfunction/TSIDBucketFunction.java | 72 +++ .../ValueCountBucketFunction.java | 56 ++ .../function/AggregatorFunction.java | 22 + .../aggregation/function/AvgFunction.java | 41 ++ .../aggregation/function/LastFunction.java | 43 ++ .../aggregation/function/MaxFunction.java | 34 ++ .../aggregation/function/MinFunction.java | 34 ++ .../aggregation/function/SumFunction.java | 34 ++ .../function/ValueCountFunction.java | 38 ++ .../timeseries/aggregation/internal/Last.java | 100 ++++ .../internal/TSIDInternalAggregation.java | 132 +++++ .../aggregations/AggregationsTests.java | 4 +- .../InternalTimeSeriesAggregationTests.java | 116 ++++ ...iesAggregationAggregationBuilderTests.java | 61 +++ .../TimeSeriesAggregationAggregatorTests.java | 15 + .../test/InternalAggregationTestCase.java | 3 + .../AggregateMetricMapperPlugin.java | 3 +- ...MetricTimeSeriesAggregationAggregator.java | 159 ++++++ .../AggregateMetricsAggregatorsRegistrar.java | 10 + 39 files changed, 3276 insertions(+), 7 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregation.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/ParsedTimeSeriesAggregation.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregation.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregations.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AggregatorBucketFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AvgBucketFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MaxBucketFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MinBucketFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/NoAggregatorBucketFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/SumBucketFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/ValueCountBucketFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AggregatorFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AvgFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/MaxFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/MinFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/SumFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregationTests.java create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilderTests.java create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorTests.java create mode 100644 x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java index 97f13449350ec..866fb2e1f470a 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java @@ -50,6 +50,7 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.sum; import static org.elasticsearch.search.aggregations.AggregationBuilders.terms; import static org.elasticsearch.search.aggregations.AggregationBuilders.timeSeries; +import static org.elasticsearch.search.aggregations.AggregationBuilders.timeSeriesAggregation; import static org.elasticsearch.search.aggregations.AggregationBuilders.topHits; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse; @@ -71,7 +72,7 @@ public class TimeSeriesAggregationsIT extends ESIntegTestCase { @Override public void setupSuiteScopeCluster() throws Exception { - int numberOfIndices = randomIntBetween(1, 3); + int numberOfIndices = 1; //randomIntBetween(1, 3); numberOfDimensions = randomIntBetween(1, 5); numberOfMetrics = randomIntBetween(1, 10); String[] routingKeys = randomSubsetOf( @@ -170,6 +171,37 @@ public void setupSuiteScopeCluster() throws Exception { indexRandom(true, false, docs); } + public void testTimeSeriesAggregations() { + SearchResponse response = client().prepareSearch("index") + .setSize(0) + .addAggregation( + timeSeriesAggregation("by_ts").field("metric_0") + .group(List.of("dim_0")) + .aggregator("sum") + .interval(DateHistogramInterval.minutes(10)) + .downsampleFunction("avg") + .size(1) + ) + .get(); + assertSearchResponse(response); + if (response != null) { + System.out.println(response); + return; + } + Aggregations aggregations = response.getAggregations(); + assertNotNull(aggregations); + TimeSeries timeSeries = aggregations.get("by_ts"); + assertThat( + timeSeries.getBuckets().stream().map(MultiBucketsAggregation.Bucket::getKey).collect(Collectors.toSet()), + equalTo(data.keySet()) + ); + for (TimeSeries.Bucket bucket : timeSeries.getBuckets()) { + @SuppressWarnings("unchecked") + Map key = (Map) bucket.getKey(); + assertThat((long) data.get(key).size(), equalTo(bucket.getDocCount())); + } + } + public void testStandAloneTimeSeriesAgg() { SearchResponse response = client().prepareSearch("index").setSize(0).addAggregation(timeSeries("by_ts")).get(); assertSearchResponse(response); diff --git a/server/src/main/java/org/elasticsearch/search/SearchModule.java b/server/src/main/java/org/elasticsearch/search/SearchModule.java index eafda206f74e4..13765834f0611 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchModule.java +++ b/server/src/main/java/org/elasticsearch/search/SearchModule.java @@ -215,6 +215,8 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.timeseries.InternalTimeSeries; import org.elasticsearch.search.aggregations.timeseries.TimeSeriesAggregationBuilder; +import org.elasticsearch.search.aggregations.timeseries.aggregation.InternalTimeSeriesAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregationAggregationBuilder; import org.elasticsearch.search.fetch.FetchPhase; import org.elasticsearch.search.fetch.FetchSubPhase; import org.elasticsearch.search.fetch.subphase.ExplainPhase; @@ -664,6 +666,16 @@ private ValuesSourceRegistry registerAggregations(List plugins) { ).addResultReader(InternalTimeSeries::new), builder ); + + registerAggregation( + new AggregationSpec( + TimeSeriesAggregationAggregationBuilder.NAME, + TimeSeriesAggregationAggregationBuilder::new, + TimeSeriesAggregationAggregationBuilder.PARSER + ).addResultReader(InternalTimeSeriesAggregation::new) + .setAggregatorRegistrar(TimeSeriesAggregationAggregationBuilder::registerAggregators), + builder + ); } if (RestApiVersion.minimumSupported() == RestApiVersion.V_7) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilders.java b/server/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilders.java index f29debd458d8d..872a962d37a96 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilders.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilders.java @@ -79,6 +79,7 @@ import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.WeightedAvgAggregationBuilder; import org.elasticsearch.search.aggregations.timeseries.TimeSeriesAggregationBuilder; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregationAggregationBuilder; import java.util.List; import java.util.Map; @@ -373,4 +374,7 @@ public static TimeSeriesAggregationBuilder timeSeries(String name) { return new TimeSeriesAggregationBuilder(name); } + public static TimeSeriesAggregationAggregationBuilder timeSeriesAggregation(String name) { + return new TimeSeriesAggregationAggregationBuilder(name); + } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/AggregationExecutionContext.java b/server/src/main/java/org/elasticsearch/search/aggregations/AggregationExecutionContext.java index eaf82541afbcd..30c6a9ef0d92d 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/AggregationExecutionContext.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/AggregationExecutionContext.java @@ -23,11 +23,17 @@ public class AggregationExecutionContext { private final CheckedSupplier tsidProvider; + private CheckedSupplier timestampProvider; private final LeafReaderContext leafReaderContext; - public AggregationExecutionContext(LeafReaderContext leafReaderContext, CheckedSupplier tsidProvider) { + public AggregationExecutionContext( + LeafReaderContext leafReaderContext, + CheckedSupplier tsidProvider, + CheckedSupplier timestampProvider + ) { this.leafReaderContext = leafReaderContext; this.tsidProvider = tsidProvider; + this.timestampProvider = timestampProvider; } public LeafReaderContext getLeafReaderContext() { @@ -37,4 +43,8 @@ public LeafReaderContext getLeafReaderContext() { public BytesRef getTsid() throws IOException { return tsidProvider != null ? tsidProvider.get() : null; } + + public Long getTimestamp() throws IOException { + return timestampProvider.get(); + } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/BucketCollector.java b/server/src/main/java/org/elasticsearch/search/aggregations/BucketCollector.java index 60907169df4ca..c9c72b3b8ac8e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/BucketCollector.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/BucketCollector.java @@ -44,7 +44,7 @@ public ScoreMode scoreMode() { // TODO: will remove it in a follow up PR @Override public final LeafBucketCollector getLeafCollector(LeafReaderContext ctx) throws IOException { - return getLeafCollector(new AggregationExecutionContext(ctx, null)); + return getLeafCollector(new AggregationExecutionContext(ctx, null, null)); } public abstract LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx) throws IOException; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/TimeSeriesIndexSearcher.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/TimeSeriesIndexSearcher.java index 347a33238b751..1f345c397d755 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/TimeSeriesIndexSearcher.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/TimeSeriesIndexSearcher.java @@ -71,7 +71,7 @@ public void search(Query query, BucketCollector bucketCollector) throws IOExcept // this is needed to trigger actions in some bucketCollectors that bypass the normal iteration logic // for example, global aggregator triggers a separate iterator that ignores the query but still needs // to know all leaves - bucketCollector.getLeafCollector(new AggregationExecutionContext(leaf, null)); + bucketCollector.getLeafCollector(new AggregationExecutionContext(leaf, null, null)); } } @@ -164,7 +164,7 @@ private static class LeafWalker { long timestamp; LeafWalker(LeafReaderContext context, Scorer scorer, BucketCollector bucketCollector, LeafReaderContext leaf) throws IOException { - AggregationExecutionContext aggCtx = new AggregationExecutionContext(leaf, scratch::get); + AggregationExecutionContext aggCtx = new AggregationExecutionContext(leaf, scratch::get, () -> timestamp); this.collector = bucketCollector.getLeafCollector(aggCtx); liveDocs = context.reader().getLiveDocs(); this.collector.setScorer(scorer); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregation.java new file mode 100644 index 0000000000000..23476e435bc80 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregation.java @@ -0,0 +1,496 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.search.aggregations.AggregationReduceContext; +import org.elasticsearch.search.aggregations.BucketOrder; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.aggregations.InternalOrder; +import org.elasticsearch.search.aggregations.KeyComparable; +import org.elasticsearch.search.aggregations.bucket.terms.AbstractInternalTerms; +import org.elasticsearch.search.aggregations.timeseries.aggregation.InternalTimeSeriesAggregation.InternalBucket; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TSIDInternalAggregation; +import org.elasticsearch.xcontent.ObjectParser; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Objects; +import java.util.TreeMap; + +import static org.elasticsearch.search.aggregations.ParsedMultiBucketAggregation.declareMultiBucketAggregationFields; +import static org.elasticsearch.search.aggregations.bucket.terms.InternalTerms.DOC_COUNT_ERROR_UPPER_BOUND_FIELD_NAME; +import static org.elasticsearch.search.aggregations.bucket.terms.InternalTerms.SUM_OF_OTHER_DOC_COUNTS; + +public class InternalTimeSeriesAggregation extends AbstractInternalTerms + implements + TimeSeriesAggregation { + + private static final ObjectParser PARSER = new ObjectParser<>( + ParsedTimeSeriesAggregation.class.getSimpleName(), + true, + ParsedTimeSeriesAggregation::new + ); + static { + declareMultiBucketAggregationFields( + PARSER, + parser -> ParsedTimeSeriesAggregation.ParsedBucket.fromXContent(parser, false), + parser -> ParsedTimeSeriesAggregation.ParsedBucket.fromXContent(parser, true) + ); + } + + public static class InternalBucket extends AbstractTermsBucket implements TimeSeriesAggregation.Bucket, KeyComparable { + protected long bucketOrd; + protected final boolean keyed; + protected final Map key; + protected long docCount; + protected Map timeBucketValues; + protected boolean showDocCountError; + protected long docCountError; + protected InternalAggregations aggregations; + + public InternalBucket( + Map key, + long docCount, + Map timeBucketValues, + InternalAggregations aggregations, + boolean keyed, + boolean showDocCountError, + long docCountError + ) { + this.key = key; + this.docCount = docCount; + this.aggregations = aggregations; + this.keyed = keyed; + this.timeBucketValues = timeBucketValues; + this.showDocCountError = showDocCountError; + this.docCountError = docCountError; + } + + /** + * Read from a stream. + */ + public InternalBucket(StreamInput in, boolean keyed, boolean showDocCountError) throws IOException { + this.keyed = keyed; + key = in.readOrderedMap(StreamInput::readString, StreamInput::readGenericValue); + docCount = in.readVLong(); + timeBucketValues = in.readOrderedMap(StreamInput::readLong, stream -> stream.readNamedWriteable(InternalAggregation.class)); + this.showDocCountError = showDocCountError; + docCountError = -1; + if (showDocCountError) { + docCountError = in.readLong(); + } + aggregations = InternalAggregations.readFrom(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeMap(key, StreamOutput::writeString, StreamOutput::writeGenericValue); + out.writeVLong(docCount); + out.writeMap(timeBucketValues, StreamOutput::writeLong, StreamOutput::writeNamedWriteable); + if (showDocCountError) { + out.writeLong(docCountError); + } + aggregations.writeTo(out); + } + + @Override + public Map getKey() { + return key; + } + + @Override + public String getKeyAsString() { + return key.toString(); + } + + @Override + public long getDocCount() { + return docCount; + } + + @Override + public InternalAggregations getAggregations() { + return aggregations; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + if (keyed) { + builder.startObject(getKeyAsString()); + } else { + builder.startObject(); + } + if (getKeyAsString() != null) { + builder.field(CommonFields.KEY_AS_STRING.getPreferredName(), getKeyAsString()); + } + builder.field(CommonFields.KEY.getPreferredName(), key); + builder.field(CommonFields.DOC_COUNT.getPreferredName(), docCount); + if (showDocCountError) { + builder.field(DOC_COUNT_ERROR_UPPER_BOUND_FIELD_NAME.getPreferredName(), docCountError); + } + builder.startObject(CommonFields.VALUES.getPreferredName()); + for (Entry entry : timeBucketValues.entrySet()) { + builder.startObject(String.valueOf(entry.getKey())); + entry.getValue().doXContentBody(builder, params); + builder.endObject(); + } + builder.endObject(); + aggregations.toXContentInternal(builder, params); + builder.endObject(); + return builder; + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + if (other == null || getClass() != other.getClass()) { + return false; + } + InternalBucket that = (InternalBucket) other; + return Objects.equals(key, that.key) + && Objects.equals(keyed, that.keyed) + && Objects.equals(docCount, that.docCount) + && Objects.equals(timeBucketValues, that.timeBucketValues); + } + + @Override + public int hashCode() { + return Objects.hash(getClass(), key, keyed, docCount, timeBucketValues); + } + + @Override + public int compareKey(InternalBucket other) { + return key.toString().compareTo(other.key.toString()); + } + + @Override + protected void setDocCountError(long docCountError) { + this.docCountError = docCountError; + } + + @Override + protected void updateDocCountError(long docCountErrorDiff) { + this.docCountError += docCountErrorDiff; + } + + @Override + protected boolean getShowDocCountError() { + return showDocCountError; + } + + @Override + protected long getDocCountError() { + if (showDocCountError == false) { + throw new IllegalStateException("show_terms_doc_count_error is false"); + } + return docCountError; + } + } + + protected final BucketOrder reduceOrder; + protected final BucketOrder order; + protected final int requiredSize; + protected final long minDocCount; + protected final int shardSize; + protected final boolean showTermDocCountError; + protected final long otherDocCount; + private final List buckets; + protected long docCountError; + private final boolean keyed; + // bucketMap gets lazily initialized from buckets in getBucketByKey() + private transient Map bucketMap; + + public InternalTimeSeriesAggregation( + String name, + BucketOrder reduceOrder, + BucketOrder order, + int requiredSize, + long minDocCount, + int shardSize, + boolean showTermDocCountError, + long otherDocCount, + List buckets, + boolean keyed, + long docCountError, + Map metadata + ) { + super(name, metadata); + this.buckets = buckets; + this.keyed = keyed; + this.reduceOrder = reduceOrder; + this.order = order; + this.requiredSize = requiredSize; + this.minDocCount = minDocCount; + this.shardSize = shardSize; + this.showTermDocCountError = showTermDocCountError; + this.otherDocCount = otherDocCount; + this.docCountError = docCountError; + } + + /** + * Read from a stream. + */ + public InternalTimeSeriesAggregation(StreamInput in) throws IOException { + super(in); + keyed = in.readBoolean(); + reduceOrder = InternalOrder.Streams.readOrder(in); + order = InternalOrder.Streams.readOrder(in); + requiredSize = readSize(in); + minDocCount = in.readVLong(); + docCountError = in.readZLong(); + shardSize = readSize(in); + showTermDocCountError = in.readBoolean(); + otherDocCount = in.readVLong(); + int size = in.readVInt(); + List buckets = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + buckets.add(new InternalBucket(in, keyed, showTermDocCountError)); + } + this.buckets = buckets; + this.bucketMap = null; + } + + @Override + public String getWriteableName() { + return TimeSeriesAggregationAggregationBuilder.NAME; + } + + @Override + public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + builder.field(DOC_COUNT_ERROR_UPPER_BOUND_FIELD_NAME.getPreferredName(), docCountError); + builder.field(SUM_OF_OTHER_DOC_COUNTS.getPreferredName(), otherDocCount); + if (keyed) { + builder.startObject(CommonFields.BUCKETS.getPreferredName()); + } else { + builder.startArray(CommonFields.BUCKETS.getPreferredName()); + } + for (InternalBucket bucket : buckets) { + bucket.toXContent(builder, params); + } + if (keyed) { + builder.endObject(); + } else { + builder.endArray(); + } + return builder; + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + out.writeBoolean(keyed); + reduceOrder.writeTo(out); + order.writeTo(out); + writeSize(requiredSize, out); + out.writeVLong(minDocCount); + out.writeZLong(docCountError); + writeSize(shardSize, out); + out.writeBoolean(showTermDocCountError); + out.writeVLong(otherDocCount); + out.writeVInt(buckets.size()); + for (InternalBucket bucket : buckets) { + bucket.writeTo(out); + } + } + + @Override + public InternalTimeSeriesAggregation create(List buckets) { + return new InternalTimeSeriesAggregation( + name, + reduceOrder, + order, + requiredSize, + minDocCount, + shardSize, + showTermDocCountError, + otherDocCount, + buckets, + keyed, + docCountError, + getMetadata() + ); + } + + @Override + protected InternalTimeSeriesAggregation create( + String name, + List buckets, + BucketOrder reduceOrder, + long docCountError, + long otherDocCount + ) { + return new InternalTimeSeriesAggregation( + name, + reduceOrder, + order, + requiredSize, + minDocCount, + shardSize, + showTermDocCountError, + otherDocCount, + buckets, + keyed, + docCountError, + getMetadata() + ); + } + + @Override + public InternalBucket createBucket(InternalAggregations aggregations, InternalBucket prototype) { + return new InternalBucket( + prototype.key, + prototype.docCount, + prototype.timeBucketValues, + prototype.aggregations, + prototype.keyed, + prototype.showDocCountError, + docCountError + ); + } + + @Override + protected InternalBucket createBucket(long docCount, InternalAggregations aggs, long docCountError, InternalBucket prototype) { + return new InternalBucket( + prototype.key, + prototype.docCount, + prototype.timeBucketValues, + prototype.aggregations, + prototype.keyed, + prototype.showDocCountError, + docCountError + ); + } + + @Override + protected int getShardSize() { + return shardSize; + } + + @Override + protected BucketOrder getReduceOrder() { + return reduceOrder; + } + + @Override + protected BucketOrder getOrder() { + return order; + } + + @Override + protected long getSumOfOtherDocCounts() { + return otherDocCount; + } + + @Override + protected Long getDocCountError() { + return docCountError; + } + + @Override + protected void setDocCountError(long docCountError) { + this.docCountError = docCountError; + } + + @Override + protected long getMinDocCount() { + return minDocCount; + } + + @Override + protected int getRequiredSize() { + return requiredSize; + } + + @Override + public InternalBucket reduceBucket(List buckets, AggregationReduceContext context) { + InternalBucket reduced = null; + List aggregationsList = new ArrayList<>(buckets.size()); + Map tsidInternalAggs = new HashMap<>(); + Map timeBucketValues = new TreeMap<>(); + long docCountError = 0; + for (InternalBucket bucket : buckets) { + if (docCountError != -1) { + if (bucket.getShowDocCountError() == false || bucket.getDocCountError() == -1) { + docCountError = -1; + } else { + docCountError += bucket.getDocCountError(); + } + } + if (reduced == null) { + reduced = new InternalBucket( + bucket.key, + bucket.docCount, + bucket.timeBucketValues, + bucket.aggregations, + bucket.keyed, + bucket.showDocCountError, + docCountError + ); + timeBucketValues.putAll(bucket.timeBucketValues); + } else { + reduced.docCount += bucket.docCount; + for (Entry entry : bucket.timeBucketValues.entrySet()) { + Long timestamp = entry.getKey(); + InternalAggregation value = entry.getValue(); + InternalAggregation aggregation = timeBucketValues.get(timestamp); + if (aggregation == null) { + timeBucketValues.put(timestamp, value); + if (value instanceof TSIDInternalAggregation) { + tsidInternalAggs.put(timestamp, value); + } + } else { + InternalAggregation result = aggregation.reduce(List.of(value), context); + if (result instanceof TSIDInternalAggregation) { + tsidInternalAggs.put(timestamp, result); + } else { + tsidInternalAggs.remove(timestamp); + } + timeBucketValues.put(timestamp, result); + } + } + } + aggregationsList.add(bucket.aggregations); + } + if (context.isFinalReduce()) { + tsidInternalAggs.forEach((timestamp, value) -> { timeBucketValues.put(timestamp, value.reduce(List.of(value), context)); }); + } + reduced.timeBucketValues = timeBucketValues; + reduced.docCountError = docCountError; + if (reduced.docCountError == -1) { + reduced.showDocCountError = false; + } + reduced.aggregations = InternalAggregations.reduce(aggregationsList, context); + return reduced; + } + + @Override + public List getBuckets() { + return buckets; + } + + @Override + public InternalBucket getBucketByKey(String key) { + if (bucketMap == null) { + bucketMap = new HashMap<>(buckets.size()); + for (InternalBucket bucket : buckets) { + bucketMap.put(bucket.getKeyAsString(), bucket); + } + } + return bucketMap.get(key); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/ParsedTimeSeriesAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/ParsedTimeSeriesAggregation.java new file mode 100644 index 0000000000000..5edc35e84ebd7 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/ParsedTimeSeriesAggregation.java @@ -0,0 +1,206 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation; + +import org.elasticsearch.common.xcontent.XContentParserUtils; +import org.elasticsearch.search.aggregations.Aggregation; +import org.elasticsearch.search.aggregations.Aggregations; +import org.elasticsearch.search.aggregations.ParsedMultiBucketAggregation; +import org.elasticsearch.search.aggregations.bucket.terms.InternalTerms; +import org.elasticsearch.xcontent.ObjectParser; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentParser; +import org.elasticsearch.xcontent.XContentParser.Token; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.TreeMap; + +import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken; +import static org.elasticsearch.search.aggregations.bucket.terms.InternalTerms.DOC_COUNT_ERROR_UPPER_BOUND_FIELD_NAME; +import static org.elasticsearch.search.aggregations.bucket.terms.InternalTerms.SUM_OF_OTHER_DOC_COUNTS; + +public class ParsedTimeSeriesAggregation extends ParsedMultiBucketAggregation + implements + TimeSeriesAggregation { + + private transient Map bucketMap; + + @Override + public String getType() { + return TimeSeriesAggregationAggregationBuilder.NAME; + } + + @Override + public List getBuckets() { + return buckets; + } + + @Override + public TimeSeriesAggregation.Bucket getBucketByKey(String key) { + if (bucketMap == null) { + bucketMap = new HashMap<>(buckets.size()); + for (ParsedBucket bucket : buckets) { + bucketMap.put(bucket.getKeyAsString(), bucket); + } + } + return bucketMap.get(key); + } + + private static final ObjectParser PARSER = new ObjectParser<>( + ParsedTimeSeriesAggregation.class.getSimpleName(), + true, + ParsedTimeSeriesAggregation::new + ); + static { + declareMultiBucketAggregationFields( + PARSER, + parser -> ParsedTimeSeriesAggregation.ParsedBucket.fromXContent(parser, false), + parser -> ParsedTimeSeriesAggregation.ParsedBucket.fromXContent(parser, true) + ); + PARSER.declareLong((parsedTerms, value) -> parsedTerms.docCountErrorUpperBound = value, + DOC_COUNT_ERROR_UPPER_BOUND_FIELD_NAME); + PARSER.declareLong((parsedTerms, value) -> parsedTerms.sumOtherDocCount = value, + SUM_OF_OTHER_DOC_COUNTS); + } + + protected long docCountErrorUpperBound; + protected long sumOtherDocCount; + + public long getDocCountError() { + return docCountErrorUpperBound; + } + + public long getSumOfOtherDocCounts() { + return sumOtherDocCount; + } + + @Override + protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + builder.field(DOC_COUNT_ERROR_UPPER_BOUND_FIELD_NAME.getPreferredName(), getDocCountError()); + builder.field(SUM_OF_OTHER_DOC_COUNTS.getPreferredName(), getSumOfOtherDocCounts()); + if (keyed) { + builder.startObject(CommonFields.BUCKETS.getPreferredName()); + } else { + builder.startArray(CommonFields.BUCKETS.getPreferredName()); + } + for (TimeSeriesAggregation.Bucket bucket : getBuckets()) { + bucket.toXContent(builder, params); + } + if (keyed) { + builder.endObject(); + } else { + builder.endArray(); + } + return builder; + } + + public static ParsedTimeSeriesAggregation fromXContent(XContentParser parser, String name) throws IOException { + ParsedTimeSeriesAggregation aggregation = PARSER.parse(parser, null); + aggregation.setName(name); + return aggregation; + } + + static class ParsedBucket extends ParsedMultiBucketAggregation.ParsedBucket implements TimeSeriesAggregation.Bucket { + + boolean showDocCountError = false; + protected long docCountError; + private Map key; + private Map values; + + @Override + public Object getKey() { + return key; + } + + @Override + public String getKeyAsString() { + return key.toString(); + } + + @Override + protected XContentBuilder keyToXContent(XContentBuilder builder) throws IOException { + return builder.field(CommonFields.KEY.getPreferredName(), getKey()); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + if (isKeyed()) { + // Subclasses can override the getKeyAsString method to handle specific cases like + // keyed bucket with RAW doc value format where the key_as_string field is not printed + // out but we still need to have a string version of the key to use as the bucket's name. + builder.startObject(getKeyAsString()); + } else { + builder.startObject(); + } + if (getKeyAsString() != null) { + builder.field(CommonFields.KEY_AS_STRING.getPreferredName(), getKeyAsString()); + } + keyToXContent(builder); + builder.field(CommonFields.DOC_COUNT.getPreferredName(), getDocCount()); + if (showDocCountError) { + builder.field(DOC_COUNT_ERROR_UPPER_BOUND_FIELD_NAME.getPreferredName(), docCountError); + } + builder.field(CommonFields.VALUES.getPreferredName(), values); + getAggregations().toXContentInternal(builder, params); + builder.endObject(); + return builder; + } + + static ParsedTimeSeriesAggregation.ParsedBucket fromXContent(XContentParser parser, boolean keyed) + throws IOException { + ParsedTimeSeriesAggregation.ParsedBucket bucket = new ParsedBucket(); + bucket.setKeyed(keyed); + String currentFieldName = parser.currentName(); + Token token = parser.currentToken(); + + if (keyed) { + ensureExpectedToken(XContentParser.Token.FIELD_NAME, token, parser); + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.nextToken(), parser); + } + + List aggregations = new ArrayList(); + while ((token = parser.nextToken()) != Token.END_OBJECT) { + if (token == Token.FIELD_NAME) { + currentFieldName = parser.currentName(); + } else if (token.isValue()) { + if (CommonFields.KEY_AS_STRING.getPreferredName().equals(currentFieldName)) { + bucket.setKeyAsString(parser.text()); + } else if (CommonFields.DOC_COUNT.getPreferredName().equals(currentFieldName)) { + bucket.setDocCount(parser.longValue()); + } else if (InternalTerms.DOC_COUNT_ERROR_UPPER_BOUND_FIELD_NAME.getPreferredName().equals( + currentFieldName)) { + bucket.docCountError = parser.longValue(); + bucket.showDocCountError = true; + } else if (CommonFields.VALUES.getPreferredName().equals(currentFieldName)) { + bucket.values = new LinkedHashMap<>(parser.mapOrdered()); + } + } else if (token == Token.START_OBJECT) { + if (CommonFields.KEY.getPreferredName().equals(currentFieldName)) { + bucket.key = new TreeMap<>(parser.map()); + } else if (CommonFields.VALUES.getPreferredName().equals(currentFieldName)) { + bucket.values = new LinkedHashMap<>(parser.mapOrdered()); + } else { + Objects.requireNonNull(aggregations); + XContentParserUtils.parseTypedKeysObject(parser, "#", Aggregation.class, aggregations::add); + } + } + } + + bucket.setAggregations(new Aggregations(aggregations)); + return bucket; + } + } + +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregation.java new file mode 100644 index 0000000000000..b88b61d8bcb16 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregation.java @@ -0,0 +1,58 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation; + +import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation; + +import java.util.List; + +public interface TimeSeriesAggregation extends MultiBucketsAggregation { + + /** + * A bucket associated with a specific time series (identified by its key) + */ + interface Bucket extends MultiBucketsAggregation.Bucket {} + + /** + * The buckets created by this aggregation. + */ + @Override + List getBuckets(); + + Bucket getBucketByKey(String key); + + class Downsample { + long range; + Function function; + } + + enum Function { + count, + sum, + min, + max, + avg, + last; + + public static Function resolve(String name) { + return Function.valueOf(name); + } + } + + enum Aggregator { + sum, + min, + max, + avg; + + public static Aggregator resolve(String name) { + return Aggregator.valueOf(name); + } + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java new file mode 100644 index 0000000000000..efcf3e7e8ebab --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java @@ -0,0 +1,510 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation; + +import org.elasticsearch.Version; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.search.aggregations.AggregationBuilder; +import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; +import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.BucketOrder; +import org.elasticsearch.search.aggregations.InternalOrder; +import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; +import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator; +import org.elasticsearch.search.aggregations.support.AggregationContext; +import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; +import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; +import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; +import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry.RegistryKey; +import org.elasticsearch.search.aggregations.support.ValuesSourceType; +import org.elasticsearch.xcontent.ObjectParser; +import org.elasticsearch.xcontent.ParseField; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class TimeSeriesAggregationAggregationBuilder extends ValuesSourceAggregationBuilder { + public static final String NAME = "time_series_aggregation"; + public static final ParseField KEYED_FIELD = new ParseField("keyed"); + public static final ParseField GROUP_FIELD = new ParseField("group"); + public static final ParseField WITHOUT_FIELD = new ParseField("without"); + public static final ParseField INTERVAL_FIELD = new ParseField("interval"); + public static final ParseField OFFSET_FIELD = new ParseField("offset"); + public static final ParseField AGGREGATOR_FIELD = new ParseField("aggregator"); + public static final ParseField DOWNSAMPLE_FIELD = new ParseField("downsample"); + public static final ParseField DOWNSAMPLE_RANGE_FIELD = new ParseField("range"); + public static final ParseField DOWNSAMPLE_FUNCTION_FIELD = new ParseField("function"); + public static final ParseField ORDER_FIELD = new ParseField("order"); + public static final ParseField SIZE_FIELD = new ParseField("size"); + public static final ParseField SHARD_SIZE_FIELD = new ParseField("shard_size"); + public static final ParseField MIN_DOC_COUNT_FIELD = new ParseField("min_doc_count"); + public static final ParseField SHARD_MIN_DOC_COUNT_FIELD = new ParseField("shard_min_doc_count"); + + static final TermsAggregator.BucketCountThresholds DEFAULT_BUCKET_COUNT_THRESHOLDS = new TermsAggregator.BucketCountThresholds( + 1, + 0, + 10, + -1 + ); + + public static final RegistryKey REGISTRY_KEY = new RegistryKey<>( + NAME, + TimeSeriesAggregationAggregatorSupplier.class + ); + + public static final ObjectParser PARSER = ObjectParser.fromBuilder( + NAME, + TimeSeriesAggregationAggregationBuilder::new + ); + + private boolean keyed; + private List group; + private List without; + private DateHistogramInterval interval; + private DateHistogramInterval offset; + private String aggregator; + private DateHistogramInterval downsampleRange; + private String downsampleFunction; + private TermsAggregator.BucketCountThresholds bucketCountThresholds = new TermsAggregator.BucketCountThresholds( + DEFAULT_BUCKET_COUNT_THRESHOLDS + ); + private BucketOrder order = BucketOrder.compound(BucketOrder.key(true)); + + static { + ValuesSourceAggregationBuilder.declareFields(PARSER, false, true, false); + + PARSER.declareField( + TimeSeriesAggregationAggregationBuilder::interval, + p -> new DateHistogramInterval(p.text()), + INTERVAL_FIELD, + ObjectParser.ValueType.STRING + ); + + PARSER.declareField( + TimeSeriesAggregationAggregationBuilder::offset, + p -> new DateHistogramInterval(p.text()), + OFFSET_FIELD, + ObjectParser.ValueType.STRING + ); + + PARSER.declareBoolean(TimeSeriesAggregationAggregationBuilder::setKeyed, KEYED_FIELD); + PARSER.declareStringArray(TimeSeriesAggregationAggregationBuilder::group, GROUP_FIELD); + PARSER.declareStringArray(TimeSeriesAggregationAggregationBuilder::without, WITHOUT_FIELD); + PARSER.declareString(TimeSeriesAggregationAggregationBuilder::aggregator, AGGREGATOR_FIELD); + PARSER.declareField( + TimeSeriesAggregationAggregationBuilder::downsampleRange, + p -> new DateHistogramInterval(p.text()), + DOWNSAMPLE_RANGE_FIELD, + ObjectParser.ValueType.STRING + ); + PARSER.declareString(TimeSeriesAggregationAggregationBuilder::downsampleFunction, DOWNSAMPLE_FUNCTION_FIELD); + PARSER.declareObjectArray( + TimeSeriesAggregationAggregationBuilder::order, + (p, c) -> InternalOrder.Parser.parseOrderParam(p), + ORDER_FIELD + ); + PARSER.declareInt(TimeSeriesAggregationAggregationBuilder::size, SIZE_FIELD); + PARSER.declareInt(TimeSeriesAggregationAggregationBuilder::shardSize, SHARD_SIZE_FIELD); + PARSER.declareInt(TimeSeriesAggregationAggregationBuilder::minDocCount, MIN_DOC_COUNT_FIELD); + PARSER.declareInt(TimeSeriesAggregationAggregationBuilder::shardMinDocCount, SHARD_MIN_DOC_COUNT_FIELD); + } + + public TimeSeriesAggregationAggregationBuilder(String name) { + super(name); + } + + protected TimeSeriesAggregationAggregationBuilder( + TimeSeriesAggregationAggregationBuilder clone, + Builder factoriesBuilder, + Map metadata + ) { + super(clone, factoriesBuilder, metadata); + this.keyed = clone.keyed; + this.group = clone.group; + this.without = clone.group; + this.interval = clone.interval; + this.offset = clone.offset; + this.aggregator = clone.aggregator; + this.downsampleRange = clone.downsampleRange; + this.downsampleFunction = clone.downsampleFunction; + this.bucketCountThresholds = clone.bucketCountThresholds; + } + + public TimeSeriesAggregationAggregationBuilder(StreamInput in) throws IOException { + super(in); + keyed = in.readBoolean(); + group = in.readOptionalStringList(); + without = in.readOptionalStringList(); + interval = in.readOptionalWriteable(DateHistogramInterval::new); + offset = in.readOptionalWriteable(DateHistogramInterval::new); + aggregator = in.readOptionalString(); + downsampleRange = in.readOptionalWriteable(DateHistogramInterval::new); + downsampleFunction = in.readOptionalString(); + order = InternalOrder.Streams.readOrder(in); + } + + @Override + protected void innerWriteTo(StreamOutput out) throws IOException { + out.writeBoolean(keyed); + out.writeOptionalStringCollection(group); + out.writeOptionalStringCollection(without); + out.writeOptionalWriteable(interval); + out.writeOptionalWriteable(offset); + out.writeOptionalString(aggregator); + out.writeOptionalWriteable(downsampleRange); + out.writeOptionalString(downsampleFunction); + order.writeTo(out); + } + + @Override + protected RegistryKey getRegistryKey() { + return REGISTRY_KEY; + } + + public static void registerAggregators(ValuesSourceRegistry.Builder builder) { + TimeSeriesAggregationAggregationFactory.registerAggregators(builder); + } + + @Override + protected ValuesSourceType defaultValueSourceType() { + return CoreValuesSourceType.NUMERIC; + } + + @Override + protected ValuesSourceAggregatorFactory innerBuild( + AggregationContext context, + ValuesSourceConfig config, + AggregatorFactory parent, + Builder subFactoriesBuilder + ) throws IOException { + TimeSeriesAggregationAggregatorSupplier aggregatorSupplier = context.getValuesSourceRegistry().getAggregator(REGISTRY_KEY, config); + return new TimeSeriesAggregationAggregationFactory( + name, + keyed, + group, + without, + interval, + offset, + aggregator, + downsampleRange, + downsampleFunction, + bucketCountThresholds, + order, + config, + context, + parent, + subFactoriesBuilder, + metadata, + aggregatorSupplier + ); + } + + @Override + protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + builder.field(KEYED_FIELD.getPreferredName(), keyed); + if (group != null) { + builder.field(GROUP_FIELD.getPreferredName(), group); + } + if (without != null) { + builder.field(WITHOUT_FIELD.getPreferredName(), without); + } + if (interval != null) { + builder.field(INTERVAL_FIELD.getPreferredName(), interval.toString()); + } + if (offset != null) { + builder.field(OFFSET_FIELD.getPreferredName(), offset.toString()); + } + if (aggregator != null) { + builder.field(AGGREGATOR_FIELD.getPreferredName(), aggregator); + } + if (downsampleRange != null) { + builder.field(DOWNSAMPLE_RANGE_FIELD.getPreferredName(), downsampleRange.toString()); + } + if (downsampleFunction != null) { + builder.field(DOWNSAMPLE_FUNCTION_FIELD.getPreferredName(), downsampleFunction); + } + + // if (downsampleRange != null || downsampleFunction != null) { + // Map downsample = new HashMap<>(); + // if (downsampleRange != null) { + // downsample.put(DOWNSAMPLE_RANGE_FIELD.getPreferredName(), downsampleRange.toString()); + // } + // if (downsampleFunction != null) { + // downsample.put(DOWNSAMPLE_FUNCTION_FIELD.getPreferredName(), downsampleFunction); + // } + // builder.field(DOWNSAMPLE_FIELD.getPreferredName(), downsample); + // } + + bucketCountThresholds.toXContent(builder, params); + builder.field(ORDER_FIELD.getPreferredName()); + order.toXContent(builder, params); + return builder; + } + + @Override + protected AggregationBuilder shallowCopy(Builder factoriesBuilder, Map metadata) { + return new TimeSeriesAggregationAggregationBuilder(this, factoriesBuilder, metadata); + } + + @Override + public BucketCardinality bucketCardinality() { + return BucketCardinality.MANY; + } + + @Override + public String getType() { + return NAME; + } + + @Override + public boolean isInSortOrderExecutionRequired() { + return true; + } + + public boolean isKeyed() { + return keyed; + } + + public void setKeyed(boolean keyed) { + this.keyed = keyed; + } + + public List getGroup() { + return group; + } + + public TimeSeriesAggregationAggregationBuilder group(List group) { + this.group = group; + return this; + } + + public List getWithout() { + return without; + } + + public TimeSeriesAggregationAggregationBuilder without(List without) { + this.without = without; + return this; + } + + public DateHistogramInterval getInterval() { + return interval; + } + + public TimeSeriesAggregationAggregationBuilder interval(DateHistogramInterval interval) { + this.interval = interval; + return this; + } + + public DateHistogramInterval getOffset() { + return offset; + } + + public TimeSeriesAggregationAggregationBuilder offset(DateHistogramInterval offset) { + this.offset = offset; + return this; + } + + public String getAggregator() { + return aggregator; + } + + public TimeSeriesAggregationAggregationBuilder aggregator(String aggregator) { + this.aggregator = aggregator; + return this; + } + + public DateHistogramInterval getDownsampleRange() { + return downsampleRange; + } + + public TimeSeriesAggregationAggregationBuilder downsampleRange(DateHistogramInterval downsampleRange) { + this.downsampleRange = downsampleRange; + return this; + } + + public String getDownsampleFunction() { + return downsampleFunction; + } + + public TimeSeriesAggregationAggregationBuilder downsampleFunction(String downsampleFunction) { + this.downsampleFunction = downsampleFunction; + return this; + } + + /** + * Sets the size - indicating how many term buckets should be returned + * (defaults to 10) + */ + public TimeSeriesAggregationAggregationBuilder size(int size) { + if (size <= 0) { + throw new IllegalArgumentException("[size] must be greater than 0. Found [" + size + "] in [" + name + "]"); + } + bucketCountThresholds.setRequiredSize(size); + return this; + } + + /** + * Returns the number of term buckets currently configured + */ + public int size() { + return bucketCountThresholds.getRequiredSize(); + } + + /** + * Sets the shard_size - indicating the number of term buckets each shard + * will return to the coordinating node (the node that coordinates the + * search execution). The higher the shard size is, the more accurate the + * results are. + */ + public TimeSeriesAggregationAggregationBuilder shardSize(int shardSize) { + if (shardSize <= 0) { + throw new IllegalArgumentException("[shardSize] must be greater than 0. Found [" + shardSize + "] in [" + name + "]"); + } + bucketCountThresholds.setShardSize(shardSize); + return this; + } + + /** + * Returns the number of term buckets per shard that are currently configured + */ + public int shardSize() { + return bucketCountThresholds.getShardSize(); + } + + /** + * Set the minimum document count terms should have in order to appear in + * the response. + */ + public TimeSeriesAggregationAggregationBuilder minDocCount(long minDocCount) { + if (minDocCount < 1) { + throw new IllegalArgumentException( + "[minDocCount] must be greater than or equal to 1. Found [" + minDocCount + "] in [" + name + "]" + ); + } + bucketCountThresholds.setMinDocCount(minDocCount); + return this; + } + + /** + * Returns the minimum document count required per term + */ + public long minDocCount() { + return bucketCountThresholds.getMinDocCount(); + } + + /** + * Set the minimum document count terms should have on the shard in order to + * appear in the response. + */ + public TimeSeriesAggregationAggregationBuilder shardMinDocCount(long shardMinDocCount) { + if (shardMinDocCount < 0) { + throw new IllegalArgumentException( + "[shardMinDocCount] must be greater than or equal to 0. Found [" + shardMinDocCount + "] in [" + name + "]" + ); + } + bucketCountThresholds.setShardMinDocCount(shardMinDocCount); + return this; + } + + /** + * Returns the minimum document count required per term, per shard + */ + public long shardMinDocCount() { + return bucketCountThresholds.getShardMinDocCount(); + } + + /** + * Set a new order on this builder and return the builder so that calls + * can be chained. A tie-breaker may be added to avoid non-deterministic ordering. + */ + public TimeSeriesAggregationAggregationBuilder order(BucketOrder order) { + if (order == null) { + throw new IllegalArgumentException("[order] must not be null: [" + name + "]"); + } + if (order instanceof InternalOrder.CompoundOrder || InternalOrder.isKeyOrder(order)) { + this.order = order; // if order already contains a tie-breaker we are good to go + } else { // otherwise add a tie-breaker by using a compound order + this.order = BucketOrder.compound(order); + } + return this; + } + + /** + * Sets the order in which the buckets will be returned. A tie-breaker may be added to avoid non-deterministic + * ordering. + */ + public TimeSeriesAggregationAggregationBuilder order(List orders) { + if (orders == null) { + throw new IllegalArgumentException("[orders] must not be null: [" + name + "]"); + } + // if the list only contains one order use that to avoid inconsistent xcontent + order(orders.size() > 1 ? BucketOrder.compound(orders) : orders.get(0)); + return this; + } + + /** + * Gets the order in which the buckets will be returned. + */ + public BucketOrder order() { + return order; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + TimeSeriesAggregationAggregationBuilder that = (TimeSeriesAggregationAggregationBuilder) o; + return keyed == that.keyed + && Objects.equals(group, that.group) + && Objects.equals(without, that.without) + && Objects.equals(interval, that.interval) + && Objects.equals(offset, that.offset) + && Objects.equals(aggregator, that.aggregator) + && Objects.equals(downsampleRange, that.downsampleRange) + && Objects.equals(downsampleFunction, that.downsampleFunction) + && Objects.equals(bucketCountThresholds, that.bucketCountThresholds) + && Objects.equals(order, that.order); + } + + @Override + public int hashCode() { + return Objects.hash( + super.hashCode(), + keyed, + group, + without, + interval, + offset, + aggregator, + downsampleRange, + downsampleFunction, + bucketCountThresholds, + order + ); + } + + @Override + public Version getMinimalSupportedVersion() { + return Version.V_8_3_0; + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java new file mode 100644 index 0000000000000..b82bd15148963 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java @@ -0,0 +1,152 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation; + +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.BucketOrder; +import org.elasticsearch.search.aggregations.CardinalityUpperBound; +import org.elasticsearch.search.aggregations.InternalOrder; +import org.elasticsearch.search.aggregations.bucket.BucketUtils; +import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; +import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator; +import org.elasticsearch.search.aggregations.support.AggregationContext; +import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; +import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; +import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +public class TimeSeriesAggregationAggregationFactory extends ValuesSourceAggregatorFactory { + + private final boolean keyed; + private final List group; + private final List without; + private final DateHistogramInterval interval; + private final DateHistogramInterval offset; + private final String aggregator; + private final DateHistogramInterval downsampleRange; + private final String downsampleFunction; + private final TermsAggregator.BucketCountThresholds bucketCountThresholds; + private final BucketOrder order; + private final ValuesSourceConfig config; + private final TimeSeriesAggregationAggregatorSupplier aggregatorSupplier; + + public TimeSeriesAggregationAggregationFactory( + String name, + boolean keyed, + List group, + List without, + DateHistogramInterval interval, + DateHistogramInterval offset, + String aggregator, + DateHistogramInterval downsampleRange, + String downsampleFunction, + TermsAggregator.BucketCountThresholds bucketCountThresholds, + BucketOrder order, + ValuesSourceConfig config, + AggregationContext context, + AggregatorFactory parent, + AggregatorFactories.Builder subFactoriesBuilder, + Map metadata, + TimeSeriesAggregationAggregatorSupplier aggregatorSupplier + ) throws IOException { + super(name, config, context, parent, subFactoriesBuilder, metadata); + this.keyed = keyed; + this.group = group; + this.without = without; + this.interval = interval; + this.offset = offset; + this.aggregator = aggregator; + this.downsampleRange = downsampleRange; + this.downsampleFunction = downsampleFunction; + this.bucketCountThresholds = bucketCountThresholds; + this.order = order; + this.config = config; + this.aggregatorSupplier = aggregatorSupplier; + } + + public static void registerAggregators(ValuesSourceRegistry.Builder builder) { + builder.register( + TimeSeriesAggregationAggregationBuilder.REGISTRY_KEY, + List.of(CoreValuesSourceType.NUMERIC), + TimeSeriesAggregationAggregator::new, + true + ); + } + + @Override + protected Aggregator createUnmapped(Aggregator parent, Map metadata) throws IOException { + TermsAggregator.BucketCountThresholds thresholds = new TermsAggregator.BucketCountThresholds(this.bucketCountThresholds); + if (InternalOrder.isKeyOrder(order) == false + && thresholds.getShardSize() == TimeSeriesAggregationAggregationBuilder.DEFAULT_BUCKET_COUNT_THRESHOLDS.getShardSize()) { + // The user has not made a shardSize selection. Use default + // heuristic to avoid any wrong-ranking caused by distributed + // counting + thresholds.setShardSize(BucketUtils.suggestShardSideQueueSize(thresholds.getRequiredSize())); + } + thresholds.ensureValidity(); + return new TimeSeriesAggregationAggregator( + name, + factories, + keyed, + group, + without, + interval, + offset, + aggregator, + downsampleRange, + downsampleFunction, + thresholds, + order, + config, + context, + parent, + CardinalityUpperBound.NONE, + metadata + ); + } + + @Override + protected Aggregator doCreateInternal(Aggregator parent, CardinalityUpperBound cardinality, Map metadata) + throws IOException { + TermsAggregator.BucketCountThresholds thresholds = new TermsAggregator.BucketCountThresholds(this.bucketCountThresholds); + if (InternalOrder.isKeyOrder(order) == false + && thresholds.getShardSize() == TimeSeriesAggregationAggregationBuilder.DEFAULT_BUCKET_COUNT_THRESHOLDS.getShardSize()) { + // The user has not made a shardSize selection. Use default + // heuristic to avoid any wrong-ranking caused by distributed + // counting + thresholds.setShardSize(BucketUtils.suggestShardSideQueueSize(thresholds.getRequiredSize())); + } + thresholds.ensureValidity(); + return aggregatorSupplier.build( + name, + factories, + keyed, + group, + without, + interval, + offset, + aggregator, + downsampleRange, + downsampleFunction, + thresholds, + order, + config, + context, + parent, + cardinality, + metadata + ); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java new file mode 100644 index 0000000000000..18fafb1d4974c --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -0,0 +1,412 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.PriorityQueue; +import org.elasticsearch.common.Rounding; +import org.elasticsearch.common.util.set.Sets; +import org.elasticsearch.core.CheckedConsumer; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.index.TimestampBounds; +import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; +import org.elasticsearch.index.mapper.TimeSeriesIdFieldMapper; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.AggregationExecutionContext; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.BucketOrder; +import org.elasticsearch.search.aggregations.CardinalityUpperBound; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalOrder; +import org.elasticsearch.search.aggregations.LeafBucketCollector; +import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; +import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; +import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; +import org.elasticsearch.search.aggregations.bucket.terms.BucketPriorityQueue; +import org.elasticsearch.search.aggregations.bucket.terms.BytesKeyedBucketOrds; +import org.elasticsearch.search.aggregations.bucket.terms.LongKeyedBucketOrds; +import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator; +import org.elasticsearch.search.aggregations.support.AggregationContext; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregation.Function; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.AggregatorBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.TSIDBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.LastFunction; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import static org.elasticsearch.search.DocValueFormat.TIME_SERIES_ID; +import static org.elasticsearch.search.aggregations.InternalOrder.isKeyOrder; + +public class TimeSeriesAggregationAggregator extends BucketsAggregator { + private static final Logger logger = LogManager.getLogger(TimeSeriesAggregationAggregator.class); + + protected BytesKeyedBucketOrds bucketOrds; + private LongKeyedBucketOrds timestampOrds; + private ValuesSource.Numeric valuesSource; + private Map> groupBucketValues; // TODO replace map + private Map aggregatorCollectors; // TODO replace map + + private boolean keyed; + + private Set group; + private Set without; + private long interval; + private long offset; + private TimeSeriesAggregation.Aggregator aggregator; + private long downsampleRange; + protected TimeSeriesAggregation.Function downsampleFunction; + private BucketOrder order; + private TermsAggregator.BucketCountThresholds bucketCountThresholds; + protected Comparator partiallyBuiltBucketComparator; + protected DocValueFormat format; + private TimestampBounds timestampBounds; + + private BytesRef preTsid; + private long preBucketOrdinal; + private long preRounding = -1; + private Rounding.Prepared rounding; + private boolean needGroupBy; + private Map timeBucketMetrics; // TODO replace map + + @SuppressWarnings("unchecked") + public TimeSeriesAggregationAggregator( + String name, + AggregatorFactories factories, + boolean keyed, + List group, + List without, + DateHistogramInterval interval, + DateHistogramInterval offset, + String aggregator, + DateHistogramInterval downsampleRange, + String downsampleFunction, + TermsAggregator.BucketCountThresholds bucketCountThresholds, + BucketOrder order, + ValuesSourceConfig valuesSourceConfig, + AggregationContext context, + org.elasticsearch.search.aggregations.Aggregator parent, + CardinalityUpperBound bucketCardinality, + Map metadata + ) throws IOException { + super(name, factories, context, parent, bucketCardinality, metadata); + this.keyed = keyed; + this.group = group != null ? Sets.newHashSet(group) : Set.of(); + this.without = without != null ? Sets.newHashSet(without) : Set.of(); + this.needGroupBy = this.group.size() > 0 || this.without.size() > 0; + + this.interval = interval != null ? interval.estimateMillis() : -1; + if (this.interval <= 0) { + throw new IllegalArgumentException("time_series_aggregation invalid interval [" + interval + "]"); + } + this.rounding = Rounding.builder(new TimeValue(this.interval)).build().prepareForUnknown(); + this.offset = offset != null ? offset.estimateMillis() : -1; + this.aggregator = aggregator != null ? TimeSeriesAggregation.Aggregator.resolve(aggregator) : null; + this.downsampleRange = downsampleRange != null ? downsampleRange.estimateMillis() : -1; + this.downsampleFunction = downsampleFunction != null ? TimeSeriesAggregation.Function.resolve(downsampleFunction) : Function.last; + if (this.downsampleRange <= 0) { + this.downsampleRange = this.interval; + } + this.bucketCountThresholds = bucketCountThresholds; + this.order = order == null ? BucketOrder.key(true) : order; + this.partiallyBuiltBucketComparator = order.partiallyBuiltBucketComparator(b -> b.bucketOrd, this); + this.bucketOrds = BytesKeyedBucketOrds.build(bigArrays(), bucketCardinality); + this.timestampOrds = LongKeyedBucketOrds.build(bigArrays(), CardinalityUpperBound.MANY); + if (valuesSourceConfig != null) { + this.valuesSource = valuesSourceConfig.hasValues() ? (ValuesSource.Numeric) valuesSourceConfig.getValuesSource() : null; + this.format = valuesSourceConfig.format(); + } else { + this.valuesSource = null; + this.format = null; + } + this.timestampBounds = context.getIndexSettings().getTimestampBounds(); + + if (needGroupBy && this.aggregator == null) { + throw new IllegalArgumentException("time_series_aggregation invalid aggregator [" + aggregator + "]"); + } + + groupBucketValues = new LinkedHashMap<>(); + aggregatorCollectors = new HashMap<>(); + } + + @Override + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + InternalTimeSeriesAggregation.InternalBucket[][] allBucketsPerOrd = + new InternalTimeSeriesAggregation.InternalBucket[owningBucketOrds.length][]; + long[] otherDocCounts = new long[owningBucketOrds.length]; + for (int ordIdx = 0; ordIdx < owningBucketOrds.length; ordIdx++) { + BytesRef spareKey = new BytesRef(); + BytesKeyedBucketOrds.BucketOrdsEnum ordsEnum = bucketOrds.ordsEnum(owningBucketOrds[ordIdx]); + long bucketsInOrd = bucketOrds.bucketsInOrd(owningBucketOrds[ordIdx]); + int size = (int) Math.min(bucketsInOrd, bucketCountThresholds.getShardSize()); + PriorityQueue ordered = new BucketPriorityQueue<>( + size, + partiallyBuiltBucketComparator + ); + while (ordsEnum.next()) { + long docCount = bucketDocCount(ordsEnum.ord()); + otherDocCounts[ordIdx] += docCount; + if (docCount < bucketCountThresholds.getShardMinDocCount()) { + continue; + } + ordsEnum.readValue(spareKey); + long ord = ordsEnum.ord(); + InternalTimeSeriesAggregation.InternalBucket bucket = new InternalTimeSeriesAggregation.InternalBucket( + TimeSeriesIdFieldMapper.decodeTsid(spareKey), + docCount, + new HashMap<>(), + null, + keyed, + false, + 0 + ); + bucket.bucketOrd = ord; + ordered.insertWithOverflow(bucket); + } + + // Get the top buckets + InternalTimeSeriesAggregation.InternalBucket[] bucketsForOrd = new InternalTimeSeriesAggregation.InternalBucket[ordered.size()]; + allBucketsPerOrd[ordIdx] = bucketsForOrd; + for (int b = ordered.size() - 1; b >= 0; --b) { + InternalTimeSeriesAggregation.InternalBucket bucket = ordered.pop(); + long ord = bucket.bucketOrd; + Map values = new LinkedHashMap<>(); + if (needGroupBy) { + AggregatorBucketFunction aggregatorBucketFunction = aggregatorCollectors.get(ord); + LongKeyedBucketOrds.BucketOrdsEnum timeOrdsEnum = timestampOrds.ordsEnum(ord); + while (timeOrdsEnum.next()) { + values.put(timeOrdsEnum.value(), aggregatorBucketFunction.getAggregation(timeOrdsEnum.ord(), format, metadata())); + } + } else { + values = groupBucketValues.get(ord); + } + bucket.timeBucketValues = values; + allBucketsPerOrd[ordIdx][b] = bucket; + otherDocCounts[ordIdx] -= allBucketsPerOrd[ordIdx][b].getDocCount(); + } + } + + buildSubAggsForAllBuckets(allBucketsPerOrd, b -> b.bucketOrd, (b, a) -> b.aggregations = a); + + InternalAggregation[] result = new InternalAggregation[owningBucketOrds.length]; + for (int ordIdx = 0; ordIdx < owningBucketOrds.length; ordIdx++) { + result[ordIdx] = buildResult(otherDocCounts[ordIdx], allBucketsPerOrd[ordIdx]); + } + return result; + } + + @Override + public InternalAggregation buildEmptyAggregation() { + return new InternalTimeSeriesAggregation( + name, + order, + order, + bucketCountThresholds.getRequiredSize(), + bucketCountThresholds.getMinDocCount(), + bucketCountThresholds.getShardSize(), + false, + 0, + new ArrayList<>(), + false, + 0, + metadata() + ); + } + + @Override + protected void doClose() { + Releasables.close(bucketOrds); + Releasables.close(timestampOrds); + aggregatorCollectors.forEach((k, v) -> { v.close(); }); + } + + public class Collector extends LeafBucketCollectorBase { + final SortedNumericDoubleValues values; + final AggregationExecutionContext aggCtx; + final LeafBucketCollector sub; + final CheckedConsumer consumer; + + public Collector( + LeafBucketCollector sub, + SortedNumericDoubleValues values, + AggregationExecutionContext aggCtx, + CheckedConsumer consumer + ) { + super(sub, values); + this.sub = sub; + this.values = values; + this.aggCtx = aggCtx; + this.consumer = consumer; + } + + @Override + public void collect(int doc, long bucket) throws IOException { + BytesRef newTsid = aggCtx.getTsid(); + + if (preTsid == null) { + reset(newTsid, bucket); + } else if (false == preTsid.equals(newTsid)) { + collectTimeSeriesValues(preBucketOrdinal); + reset(newTsid, bucket); + } + + if (preRounding < 0 || aggCtx.getTimestamp() <= preRounding - interval) { + preRounding = rounding.nextRoundingValue(aggCtx.getTimestamp()); + } + consumer.accept(doc); + collectBucket(sub, doc, preBucketOrdinal); + } + + private void reset(BytesRef tsid, long bucket) { + timeBucketMetrics = new LinkedHashMap<>(); + preTsid = BytesRef.deepCopyOf(tsid); + preRounding = -1; + + BytesRef bucketValue = needGroupBy ? packKey(preTsid) : preTsid; + long bucketOrdinal = bucketOrds.add(bucket, bucketValue); + if (bucketOrdinal < 0) { // already seen + bucketOrdinal = -1 - bucketOrdinal; + grow(bucketOrdinal + 1); + } + preBucketOrdinal = bucketOrdinal; + } + } + + @Override + protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBucketCollector sub) throws IOException { + // TODO: remove this method in a follow up PR + throw new UnsupportedOperationException("Shouldn't be here"); + } + + protected AggregatorFunction getAggregatorFunction() throws IOException { + AggregatorFunction function = timeBucketMetrics.get(preRounding); + if (function == null) { + function = TimeSeriesAggregations.getAggregatorFunction(downsampleFunction); + timeBucketMetrics.put(preRounding, function); + } + return function; + } + + protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBucketCollector sub, AggregationExecutionContext aggCtx) + throws IOException { + if (valuesSource == null) { + return LeafBucketCollector.NO_OP_COLLECTOR; + } + final SortedNumericDoubleValues values = valuesSource.doubleValues(context); + return new Collector(sub, values, aggCtx, (doc) -> { + if (values.advanceExact(doc)) { + final int valuesCount = values.docValueCount(); + for (int i = 0; i < valuesCount; i++) { + double value = values.nextValue(); + AggregatorFunction function = getAggregatorFunction(); + if (function instanceof LastFunction last) { + last.collectExact(value, aggCtx.getTimestamp()); + } else { + function.collect(value); + } + } + } + }); + } + + @Override + protected void doPostCollection() throws IOException { + if (timeBucketMetrics != null && timeBucketMetrics.size() > 0) { + collectTimeSeriesValues(preBucketOrdinal); + } + } + + InternalTimeSeriesAggregation buildResult(long otherDocCount, InternalTimeSeriesAggregation.InternalBucket[] topBuckets) { + final BucketOrder reduceOrder; + if (isKeyOrder(order) == false) { + reduceOrder = InternalOrder.key(true); + Arrays.sort(topBuckets, reduceOrder.comparator()); + } else { + reduceOrder = order; + } + return new InternalTimeSeriesAggregation( + name, + reduceOrder, + order, + bucketCountThresholds.getRequiredSize(), + bucketCountThresholds.getMinDocCount(), + bucketCountThresholds.getShardSize(), + false, + otherDocCount, + List.of(topBuckets), + keyed, + 0, + metadata() + ); + } + + private BytesRef packKey(BytesRef tsid) { + Map tsidMap = TimeSeriesIdFieldMapper.decodeTsid(tsid); + Map groupMap = new LinkedHashMap<>(); + tsidMap.forEach((key, value) -> { + if (group.size() > 0) { + if (group.contains(key) && !without.contains(key)) { + groupMap.put(key, value); + } + } else { + if (!without.contains(key)) { + groupMap.put(key, value); + } + } + }); + return TIME_SERIES_ID.parseBytesRef(groupMap); + } + + public void collectTimeSeriesValues(long bucketOrd) throws IOException { + if (needGroupBy) { + AggregatorBucketFunction aggregatorBucketFunction = aggregatorCollectors.get(bucketOrd); + if (aggregatorBucketFunction == null) { + AggregatorBucketFunction internal = TimeSeriesAggregations.getAggregatorBucketFunction(aggregator, bigArrays()); + aggregatorBucketFunction = new TSIDBucketFunction(internal); + aggregatorCollectors.put(bucketOrd, aggregatorBucketFunction); + } + + for (Entry entry : timeBucketMetrics.entrySet()) { + Long timestamp = entry.getKey(); + AggregatorFunction value = entry.getValue(); + long ord = timestampOrds.add(bucketOrd, timestamp); + if (ord < 0) { // already seen + ord = -1 - ord; + } + if (timestamp - interval <= timestampBounds.startTime() || timestamp > timestampBounds.endTime()) { + aggregatorBucketFunction.collect( + new TSIDBucketFunction.TSIDValue(preTsid, value.getAggregation(format, metadata()), true), + ord + ); + } else { + aggregatorBucketFunction.collect(new TSIDBucketFunction.TSIDValue(preTsid, value.get(), false), ord); + } + } + } else { + Map tsids = new LinkedHashMap<>(); + timeBucketMetrics.forEach((k, v) -> { tsids.put(k, v.getAggregation(format, metadata())); }); + groupBucketValues.put(bucketOrd, tsids); + } + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java new file mode 100644 index 0000000000000..02898bf89893c --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java @@ -0,0 +1,45 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation; + +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.BucketOrder; +import org.elasticsearch.search.aggregations.CardinalityUpperBound; +import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; +import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator; +import org.elasticsearch.search.aggregations.support.AggregationContext; +import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +@FunctionalInterface +public interface TimeSeriesAggregationAggregatorSupplier { + Aggregator build( + String name, + AggregatorFactories factories, + boolean keyed, + List group, + List without, + DateHistogramInterval interval, + DateHistogramInterval offset, + String aggregator, + DateHistogramInterval downsampleRange, + String downsampleFunction, + TermsAggregator.BucketCountThresholds bucketCountThresholds, + BucketOrder order, + ValuesSourceConfig valuesSourceConfig, + AggregationContext context, + Aggregator parent, + CardinalityUpperBound bucketCardinality, + Map metadata + ) throws IOException; +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregations.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregations.java new file mode 100644 index 0000000000000..542489636c1fd --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregations.java @@ -0,0 +1,59 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.AggregatorBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.AvgBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.MaxBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.MinBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.NoAggregatorBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.SumBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AvgFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MaxFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MinFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.SumFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.ValueCountFunction; + +public class TimeSeriesAggregations { + public static AggregatorFunction getAggregatorFunction(TimeSeriesAggregation.Function function) { + switch (function) { + case avg: + return new AvgFunction(); + case max: + return new MaxFunction(); + case min: + return new MinFunction(); + case sum: + return new SumFunction(); + case count: + return new ValueCountFunction(); + } + return new AvgFunction(); + } + + public static AggregatorBucketFunction getAggregatorBucketFunction(TimeSeriesAggregation.Aggregator aggregator, BigArrays bigArrays) { + if (aggregator == null) { + return new NoAggregatorBucketFunction(); + } + + switch (aggregator) { + case max: + return new MaxBucketFunction(bigArrays); + case min: + return new MinBucketFunction(bigArrays); + case sum: + return new SumBucketFunction(bigArrays); + case avg: + return new AvgBucketFunction(bigArrays); + } + return new AvgBucketFunction(bigArrays); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AggregatorBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AggregatorBucketFunction.java new file mode 100644 index 0000000000000..3d8edcac2d044 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AggregatorBucketFunction.java @@ -0,0 +1,24 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction; + +import java.util.Map; + +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; + +public interface AggregatorBucketFunction { + String name(); + + void collect(Input number, long bucket); + + InternalAggregation getAggregation(long bucket, DocValueFormat formatter, Map metadata); + + void close(); +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AvgBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AvgBucketFunction.java new file mode 100644 index 0000000000000..596939f2f5b64 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AvgBucketFunction.java @@ -0,0 +1,69 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.DoubleArray; +import org.elasticsearch.common.util.LongArray; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.AvgAggregationBuilder; + +import java.util.Map; + +public class AvgBucketFunction implements AggregatorBucketFunction { + private final BigArrays bigArrays; + private DoubleArray sums; + private LongArray counts; + + public AvgBucketFunction(BigArrays bigArrays) { + this.bigArrays = bigArrays; + sums = bigArrays.newDoubleArray(1, true); + sums.fill(0, sums.size(), 0); + counts = bigArrays.newLongArray(1, true); + counts.fill(0, counts.size(), 0); + } + + @Override + public String name() { + return AvgAggregationBuilder.NAME; + } + + @Override + public void collect(Double number, long bucket) { + if (bucket >= sums.size()) { + long from = sums.size(); + sums = bigArrays.grow(sums, bucket + 1); + sums.fill(from, sums.size(), 0); + counts = bigArrays.grow(counts, bucket + 1); + counts.fill(from, counts.size(), 0); + } + + double current = sums.get(bucket); + sums.set(bucket, current + number); + counts.increment(bucket, 1); + } + + @Override + public InternalAggregation getAggregation(long bucket, DocValueFormat formatter, Map metadata) { + return new org.elasticsearch.search.aggregations.metrics.InternalAvg( + name(), + sums.get(bucket), + counts.get(bucket), + formatter, + metadata + ); + } + + @Override + public void close() { + Releasables.close(sums, counts); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MaxBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MaxBucketFunction.java new file mode 100644 index 0000000000000..ca30d765eddfd --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MaxBucketFunction.java @@ -0,0 +1,56 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.DoubleArray; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder; + +import java.util.Map; + +public class MaxBucketFunction implements AggregatorBucketFunction { + private final BigArrays bigArrays; + private DoubleArray value; + + public MaxBucketFunction(BigArrays bigArrays) { + this.bigArrays = bigArrays; + value = bigArrays.newDoubleArray(1, true); + value.fill(0, value.size(), Double.NEGATIVE_INFINITY); + } + + @Override + public String name() { + return MaxAggregationBuilder.NAME; + } + + @Override + public void collect(Double number, long bucket) { + if (bucket >= value.size()) { + long from = value.size(); + value = bigArrays.grow(value, bucket + 1); + value.fill(from, value.size(), Double.NEGATIVE_INFINITY); + } + + double current = value.get(bucket); + value.set(bucket, Math.max(current, number)); + } + + @Override + public InternalAggregation getAggregation(long bucket, DocValueFormat formatter, Map metadata) { + return new org.elasticsearch.search.aggregations.metrics.Max(name(), value.get(bucket), formatter, metadata); + } + + @Override + public void close() { + Releasables.close(value); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MinBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MinBucketFunction.java new file mode 100644 index 0000000000000..e6288619fb36a --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MinBucketFunction.java @@ -0,0 +1,56 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.DoubleArray; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.MinAggregationBuilder; + +import java.util.Map; + +public class MinBucketFunction implements AggregatorBucketFunction { + private final BigArrays bigArrays; + private DoubleArray value; + + public MinBucketFunction(BigArrays bigArrays) { + this.bigArrays = bigArrays; + value = bigArrays.newDoubleArray(1, true); + value.fill(0, value.size(), Double.POSITIVE_INFINITY); + } + + @Override + public String name() { + return MinAggregationBuilder.NAME; + } + + @Override + public void collect(Double number, long bucket) { + if (bucket >= value.size()) { + long from = value.size(); + value = bigArrays.grow(value, bucket + 1); + value.fill(from, value.size(), Double.POSITIVE_INFINITY); + } + + double current = value.get(bucket); + value.set(bucket, Math.min(current, number)); + } + + @Override + public InternalAggregation getAggregation(long bucket, DocValueFormat formatter, Map metadata) { + return new org.elasticsearch.search.aggregations.metrics.Min(name(), value.get(bucket), formatter, metadata); + } + + @Override + public void close() { + Releasables.close(value); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/NoAggregatorBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/NoAggregatorBucketFunction.java new file mode 100644 index 0000000000000..dece2d0afe8b6 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/NoAggregatorBucketFunction.java @@ -0,0 +1,39 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction; + +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.Last; + +import java.util.Map; + +public class NoAggregatorBucketFunction implements AggregatorBucketFunction { + private double value; + + @Override + public String name() { + return "no_aggregator"; + } + + @Override + public void collect(Double number, long bucket) { + value = number; + } + + @Override + public InternalAggregation getAggregation(long bucket, DocValueFormat formatter, Map metadata) { + return new Last(name(), value, 0, formatter, metadata); + } + + @Override + public void close() { + + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/SumBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/SumBucketFunction.java new file mode 100644 index 0000000000000..49b62201cf9d3 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/SumBucketFunction.java @@ -0,0 +1,56 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.DoubleArray; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder; + +import java.util.Map; + +public class SumBucketFunction implements AggregatorBucketFunction { + private final BigArrays bigArrays; + private DoubleArray value; + + public SumBucketFunction(BigArrays bigArrays) { + this.bigArrays = bigArrays; + value = bigArrays.newDoubleArray(1, true); + value.fill(0, value.size(), 0); + } + + @Override + public String name() { + return SumAggregationBuilder.NAME; + } + + @Override + public void collect(Double number, long bucket) { + if (bucket >= value.size()) { + long from = value.size(); + value = bigArrays.grow(value, bucket + 1); + value.fill(from, value.size(), 0); + } + + double current = value.get(bucket); + value.set(bucket, current + number); + } + + @Override + public InternalAggregation getAggregation(long bucket, DocValueFormat formatter, Map metadata) { + return new org.elasticsearch.search.aggregations.metrics.Sum(name(), value.get(bucket), formatter, metadata); + } + + @Override + public void close() { + Releasables.close(value); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java new file mode 100644 index 0000000000000..ce77e32fcb8c9 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java @@ -0,0 +1,72 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.TSIDBucketFunction.TSIDValue; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TSIDInternalAggregation; + +import java.util.HashMap; +import java.util.Map; + +public class TSIDBucketFunction implements AggregatorBucketFunction { + private Map> values = new HashMap<>(); + private final AggregatorBucketFunction aggregatorBucketFunction; + + public TSIDBucketFunction(AggregatorBucketFunction aggregatorBucketFunction) { + this.aggregatorBucketFunction = aggregatorBucketFunction; + } + + @Override + public String name() { + return "tsid"; + } + + @Override + public void collect(TSIDValue tsidValue, long bucket) { + if (tsidValue.detailed) { + Map tsidValues = values.get(bucket); + if (tsidValues == null) { + tsidValues = new HashMap<>(); + values.put(bucket, tsidValues); + } + tsidValues.put(tsidValue.tsid, (InternalAggregation) tsidValue.value); + } else { + aggregatorBucketFunction.collect(tsidValue.value, bucket); + } + } + + @Override + public void close() { + aggregatorBucketFunction.close(); + } + + @Override + public InternalAggregation getAggregation(long bucket, DocValueFormat formatter, Map metadata) { + if (values.containsKey(bucket)) { + return new TSIDInternalAggregation(name(), values.get(bucket), aggregatorBucketFunction.name(), formatter, metadata); + } else { + return aggregatorBucketFunction.getAggregation(bucket, formatter, metadata); + } + } + + public static class TSIDValue { + public TSIDValue(BytesRef tsid, Object value, boolean detailed) { + this.tsid = tsid; + this.value = value; + this.detailed = detailed; + } + + BytesRef tsid; + Object value; + boolean detailed; + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/ValueCountBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/ValueCountBucketFunction.java new file mode 100644 index 0000000000000..6fbbb2bb8aa69 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/ValueCountBucketFunction.java @@ -0,0 +1,56 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.LongArray; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.InternalValueCount; +import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder; + +import java.util.Map; + +public class ValueCountBucketFunction implements AggregatorBucketFunction { + private final BigArrays bigArrays; + private LongArray counts; + + public ValueCountBucketFunction(BigArrays bigArrays) { + this.bigArrays = bigArrays; + counts = bigArrays.newLongArray(1, true); + counts.fill(0, counts.size(), 0); + } + + @Override + public String name() { + return ValueCountAggregationBuilder.NAME; + } + + @Override + public void collect(Double number, long bucket) { + if (bucket >= counts.size()) { + long from = counts.size(); + counts = bigArrays.grow(counts, bucket + 1); + counts.fill(from, counts.size(), 0); + } + + counts.increment(bucket, 1); + } + + @Override + public InternalAggregation getAggregation(long bucket, DocValueFormat formatter, Map metadata) { + return new InternalValueCount(name(), counts.get(bucket), metadata); + } + + @Override + public void close() { + Releasables.close(counts); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AggregatorFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AggregatorFunction.java new file mode 100644 index 0000000000000..f3da28d85e951 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AggregatorFunction.java @@ -0,0 +1,22 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.function; + +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; + +import java.util.Map; + +public interface AggregatorFunction { + void collect(Input value); + + Output get(); + + InternalAggregation getAggregation(DocValueFormat formatter, Map metadata); +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AvgFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AvgFunction.java new file mode 100644 index 0000000000000..e240092ddd402 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AvgFunction.java @@ -0,0 +1,41 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.function; + +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.AvgAggregationBuilder; + +import java.util.Map; + +public class AvgFunction implements AggregatorFunction { + private double sum = 0; + private long count = 0; + + @Override + public void collect(Double value) { + sum += value; + count += 1; + } + + public void collectExact(double sum, long count) { + this.sum += sum; + this.count += count; + } + + @Override + public Double get() { + return sum / count; + } + + @Override + public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { + return new org.elasticsearch.search.aggregations.metrics.InternalAvg(AvgAggregationBuilder.NAME, sum, count, formatter, metadata); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java new file mode 100644 index 0000000000000..1db44cff9c8fb --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java @@ -0,0 +1,43 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.function; + +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.Last; + +import java.util.Map; + +public class LastFunction implements AggregatorFunction { + private double last = Double.NEGATIVE_INFINITY; + private long timestamp = Long.MIN_VALUE; + + @Override + public void collect(Double value) { + if (last != Double.NEGATIVE_INFINITY) { + last = value; + } + } + + public void collectExact(double number, long timestamp) { + if (timestamp > this.timestamp) { + last = number; + } + } + + @Override + public Double get() { + return last; + } + + @Override + public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { + return new Last("last", last, timestamp, formatter, metadata); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/MaxFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/MaxFunction.java new file mode 100644 index 0000000000000..e443eaed800fa --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/MaxFunction.java @@ -0,0 +1,34 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.function; + +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder; + +import java.util.Map; + +public class MaxFunction implements AggregatorFunction { + private Double max = Double.NEGATIVE_INFINITY; + + @Override + public void collect(Double value) { + this.max = Math.max(value, max); + } + + @Override + public Double get() { + return max; + } + + @Override + public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { + return new org.elasticsearch.search.aggregations.metrics.Max(MaxAggregationBuilder.NAME, max, formatter, metadata); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/MinFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/MinFunction.java new file mode 100644 index 0000000000000..e2600654d8579 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/MinFunction.java @@ -0,0 +1,34 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.function; + +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.MinAggregationBuilder; + +import java.util.Map; + +public class MinFunction implements AggregatorFunction { + private Double min = Double.POSITIVE_INFINITY; + + @Override + public void collect(Double value) { + this.min = Math.min(value, min); + } + + @Override + public Double get() { + return min; + } + + @Override + public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { + return new org.elasticsearch.search.aggregations.metrics.Min(MinAggregationBuilder.NAME, min, formatter, metadata); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/SumFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/SumFunction.java new file mode 100644 index 0000000000000..51eb7906fb7ca --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/SumFunction.java @@ -0,0 +1,34 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.function; + +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder; + +import java.util.Map; + +public class SumFunction implements AggregatorFunction { + private double sum = 0; + + @Override + public void collect(Double value) { + this.sum += value; + } + + @Override + public Double get() { + return sum; + } + + @Override + public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { + return new org.elasticsearch.search.aggregations.metrics.Sum(SumAggregationBuilder.NAME, sum, formatter, metadata); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountFunction.java new file mode 100644 index 0000000000000..c1d62a79960e4 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountFunction.java @@ -0,0 +1,38 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.function; + +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder; + +import java.util.Map; + +public class ValueCountFunction implements AggregatorFunction { + private long count = 0; + + @Override + public void collect(Double value) { + count += 1; + } + + public void collectExact(long count) { + this.count += count; + } + + @Override + public Long get() { + return count; + } + + @Override + public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { + return new org.elasticsearch.search.aggregations.metrics.InternalValueCount(ValueCountAggregationBuilder.NAME, count, metadata); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java new file mode 100644 index 0000000000000..5949db561a9cc --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java @@ -0,0 +1,100 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.internal; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.AggregationReduceContext; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; +import org.elasticsearch.search.aggregations.support.SamplingContext; +import org.elasticsearch.xcontent.XContentBuilder; + +public class Last extends InternalNumericMetricsAggregation.SingleValue { + private final double last; + private final long timestamp; + + public Last(String name, double last, long timestamp, DocValueFormat formatter, Map metadata) { + super(name, formatter, metadata); + this.last = last; + this.timestamp = timestamp; + } + + /** + * Read from a stream. + */ + public Last(StreamInput in) throws IOException { + super(in); + last = in.readDouble(); + timestamp = in.readLong(); + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + out.writeNamedWriteable(format); + out.writeDouble(last); + out.writeLong(timestamp); + } + + @Override + public String getWriteableName() { + return "last"; + } + + public InternalAggregation finalizeSampling(SamplingContext samplingContext) { + return this; + } + + @Override + public double value() { + return last; + } + + @Override + public Last reduce(List aggregations, AggregationReduceContext reduceContext) { + double last = Double.NEGATIVE_INFINITY; + long timestamp = Long.MIN_VALUE; + for (InternalAggregation aggregation : aggregations) { + if (((Last) aggregation).timestamp > timestamp) { + last = ((Last) aggregation).last; + } + } + return new Last(name, last, timestamp, format, getMetadata()); + } + + @Override + public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + boolean hasValue = Double.isInfinite(last) == false; + builder.field(CommonFields.VALUE.getPreferredName(), hasValue ? last : null); + if (hasValue && format != DocValueFormat.RAW) { + builder.field(CommonFields.VALUE_AS_STRING.getPreferredName(), format.format(last).toString()); + } + return builder; + } + + @Override + public boolean equals(Object o) { + if (this == o) {return true;} + if (o == null || getClass() != o.getClass()) {return false;} + if (!super.equals(o)) {return false;} + Last last1 = (Last)o; + return Double.compare(last1.last, last) == 0 && timestamp == last1.timestamp; + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), last, timestamp); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java new file mode 100644 index 0000000000000..72467676cb1b7 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java @@ -0,0 +1,132 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.internal; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.AggregationReduceContext; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; +import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation.SingleValue; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregations; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +public class TSIDInternalAggregation extends InternalAggregation { + private final Map values; + private final String aggreagator; + private final DocValueFormat formatter; + + public TSIDInternalAggregation( + String name, + Map values, + String aggregator, + DocValueFormat formatter, + Map metadata + ) { + super(name, metadata); + this.values = values; + this.formatter = formatter; + this.aggreagator = aggregator; + } + + public TSIDInternalAggregation(StreamInput in) throws IOException { + super(in); + formatter = in.readNamedWriteable(DocValueFormat.class); + aggreagator = in.readString(); + values = in.readOrderedMap(StreamInput::readBytesRef, stream -> stream.readNamedWriteable(InternalAggregation.class)); + + } + + @Override + public String getWriteableName() { + return "tsid"; + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + out.writeNamedWriteable(formatter); + out.writeString(aggreagator); + out.writeMap(values, StreamOutput::writeBytesRef, StreamOutput::writeNamedWriteable); + } + + @Override + public InternalAggregation reduce(List aggregations, AggregationReduceContext reduceContext) { + if (aggregations.size() == 1) { + TSIDInternalAggregation tsidAgg = (TSIDInternalAggregation) aggregations.get(0); + if (reduceContext.isFinalReduce()) { + TimeSeriesAggregation.Function function = TimeSeriesAggregation.Function.valueOf(aggreagator); + final AggregatorFunction aggregatorFunction = TimeSeriesAggregations.getAggregatorFunction(function); + tsidAgg.values.forEach( + (tsid, agg) -> { aggregatorFunction.collect(((InternalNumericMetricsAggregation.SingleValue) agg).value()); } + ); + return aggregatorFunction.getAggregation(formatter, getMetadata()); + } else { + return tsidAgg; + } + } + + Map> reduced = new TreeMap<>(); + for (InternalAggregation aggregation : aggregations) { + TSIDInternalAggregation tsidAgg = (TSIDInternalAggregation) aggregation; + tsidAgg.values.forEach((tsid, value) -> { + if (false == reduced.containsKey(tsid)) { + reduced.put(tsid, List.of(value)); + } else { + reduced.get(tsid).add(value); + } + }); + } + + if (reduceContext.isFinalReduce()) { + TimeSeriesAggregation.Function function = TimeSeriesAggregation.Function.valueOf(aggreagator); + final AggregatorFunction aggregatorFunction = TimeSeriesAggregations.getAggregatorFunction(function); + reduced.forEach((tsid, aggs) -> { + if (aggs.size() > 0) { + InternalAggregation first = aggs.get(0); + InternalNumericMetricsAggregation.SingleValue internalAggregation = (SingleValue) first.reduce(aggs, reduceContext); + aggregatorFunction.collect(internalAggregation.value()); + } + }); + return aggregatorFunction.getAggregation(formatter, getMetadata()); + } else { + Map finalReduces = new TreeMap<>(); + reduced.forEach((tsid, aggs) -> { + if (aggs.size() > 0) { + InternalAggregation first = aggs.get(0); + finalReduces.put(tsid, first.reduce(aggs, reduceContext)); + } + }); + return new TSIDInternalAggregation(name, finalReduces, aggreagator, formatter, getMetadata()); + } + } + + @Override + protected boolean mustReduceOnSingleInternalAgg() { + return false; + } + + @Override + public Object getProperty(List path) { + return null; + } + + @Override + public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + return null; + } +} diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/AggregationsTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/AggregationsTests.java index 7be3c013eed99..da81a4fe45c6f 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/AggregationsTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/AggregationsTests.java @@ -64,6 +64,7 @@ import org.elasticsearch.search.aggregations.pipeline.InternalPercentilesBucketTests; import org.elasticsearch.search.aggregations.pipeline.InternalSimpleValueTests; import org.elasticsearch.search.aggregations.timeseries.InternalTimeSeriesTests; +import org.elasticsearch.search.aggregations.timeseries.aggregation.InternalTimeSeriesAggregationTests; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.InternalAggregationTestCase; import org.elasticsearch.test.InternalMultiBucketAggregationTestCase; @@ -146,7 +147,8 @@ public class AggregationsTests extends ESTestCase { new InternalTopHitsTests(), new InternalCompositeTests(), new InternalMedianAbsoluteDeviationTests(), - new InternalTimeSeriesTests() + new InternalTimeSeriesTests(), + new InternalTimeSeriesAggregationTests() ); @Override diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregationTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregationTests.java new file mode 100644 index 0000000000000..68c50414c034d --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregationTests.java @@ -0,0 +1,116 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation; + +import org.elasticsearch.search.aggregations.BucketOrder; +import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.aggregations.ParsedMultiBucketAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.InternalTimeSeriesAggregation.InternalBucket; +import org.elasticsearch.test.InternalMultiBucketAggregationTestCase; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.function.Predicate; + +public class InternalTimeSeriesAggregationTests extends InternalMultiBucketAggregationTestCase { + + @Override + protected void assertReduced(InternalTimeSeriesAggregation reduced, List inputs) { + // TODO + } + + @Override + protected InternalTimeSeriesAggregation createTestInstance(String name, Map metadata, InternalAggregations aggregations) { + BucketOrder order = BucketOrder.count(false); + long minDocCount = 1; + int requiredSize = 3; + int shardSize = requiredSize + 2; + long otherDocCount = 0; + boolean keyed = randomBoolean(); + boolean showTermDocCountError = randomBoolean(); + List buckets = randomBuckets(keyed, shardSize, showTermDocCountError, requiredSize); + BucketOrder reduceOrder = randomBoolean() + ? BucketOrder.compound(BucketOrder.key(true), BucketOrder.count(false)) + : BucketOrder.key(true); + Collections.sort(buckets, reduceOrder.comparator()); + return new InternalTimeSeriesAggregation( + name, + reduceOrder, + order, + requiredSize, + minDocCount, + shardSize, + showTermDocCountError, + otherDocCount, + buckets, + keyed, + randomNonNegativeLong(), + metadata + ); + } + + @Override + protected Class> implementationClass() { + return ParsedTimeSeriesAggregation.class; + } + + private List randomBuckets( + boolean keyed, + int shardSize, + boolean showTermDocCountError, + int size + ) { + int numberOfBuckets = randomIntBetween(0, shardSize); + List bucketList = new ArrayList<>(numberOfBuckets); + List> keys = randomKeys(bucketKeys(randomIntBetween(1, 4)), numberOfBuckets); + for (int j = 0; j < numberOfBuckets; j++) { + long docCount = randomLongBetween(0, Long.MAX_VALUE / (size * numberOfBuckets)); + long docCountError = showTermDocCountError ? randomLongBetween(0, Long.MAX_VALUE / (size * numberOfBuckets)) : -1; + bucketList.add( + new InternalTimeSeriesAggregation.InternalBucket( + keys.get(j), + docCount, + new HashMap<>(), + InternalAggregations.EMPTY, + keyed, + showTermDocCountError, + docCountError + ) + ); + } + return bucketList; + } + + private List bucketKeys(int numberOfKeys) { + return randomUnique(() -> randomAlphaOfLength(10), numberOfKeys).stream().toList(); + } + + private List> randomKeys(List bucketKeys, int numberOfBuckets) { + List> keys = new ArrayList<>(); + for (int i = 0; i < numberOfBuckets; i++) { + keys.add(randomValueOtherThanMany(keys::contains, () -> { + Map key = new TreeMap<>(); + for (String name : bucketKeys) { + key.put(name, randomAlphaOfLength(4)); + } + return key; + })); + } + return keys; + } + + @Override + protected Predicate excludePathsFromXContentInsertion() { + return s -> s.endsWith(".key") || s.endsWith("values"); + } +} diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilderTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilderTests.java new file mode 100644 index 0000000000000..132323769c536 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilderTests.java @@ -0,0 +1,61 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation; + +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.search.SearchModule; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.test.AbstractSerializingTestCase; +import org.elasticsearch.xcontent.NamedXContentRegistry; +import org.elasticsearch.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Collections; + +import static org.hamcrest.Matchers.hasSize; + +public class TimeSeriesAggregationAggregationBuilderTests extends AbstractSerializingTestCase { + + @Override + protected TimeSeriesAggregationAggregationBuilder doParseInstance(XContentParser parser) throws IOException { + assertSame(XContentParser.Token.START_OBJECT, parser.nextToken()); + AggregatorFactories.Builder parsed = AggregatorFactories.parseAggregators(parser); + assertThat(parsed.getAggregatorFactories(), hasSize(1)); + assertThat(parsed.getPipelineAggregatorFactories(), hasSize(0)); + TimeSeriesAggregationAggregationBuilder agg = (TimeSeriesAggregationAggregationBuilder) parsed.getAggregatorFactories() + .iterator() + .next(); + assertNull(parser.nextToken()); + assertNotNull(agg); + return agg; + } + + @Override + protected Writeable.Reader instanceReader() { + return TimeSeriesAggregationAggregationBuilder::new; + } + + @Override + protected TimeSeriesAggregationAggregationBuilder createTestInstance() { + return new TimeSeriesAggregationAggregationBuilder(randomAlphaOfLength(10)); + } + + @Override + protected NamedWriteableRegistry getNamedWriteableRegistry() { + return new NamedWriteableRegistry(new SearchModule(Settings.EMPTY, Collections.emptyList()).getNamedWriteables()); + } + + @Override + protected NamedXContentRegistry xContentRegistry() { + return new NamedXContentRegistry(new SearchModule(Settings.EMPTY, Collections.emptyList()).getNamedXContents()); + } + +} diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorTests.java new file mode 100644 index 0000000000000..6fbf92da77ff6 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorTests.java @@ -0,0 +1,15 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation; + +import org.elasticsearch.search.aggregations.AggregatorTestCase; + +public class TimeSeriesAggregationAggregatorTests extends AggregatorTestCase { + +} diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalAggregationTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/InternalAggregationTestCase.java index 95ecf3f3a24fe..e6516fa6cf1c3 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalAggregationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalAggregationTestCase.java @@ -145,6 +145,8 @@ import org.elasticsearch.search.aggregations.support.SamplingContext; import org.elasticsearch.search.aggregations.timeseries.ParsedTimeSeries; import org.elasticsearch.search.aggregations.timeseries.TimeSeriesAggregationBuilder; +import org.elasticsearch.search.aggregations.timeseries.aggregation.ParsedTimeSeriesAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregationAggregationBuilder; import org.elasticsearch.xcontent.ContextParser; import org.elasticsearch.xcontent.NamedXContentRegistry; import org.elasticsearch.xcontent.ParseField; @@ -303,6 +305,7 @@ public AggregationReduceContext forFinalReduction() { map.put(TopHitsAggregationBuilder.NAME, (p, c) -> ParsedTopHits.fromXContent(p, (String) c)); map.put(CompositeAggregationBuilder.NAME, (p, c) -> ParsedComposite.fromXContent(p, (String) c)); map.put(TimeSeriesAggregationBuilder.NAME, (p, c) -> ParsedTimeSeries.fromXContent(p, (String) c)); + map.put(TimeSeriesAggregationAggregationBuilder.NAME, (p, c) -> ParsedTimeSeriesAggregation.fromXContent(p, (String) c)); namedXContents = map.entrySet() .stream() diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/AggregateMetricMapperPlugin.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/AggregateMetricMapperPlugin.java index e44b65ee2212b..9ebc097b12d1d 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/AggregateMetricMapperPlugin.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/AggregateMetricMapperPlugin.java @@ -49,7 +49,8 @@ public List> getAggregationExtentions() { AggregateMetricsAggregatorsRegistrar::registerAvgAggregator, AggregateMetricsAggregatorsRegistrar::registerMinAggregator, AggregateMetricsAggregatorsRegistrar::registerMaxAggregator, - AggregateMetricsAggregatorsRegistrar::registerValueCountAggregator + AggregateMetricsAggregatorsRegistrar::registerValueCountAggregator, + AggregateMetricsAggregatorsRegistrar::registerTimeSeriesAggregationAggregator ); } } diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java new file mode 100644 index 0000000000000..e8fd9c3cff78c --- /dev/null +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java @@ -0,0 +1,159 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.aggregatemetric.aggregations.metrics; + +import org.apache.lucene.index.LeafReaderContext; +import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; +import org.elasticsearch.search.aggregations.AggregationExecutionContext; +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.BucketOrder; +import org.elasticsearch.search.aggregations.CardinalityUpperBound; +import org.elasticsearch.search.aggregations.LeafBucketCollector; +import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; +import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds; +import org.elasticsearch.search.aggregations.support.AggregationContext; +import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregation.Avg; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregation.Function; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregation.ValueCount; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregationAggregator; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AvgFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.ValueCountFunction; +import org.elasticsearch.xpack.aggregatemetric.aggregations.support.AggregateMetricsValuesSource; +import org.elasticsearch.xpack.aggregatemetric.mapper.AggregateDoubleMetricFieldMapper.Metric; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +public class AggregateMetricTimeSeriesAggregationAggregator extends TimeSeriesAggregationAggregator { + private final AggregateMetricsValuesSource.AggregateDoubleMetric valuesSource; + + public AggregateMetricTimeSeriesAggregationAggregator( + String name, + AggregatorFactories factories, + boolean keyed, + List group, + List without, + DateHistogramInterval interval, + DateHistogramInterval offset, + String aggregator, + DateHistogramInterval downsampleRange, + String downsampleFunction, + BucketCountThresholds bucketCountThresholds, + BucketOrder order, + ValuesSourceConfig valuesSourceConfig, + AggregationContext context, + Aggregator parent, + CardinalityUpperBound bucketCardinality, + Map metadata + ) throws IOException { + super( + name, + factories, + keyed, + group, + without, + interval, + offset, + aggregator, + downsampleRange, + downsampleFunction, + bucketCountThresholds, + order, + null, + context, + parent, + bucketCardinality, + metadata + ); + this.valuesSource = valuesSourceConfig.hasValues() + ? (AggregateMetricsValuesSource.AggregateDoubleMetric) valuesSourceConfig.getValuesSource() + : null; + this.format = valuesSourceConfig.format(); + if (this.downsampleFunction == null) { + this.downsampleFunction = Function.avg; + } + } + + @Override + protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBucketCollector sub, AggregationExecutionContext aggCtx) + throws IOException { + if (valuesSource == null) { + return LeafBucketCollector.NO_OP_COLLECTOR; + } + Metric metricType = getAggregateMetric(); + if (metricType != null) { + final SortedNumericDoubleValues values = valuesSource.getAggregateMetricValues(context, metricType); + return new Collector(sub, values, aggCtx, (doc) -> { + if (values.advanceExact(doc)) { + final int valuesCount = values.docValueCount(); + for (int i = 0; i < valuesCount; i++) { + double value = values.nextValue(); + AggregatorFunction function = getAggregatorFunction(); + function.collect(value); + } + } + }); + } else if (metricType == Metric.value_count) { + final SortedNumericDoubleValues values = valuesSource.getAggregateMetricValues(context, metricType); + return new Collector(sub, values, aggCtx, (doc) -> { + if (values.advanceExact(doc)) { + final int valuesCount = values.docValueCount(); + for (int i = 0; i < valuesCount; i++) { + double value = values.nextValue(); + AggregatorFunction function = getAggregatorFunction(); + ((ValueCountFunction) function).collectExact((long) value); + } + } + }); + } else { + final SortedNumericDoubleValues aggregateSums = valuesSource.getAggregateMetricValues(context, Metric.sum); + final SortedNumericDoubleValues aggregateValueCounts = valuesSource.getAggregateMetricValues(context, Metric.value_count); + return new Collector(sub, aggregateSums, aggCtx, (doc) -> { + double sum = 0; + long valueCount = 0; + if (aggregateSums.advanceExact(doc)) { + final int valuesCount = aggregateSums.docValueCount(); + for (int i = 0; i < valuesCount; i++) { + double value = aggregateSums.nextValue(); + sum += value; + } + } + + if (aggregateValueCounts.advanceExact(doc)) { + final int valuesCount = aggregateValueCounts.docValueCount(); + for (int i = 0; i < valuesCount; i++) { + double value = aggregateValueCounts.nextValue(); + valueCount += value; + } + } + + AggregatorFunction function = getAggregatorFunction(); + ((AvgFunction) function).collectExact(sum, valueCount); + }); + } + } + + private Metric getAggregateMetric() { + switch (downsampleFunction) { + case max: + return Metric.max; + case min: + return Metric.min; + case sum: + return Metric.sum; + case count: + return Metric.value_count; + } + return null; + } +} diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricsAggregatorsRegistrar.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricsAggregatorsRegistrar.java index 8b612b6d8c7bb..1211ca2e6d045 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricsAggregatorsRegistrar.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricsAggregatorsRegistrar.java @@ -12,6 +12,7 @@ import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregationAggregationBuilder; import org.elasticsearch.xpack.aggregatemetric.aggregations.support.AggregateMetricsValuesSourceType; /** @@ -63,4 +64,13 @@ public static void registerValueCountAggregator(ValuesSourceRegistry.Builder bui true ); } + + public static void registerTimeSeriesAggregationAggregator(ValuesSourceRegistry.Builder builder) { + builder.register( + TimeSeriesAggregationAggregationBuilder.REGISTRY_KEY, + AggregateMetricsValuesSourceType.AGGREGATE_METRIC, + AggregateMetricTimeSeriesAggregationAggregator::new, + true + ); + } } From c0e19d587643af3b070148344b519ebcf6aa4838 Mon Sep 17 00:00:00 2001 From: weizijun Date: Thu, 21 Apr 2022 20:55:23 +0800 Subject: [PATCH 02/53] add tests and fixed error code --- .../TimeSeriesAggregationsIT.java | 153 ++++++++++++++---- .../elasticsearch/search/SearchModule.java | 4 + .../InternalTimeSeriesAggregation.java | 49 +++--- ...meSeriesAggregationAggregationBuilder.java | 2 + .../TimeSeriesAggregationAggregator.java | 8 + .../aggregation/TimeSeriesAggregations.java | 3 + .../timeseries/aggregation/internal/Last.java | 4 +- .../internal/TSIDInternalAggregation.java | 7 +- 8 files changed, 170 insertions(+), 60 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java index 866fb2e1f470a..aa0b170849529 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java @@ -14,7 +14,9 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.common.Rounding; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.mapper.DateFieldMapper; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; @@ -25,10 +27,12 @@ import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; import org.elasticsearch.search.aggregations.bucket.terms.Terms; import org.elasticsearch.search.aggregations.metrics.CompensatedSum; +import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; import org.elasticsearch.search.aggregations.metrics.Stats; import org.elasticsearch.search.aggregations.metrics.Sum; import org.elasticsearch.search.aggregations.pipeline.SimpleValue; import org.elasticsearch.search.aggregations.timeseries.TimeSeries; +import org.elasticsearch.search.aggregations.timeseries.aggregation.InternalTimeSeriesAggregation; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.xcontent.XContentBuilder; @@ -40,6 +44,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -171,37 +176,6 @@ public void setupSuiteScopeCluster() throws Exception { indexRandom(true, false, docs); } - public void testTimeSeriesAggregations() { - SearchResponse response = client().prepareSearch("index") - .setSize(0) - .addAggregation( - timeSeriesAggregation("by_ts").field("metric_0") - .group(List.of("dim_0")) - .aggregator("sum") - .interval(DateHistogramInterval.minutes(10)) - .downsampleFunction("avg") - .size(1) - ) - .get(); - assertSearchResponse(response); - if (response != null) { - System.out.println(response); - return; - } - Aggregations aggregations = response.getAggregations(); - assertNotNull(aggregations); - TimeSeries timeSeries = aggregations.get("by_ts"); - assertThat( - timeSeries.getBuckets().stream().map(MultiBucketsAggregation.Bucket::getKey).collect(Collectors.toSet()), - equalTo(data.keySet()) - ); - for (TimeSeries.Bucket bucket : timeSeries.getBuckets()) { - @SuppressWarnings("unchecked") - Map key = (Map) bucket.getKey(); - assertThat((long) data.get(key).size(), equalTo(bucket.getDocCount())); - } - } - public void testStandAloneTimeSeriesAgg() { SearchResponse response = client().prepareSearch("index").setSize(0).addAggregation(timeSeries("by_ts")).get(); assertSearchResponse(response); @@ -553,4 +527,121 @@ public void testGetHitsFailure() throws Exception { } + public void testBasicTimeSeriesAggregations() { + DateHistogramInterval fixedInterval = DateHistogramInterval.days(randomIntBetween(10, 100)); + Rounding.Prepared rounding = Rounding.builder(new TimeValue(fixedInterval.estimateMillis())).build().prepareForUnknown(); + SearchResponse response = client().prepareSearch("index") + .setSize(0) + .addAggregation( + timeSeriesAggregation("by_ts").field("metric_0") + .interval(fixedInterval) + .downsampleFunction("sum") + .size(data.size()) + ) + .get(); + Aggregations aggregations = response.getAggregations(); + assertNotNull(aggregations); + InternalTimeSeriesAggregation timeSeries = aggregations.get("by_ts"); + assertThat( + timeSeries.getBuckets().stream().map(MultiBucketsAggregation.Bucket::getKey).collect(Collectors.toSet()), + equalTo(data.keySet()) + ); + for (InternalTimeSeriesAggregation.InternalBucket bucket : timeSeries.getBuckets()) { + Map key = bucket.getKey(); + Map> dataValues = data.get(key); + assertThat((long) dataValues.size(), equalTo(bucket.getDocCount())); + Map dataBucketValues = new HashMap<>(); + dataValues.forEach((timestamp, metrics) -> { + long roundValue = rounding.nextRoundingValue(timestamp); + if (dataBucketValues.containsKey(roundValue)) { + dataBucketValues.put(roundValue, dataBucketValues.get(roundValue) + metrics.get("metric_0")); + } else { + dataBucketValues.put(roundValue, metrics.get("metric_0")); + } + }); + + dataBucketValues.forEach((timestamp, value) -> { + assertTrue(bucket.getTimeBucketValues().containsKey(timestamp)); + InternalAggregation aggregation = bucket.getTimeBucketValues().get(timestamp); + assertTrue(aggregation instanceof InternalNumericMetricsAggregation.SingleValue); + assertThat(((InternalNumericMetricsAggregation.SingleValue)aggregation).value(), closeTo(value, 0.0001d)); + }); + } + } + + public void testTimeSeriesAggregationsGroupBy() { + DateHistogramInterval fixedInterval = DateHistogramInterval.days(randomIntBetween(10, 100)); + Rounding.Prepared rounding = Rounding.builder(new TimeValue(fixedInterval.estimateMillis())).build().prepareForUnknown(); + SearchResponse response = client().prepareSearch("index") + .setSize(0) + .addAggregation( + timeSeriesAggregation("by_ts").field("metric_0") + .group(List.of("dim_0")) + .interval(fixedInterval) + .downsampleFunction("max") + .aggregator("sum") + .size(data.size()) + ) + .get(); + Aggregations aggregations = response.getAggregations(); + assertNotNull(aggregations); + InternalTimeSeriesAggregation timeSeries = aggregations.get("by_ts"); + assertThat( + timeSeries.getBuckets().stream().map(MultiBucketsAggregation.Bucket::getKey).collect(Collectors.toSet()), + equalTo( + data.keySet() + .stream() + .map(key -> key.get("dim_0")) + .collect(Collectors.toSet()) + .stream() + .map(key -> Map.of("dim_0", key)) + .collect(Collectors.toSet()) + ) + ); + + + Map, Map> aggResults = new HashMap<>(); + data.forEach((key, value) -> { + String dim = key.get("dim_0"); + Map bucketKey = Map.of("dim_0", dim); + Map bucketResult = aggResults.get(bucketKey); + if (bucketResult == null) { + bucketResult = new HashMap<>(); + aggResults.put(bucketKey, bucketResult); + } + + Map downsampleResult = new HashMap<>(); + value.forEach((timestamp, metrics) -> { + long roundValue = rounding.nextRoundingValue(timestamp); + if (downsampleResult.containsKey(roundValue)) { + downsampleResult.put(roundValue, Math.max(downsampleResult.get(roundValue), metrics.get("metric_0"))); + } else { + downsampleResult.put(roundValue, metrics.get("metric_0")); + } + }); + + for (Entry entry : downsampleResult.entrySet()) { + Long timestamp = entry.getKey(); + Double downsampleValue = entry.getValue(); + if (bucketResult.containsKey(timestamp)) { + bucketResult.put(timestamp, bucketResult.get(timestamp) + downsampleValue); + } else { + bucketResult.put(timestamp, downsampleValue); + } + } + }); + System.out.println(data); + System.out.println(aggResults); + + for (InternalTimeSeriesAggregation.InternalBucket bucket : timeSeries.getBuckets()) { + Map key = bucket.getKey(); + Map dataValues = aggResults.get(key); + dataValues.forEach((timestamp, metric) -> { + assertTrue(bucket.getTimeBucketValues().containsKey(timestamp)); + InternalAggregation aggregation = bucket.getTimeBucketValues().get(timestamp); + assertTrue(aggregation instanceof InternalNumericMetricsAggregation.SingleValue); + assertThat(((InternalNumericMetricsAggregation.SingleValue)aggregation).value(), closeTo(metric, 0.0001d)); + }); + } + } } diff --git a/server/src/main/java/org/elasticsearch/search/SearchModule.java b/server/src/main/java/org/elasticsearch/search/SearchModule.java index 13765834f0611..4296ec6f66e04 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchModule.java +++ b/server/src/main/java/org/elasticsearch/search/SearchModule.java @@ -217,6 +217,8 @@ import org.elasticsearch.search.aggregations.timeseries.TimeSeriesAggregationBuilder; import org.elasticsearch.search.aggregations.timeseries.aggregation.InternalTimeSeriesAggregation; import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregationAggregationBuilder; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.Last; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TSIDInternalAggregation; import org.elasticsearch.search.fetch.FetchPhase; import org.elasticsearch.search.fetch.FetchSubPhase; import org.elasticsearch.search.fetch.subphase.ExplainPhase; @@ -673,6 +675,8 @@ private ValuesSourceRegistry registerAggregations(List plugins) { TimeSeriesAggregationAggregationBuilder::new, TimeSeriesAggregationAggregationBuilder.PARSER ).addResultReader(InternalTimeSeriesAggregation::new) + .addResultReader(Last.NAME, Last::new) + .addResultReader(TSIDInternalAggregation.NAME, TSIDInternalAggregation::new) .setAggregatorRegistrar(TimeSeriesAggregationAggregationBuilder::registerAggregators), builder ); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregation.java index 23476e435bc80..fa60edf025ded 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregation.java @@ -18,7 +18,6 @@ import org.elasticsearch.search.aggregations.KeyComparable; import org.elasticsearch.search.aggregations.bucket.terms.AbstractInternalTerms; import org.elasticsearch.search.aggregations.timeseries.aggregation.InternalTimeSeriesAggregation.InternalBucket; -import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TSIDInternalAggregation; import org.elasticsearch.xcontent.ObjectParser; import org.elasticsearch.xcontent.XContentBuilder; @@ -122,6 +121,10 @@ public long getDocCount() { return docCount; } + public Map getTimeBucketValues() { + return timeBucketValues; + } + @Override public InternalAggregations getAggregations() { return aggregations; @@ -420,8 +423,8 @@ protected int getRequiredSize() { public InternalBucket reduceBucket(List buckets, AggregationReduceContext context) { InternalBucket reduced = null; List aggregationsList = new ArrayList<>(buckets.size()); - Map tsidInternalAggs = new HashMap<>(); - Map timeBucketValues = new TreeMap<>(); + Map timeBucketResults = new TreeMap<>(); + Map> timeBucketAggregationsList = new TreeMap<>(); long docCountError = 0; for (InternalBucket bucket : buckets) { if (docCountError != -1) { @@ -441,35 +444,29 @@ public InternalBucket reduceBucket(List buckets, AggregationRedu bucket.showDocCountError, docCountError ); - timeBucketValues.putAll(bucket.timeBucketValues); } else { reduced.docCount += bucket.docCount; - for (Entry entry : bucket.timeBucketValues.entrySet()) { - Long timestamp = entry.getKey(); - InternalAggregation value = entry.getValue(); - InternalAggregation aggregation = timeBucketValues.get(timestamp); - if (aggregation == null) { - timeBucketValues.put(timestamp, value); - if (value instanceof TSIDInternalAggregation) { - tsidInternalAggs.put(timestamp, value); - } - } else { - InternalAggregation result = aggregation.reduce(List.of(value), context); - if (result instanceof TSIDInternalAggregation) { - tsidInternalAggs.put(timestamp, result); - } else { - tsidInternalAggs.remove(timestamp); - } - timeBucketValues.put(timestamp, result); - } + } + for (Entry entry : bucket.timeBucketValues.entrySet()) { + Long timestamp = entry.getKey(); + InternalAggregation value = entry.getValue(); + List values = timeBucketAggregationsList.get(timestamp); + if (values == null) { + values = new ArrayList<>(); + timeBucketAggregationsList.put(timestamp, values); } + values.add(value); } aggregationsList.add(bucket.aggregations); } - if (context.isFinalReduce()) { - tsidInternalAggs.forEach((timestamp, value) -> { timeBucketValues.put(timestamp, value.reduce(List.of(value), context)); }); - } - reduced.timeBucketValues = timeBucketValues; + + timeBucketAggregationsList.forEach((timestamp, aggs) -> { + if (aggs.size() > 0) { + InternalAggregation first = aggs.get(0); + timeBucketResults.put(timestamp, first.reduce(aggs, context)); + } + }); + reduced.timeBucketValues = timeBucketResults; reduced.docCountError = docCountError; if (reduced.docCountError == -1) { reduced.showDocCountError = false; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java index efcf3e7e8ebab..bcddaba75ff4f 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java @@ -153,6 +153,7 @@ public TimeSeriesAggregationAggregationBuilder(StreamInput in) throws IOExceptio downsampleRange = in.readOptionalWriteable(DateHistogramInterval::new); downsampleFunction = in.readOptionalString(); order = InternalOrder.Streams.readOrder(in); + bucketCountThresholds = new TermsAggregator.BucketCountThresholds(in); } @Override @@ -166,6 +167,7 @@ protected void innerWriteTo(StreamOutput out) throws IOException { out.writeOptionalWriteable(downsampleRange); out.writeOptionalString(downsampleFunction); order.writeTo(out); + bucketCountThresholds.writeTo(out); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index 18fafb1d4974c..d2766aa1cadaa 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -390,6 +390,14 @@ public void collectTimeSeriesValues(long bucketOrd) throws IOException { for (Entry entry : timeBucketMetrics.entrySet()) { Long timestamp = entry.getKey(); AggregatorFunction value = entry.getValue(); + logger.info( + "collect time_series, time={}, value={}, tsid={}, hashcode={}", + timestamp, + value.get(), + TimeSeriesIdFieldMapper.decodeTsid(preTsid), + this.hashCode() + ); + long ord = timestampOrds.add(bucketOrd, timestamp); if (ord < 0) { // already seen ord = -1 - ord; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregations.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregations.java index 542489636c1fd..9e575803de714 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregations.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregations.java @@ -17,6 +17,7 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.SumBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AvgFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.LastFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MaxFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MinFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.SumFunction; @@ -35,6 +36,8 @@ public static AggregatorFunction getAggregatorFunction(TimeSeriesAggregation.Fun return new SumFunction(); case count: return new ValueCountFunction(); + case last: + return new LastFunction(); } return new AvgFunction(); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java index 5949db561a9cc..78d7148a8648d 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java @@ -23,6 +23,8 @@ import org.elasticsearch.xcontent.XContentBuilder; public class Last extends InternalNumericMetricsAggregation.SingleValue { + public static final String NAME = "last"; + private final double last; private final long timestamp; @@ -50,7 +52,7 @@ protected void doWriteTo(StreamOutput out) throws IOException { @Override public String getWriteableName() { - return "last"; + return NAME; } public InternalAggregation finalizeSampling(SamplingContext samplingContext) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java index 72467676cb1b7..d86ed89b3aa5a 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java @@ -27,6 +27,8 @@ import java.util.TreeMap; public class TSIDInternalAggregation extends InternalAggregation { + public static final String NAME = "tsid"; + private final Map values; private final String aggreagator; private final DocValueFormat formatter; @@ -54,7 +56,7 @@ public TSIDInternalAggregation(StreamInput in) throws IOException { @Override public String getWriteableName() { - return "tsid"; + return NAME; } @Override @@ -127,6 +129,7 @@ public Object getProperty(List path) { @Override public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { - return null; + builder.field(CommonFields.VALUE.getPreferredName(), aggreagator); + return builder; } } From 8503e1887d65b13d944857deae6aa72971b38e2c Mon Sep 17 00:00:00 2001 From: weizijun Date: Fri, 22 Apr 2022 09:31:43 +0800 Subject: [PATCH 03/53] add comments --- .../TimeSeriesAggregationsIT.java | 5 +- .../InternalTimeSeriesAggregation.java | 18 +++++++ ...meSeriesAggregationAggregationBuilder.java | 48 +++++++++++++++++++ .../TimeSeriesAggregationAggregator.java | 42 +++++++++++----- .../AggregatorBucketFunction.java | 24 +++++++++- .../bucketfunction/TSIDBucketFunction.java | 14 ++++++ .../function/AggregatorFunction.java | 18 +++++++ 7 files changed, 150 insertions(+), 19 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java index aa0b170849529..916efeb1ba8d0 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java @@ -77,7 +77,7 @@ public class TimeSeriesAggregationsIT extends ESIntegTestCase { @Override public void setupSuiteScopeCluster() throws Exception { - int numberOfIndices = 1; //randomIntBetween(1, 3); + int numberOfIndices = randomIntBetween(1, 3); numberOfDimensions = randomIntBetween(1, 5); numberOfMetrics = randomIntBetween(1, 10); String[] routingKeys = randomSubsetOf( @@ -599,7 +599,6 @@ public void testTimeSeriesAggregationsGroupBy() { ) ); - Map, Map> aggResults = new HashMap<>(); data.forEach((key, value) -> { String dim = key.get("dim_0"); @@ -630,8 +629,6 @@ public void testTimeSeriesAggregationsGroupBy() { } } }); - System.out.println(data); - System.out.println(aggResults); for (InternalTimeSeriesAggregation.InternalBucket bucket : timeSeries.getBuckets()) { Map key = bucket.getKey(); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregation.java index fa60edf025ded..2c841ad9054cc 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregation.java @@ -8,6 +8,8 @@ package org.elasticsearch.search.aggregations.timeseries.aggregation; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.search.aggregations.AggregationReduceContext; @@ -38,6 +40,8 @@ public class InternalTimeSeriesAggregation extends AbstractInternalTerms PARSER = new ObjectParser<>( ParsedTimeSeriesAggregation.class.getSimpleName(), true, @@ -419,6 +423,20 @@ protected int getRequiredSize() { return requiredSize; } + @Override + public InternalAggregation reduce(List aggregations, AggregationReduceContext reduceContext) { + long startTime = System.currentTimeMillis(); + InternalAggregation reduce = super.reduce(aggregations, reduceContext); + if (logger.isTraceEnabled()) { + logger.trace( + "time series reduce the current aggregations, isFinal [{}], cost [{}]", + reduceContext.isFinalReduce(), + (System.currentTimeMillis() - startTime) + ); + } + return reduce; + } + @Override public InternalBucket reduceBucket(List buckets, AggregationReduceContext context) { InternalBucket reduced = null; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java index bcddaba75ff4f..313d4100c648d 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java @@ -275,72 +275,120 @@ public boolean isInSortOrderExecutionRequired() { return true; } + /** + * Returns the keyed value + */ public boolean isKeyed() { return keyed; } + /** + * Sets if keyed for the results. + */ public void setKeyed(boolean keyed) { this.keyed = keyed; } + /** + * Returns the group values + */ public List getGroup() { return group; } + /** + * Sets the group values, it used to include dimension fields + */ public TimeSeriesAggregationAggregationBuilder group(List group) { this.group = group; return this; } + /** + * Returns the without values + */ public List getWithout() { return without; } + /** + * Sets the without values, it used to exclude dimension fields + */ public TimeSeriesAggregationAggregationBuilder without(List without) { this.without = without; return this; } + /** + * Return the interval value + */ public DateHistogramInterval getInterval() { return interval; } + /** + * Sets the interval value + */ public TimeSeriesAggregationAggregationBuilder interval(DateHistogramInterval interval) { this.interval = interval; return this; } + /** + * Return the offset value + */ public DateHistogramInterval getOffset() { return offset; } + /** + * Sets the offset value + */ public TimeSeriesAggregationAggregationBuilder offset(DateHistogramInterval offset) { this.offset = offset; return this; } + /** + * Returns the aggregator function + */ public String getAggregator() { return aggregator; } + /** + * Sets the aggregator function, it used to aggregator time series lines to one time serie line + */ public TimeSeriesAggregationAggregationBuilder aggregator(String aggregator) { this.aggregator = aggregator; return this; } + /** + * Returns the downsample range value + */ public DateHistogramInterval getDownsampleRange() { return downsampleRange; } + /** + * Set the downsample range interval, if downsample range not set, the downsample range value is replace by interval + */ public TimeSeriesAggregationAggregationBuilder downsampleRange(DateHistogramInterval downsampleRange) { this.downsampleRange = downsampleRange; return this; } + /** + * Returns the downsample function + */ public String getDownsampleFunction() { return downsampleFunction; } + /** + * Sets the downsample function + */ public TimeSeriesAggregationAggregationBuilder downsampleFunction(String downsampleFunction) { this.downsampleFunction = downsampleFunction; return this; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index d2766aa1cadaa..45b367438e98b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -65,8 +65,6 @@ public class TimeSeriesAggregationAggregator extends BucketsAggregator { protected BytesKeyedBucketOrds bucketOrds; private LongKeyedBucketOrds timestampOrds; private ValuesSource.Numeric valuesSource; - private Map> groupBucketValues; // TODO replace map - private Map aggregatorCollectors; // TODO replace map private boolean keyed; @@ -89,6 +87,8 @@ public class TimeSeriesAggregationAggregator extends BucketsAggregator { private Rounding.Prepared rounding; private boolean needGroupBy; private Map timeBucketMetrics; // TODO replace map + private Map> groupBucketValues; // TODO replace map + private Map aggregatorCollectors; // TODO replace map @SuppressWarnings("unchecked") public TimeSeriesAggregationAggregator( @@ -245,19 +245,19 @@ public class Collector extends LeafBucketCollectorBase { final SortedNumericDoubleValues values; final AggregationExecutionContext aggCtx; final LeafBucketCollector sub; - final CheckedConsumer consumer; + final CheckedConsumer docConsumer; public Collector( LeafBucketCollector sub, SortedNumericDoubleValues values, AggregationExecutionContext aggCtx, - CheckedConsumer consumer + CheckedConsumer docConsumer ) { super(sub, values); this.sub = sub; this.values = values; this.aggCtx = aggCtx; - this.consumer = consumer; + this.docConsumer = docConsumer; } @Override @@ -274,7 +274,9 @@ public void collect(int doc, long bucket) throws IOException { if (preRounding < 0 || aggCtx.getTimestamp() <= preRounding - interval) { preRounding = rounding.nextRoundingValue(aggCtx.getTimestamp()); } - consumer.accept(doc); + + // calculate the value of the current doc + docConsumer.accept(doc); collectBucket(sub, doc, preBucketOrdinal); } @@ -299,6 +301,9 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBu throw new UnsupportedOperationException("Shouldn't be here"); } + /** + * get the downsample function of the current rouding + */ protected AggregatorFunction getAggregatorFunction() throws IOException { AggregatorFunction function = timeBucketMetrics.get(preRounding); if (function == null) { @@ -332,6 +337,9 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBu @Override protected void doPostCollection() throws IOException { + /** + * collect the last tsid + */ if (timeBucketMetrics != null && timeBucketMetrics.size() > 0) { collectTimeSeriesValues(preBucketOrdinal); } @@ -361,6 +369,9 @@ InternalTimeSeriesAggregation buildResult(long otherDocCount, InternalTimeSeries ); } + /** + * decode the tsid and pack the bucket key from the group and without config + */ private BytesRef packKey(BytesRef tsid) { Map tsidMap = TimeSeriesIdFieldMapper.decodeTsid(tsid); Map groupMap = new LinkedHashMap<>(); @@ -378,6 +389,9 @@ private BytesRef packKey(BytesRef tsid) { return TIME_SERIES_ID.parseBytesRef(groupMap); } + /** + * collect the value of one time series line + */ public void collectTimeSeriesValues(long bucketOrd) throws IOException { if (needGroupBy) { AggregatorBucketFunction aggregatorBucketFunction = aggregatorCollectors.get(bucketOrd); @@ -390,13 +404,15 @@ public void collectTimeSeriesValues(long bucketOrd) throws IOException { for (Entry entry : timeBucketMetrics.entrySet()) { Long timestamp = entry.getKey(); AggregatorFunction value = entry.getValue(); - logger.info( - "collect time_series, time={}, value={}, tsid={}, hashcode={}", - timestamp, - value.get(), - TimeSeriesIdFieldMapper.decodeTsid(preTsid), - this.hashCode() - ); + if (logger.isTraceEnabled()) { + logger.trace( + "collect time_series, time={}, value={}, tsid={}, hashcode={}", + timestamp, + value.get(), + TimeSeriesIdFieldMapper.decodeTsid(preTsid), + this.hashCode() + ); + } long ord = timestampOrds.add(bucketOrd, timestamp); if (ord < 0) { // already seen diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AggregatorBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AggregatorBucketFunction.java index 3d8edcac2d044..a8875a4eb2138 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AggregatorBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AggregatorBucketFunction.java @@ -8,17 +8,37 @@ package org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction; -import java.util.Map; - import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; +import java.util.Map; + +/** + * time series aggregator function interface + * Different from {@link AggregatorFunction}, this interface need to pass a bucket, which is the key of the aggregator + * @param the input type of value + */ public interface AggregatorBucketFunction { + /** + * + * @return + */ String name(); + /** + * collect value of the bucket + */ void collect(Input number, long bucket); + /** + * get the {@link InternalAggregation}, it used to transport cross nodes and reduce result in the coordinate node + * @return the {@link InternalAggregation} of the input bucket + */ InternalAggregation getAggregation(long bucket, DocValueFormat formatter, Map metadata); + /** + * close the inner big array + */ void close(); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java index ce77e32fcb8c9..817fdc5559e64 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java @@ -17,6 +17,20 @@ import java.util.HashMap; import java.util.Map; +/** + * The function is used to aggregator time series lines in the coordinate reduce phase. + * The _tsid may be exist in many indices, when the bucket ranges will flowout the range of the index, + * it may be exist + * e.g a index settings and query config is:
    + *
  • time_series.start_time = 10 + *
  • time_series.end_time = 20 + *
  • interval = 2 + *
+ * When the bucket range is 11-13, the bucket must only in the index. + * But if the bucket range is 9-11, the bucket may be include other index, so the aggregator function + * can't compute in the datanode. the tsid bucket function gather all _tsid and the value, and aggregator + * the result in the coordinate reduce phase. + */ public class TSIDBucketFunction implements AggregatorBucketFunction { private Map> values = new HashMap<>(); private final AggregatorBucketFunction aggregatorBucketFunction; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AggregatorFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AggregatorFunction.java index f3da28d85e951..551c7fd777d12 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AggregatorFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AggregatorFunction.java @@ -13,10 +13,28 @@ import java.util.Map; +/** + * time series aggregator function interface + * @param the input type of value + * @param the output type of the function + */ public interface AggregatorFunction { + /** + * collect value + * @param value input value + */ void collect(Input value); + /** + * get the result of aggregator function + */ Output get(); + /** + * get the {@link InternalAggregation}, it used to transport cross nodes and reduce result in the coordinate node + * @param formatter the value formatter + * @param metadata aggregation metadata + * @return the {@link InternalAggregation} of aggregator function + */ InternalAggregation getAggregation(DocValueFormat formatter, Map metadata); } From 6133e425d1e7b6145aaa46e5dcb52199bbfa32dc Mon Sep 17 00:00:00 2001 From: weizijun Date: Fri, 22 Apr 2022 10:39:25 +0800 Subject: [PATCH 04/53] spotless --- .../TimeSeriesAggregationsIT.java | 9 +++---- .../ParsedTimeSeriesAggregation.java | 14 ++++------- .../TimeSeriesAggregationAggregator.java | 1 + .../aggregation/TimeSeriesAggregations.java | 7 ++++-- .../bucketfunction/TSIDBucketFunction.java | 3 ++- .../timeseries/aggregation/internal/Last.java | 24 ++++++++++++------- .../internal/TSIDInternalAggregation.java | 7 +++++- ...MetricTimeSeriesAggregationAggregator.java | 4 +--- 8 files changed, 38 insertions(+), 31 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java index 916efeb1ba8d0..bb977212eddf4 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java @@ -533,10 +533,7 @@ public void testBasicTimeSeriesAggregations() { SearchResponse response = client().prepareSearch("index") .setSize(0) .addAggregation( - timeSeriesAggregation("by_ts").field("metric_0") - .interval(fixedInterval) - .downsampleFunction("sum") - .size(data.size()) + timeSeriesAggregation("by_ts").field("metric_0").interval(fixedInterval).downsampleFunction("sum").size(data.size()) ) .get(); Aggregations aggregations = response.getAggregations(); @@ -564,7 +561,7 @@ public void testBasicTimeSeriesAggregations() { assertTrue(bucket.getTimeBucketValues().containsKey(timestamp)); InternalAggregation aggregation = bucket.getTimeBucketValues().get(timestamp); assertTrue(aggregation instanceof InternalNumericMetricsAggregation.SingleValue); - assertThat(((InternalNumericMetricsAggregation.SingleValue)aggregation).value(), closeTo(value, 0.0001d)); + assertThat(((InternalNumericMetricsAggregation.SingleValue) aggregation).value(), closeTo(value, 0.0001d)); }); } } @@ -637,7 +634,7 @@ public void testTimeSeriesAggregationsGroupBy() { assertTrue(bucket.getTimeBucketValues().containsKey(timestamp)); InternalAggregation aggregation = bucket.getTimeBucketValues().get(timestamp); assertTrue(aggregation instanceof InternalNumericMetricsAggregation.SingleValue); - assertThat(((InternalNumericMetricsAggregation.SingleValue)aggregation).value(), closeTo(metric, 0.0001d)); + assertThat(((InternalNumericMetricsAggregation.SingleValue) aggregation).value(), closeTo(metric, 0.0001d)); }); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/ParsedTimeSeriesAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/ParsedTimeSeriesAggregation.java index 5edc35e84ebd7..b1bfa6235fbe6 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/ParsedTimeSeriesAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/ParsedTimeSeriesAggregation.java @@ -69,10 +69,8 @@ public TimeSeriesAggregation.Bucket getBucketByKey(String key) { parser -> ParsedTimeSeriesAggregation.ParsedBucket.fromXContent(parser, false), parser -> ParsedTimeSeriesAggregation.ParsedBucket.fromXContent(parser, true) ); - PARSER.declareLong((parsedTerms, value) -> parsedTerms.docCountErrorUpperBound = value, - DOC_COUNT_ERROR_UPPER_BOUND_FIELD_NAME); - PARSER.declareLong((parsedTerms, value) -> parsedTerms.sumOtherDocCount = value, - SUM_OF_OTHER_DOC_COUNTS); + PARSER.declareLong((parsedTerms, value) -> parsedTerms.docCountErrorUpperBound = value, DOC_COUNT_ERROR_UPPER_BOUND_FIELD_NAME); + PARSER.declareLong((parsedTerms, value) -> parsedTerms.sumOtherDocCount = value, SUM_OF_OTHER_DOC_COUNTS); } protected long docCountErrorUpperBound; @@ -158,8 +156,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws return builder; } - static ParsedTimeSeriesAggregation.ParsedBucket fromXContent(XContentParser parser, boolean keyed) - throws IOException { + static ParsedTimeSeriesAggregation.ParsedBucket fromXContent(XContentParser parser, boolean keyed) throws IOException { ParsedTimeSeriesAggregation.ParsedBucket bucket = new ParsedBucket(); bucket.setKeyed(keyed); String currentFieldName = parser.currentName(); @@ -170,7 +167,7 @@ static ParsedTimeSeriesAggregation.ParsedBucket fromXContent(XContentParser pars ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.nextToken(), parser); } - List aggregations = new ArrayList(); + List aggregations = new ArrayList<>(); while ((token = parser.nextToken()) != Token.END_OBJECT) { if (token == Token.FIELD_NAME) { currentFieldName = parser.currentName(); @@ -179,8 +176,7 @@ static ParsedTimeSeriesAggregation.ParsedBucket fromXContent(XContentParser pars bucket.setKeyAsString(parser.text()); } else if (CommonFields.DOC_COUNT.getPreferredName().equals(currentFieldName)) { bucket.setDocCount(parser.longValue()); - } else if (InternalTerms.DOC_COUNT_ERROR_UPPER_BOUND_FIELD_NAME.getPreferredName().equals( - currentFieldName)) { + } else if (InternalTerms.DOC_COUNT_ERROR_UPPER_BOUND_FIELD_NAME.getPreferredName().equals(currentFieldName)) { bucket.docCountError = parser.longValue(); bucket.showDocCountError = true; } else if (CommonFields.VALUES.getPreferredName().equals(currentFieldName)) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index 45b367438e98b..01aefe57f6284 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -59,6 +59,7 @@ import static org.elasticsearch.search.DocValueFormat.TIME_SERIES_ID; import static org.elasticsearch.search.aggregations.InternalOrder.isKeyOrder; +@SuppressWarnings({ "unchecked", "rawtypes" }) public class TimeSeriesAggregationAggregator extends BucketsAggregator { private static final Logger logger = LogManager.getLogger(TimeSeriesAggregationAggregator.class); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregations.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregations.java index 9e575803de714..a0a4b8397daa3 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregations.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregations.java @@ -24,7 +24,7 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.function.ValueCountFunction; public class TimeSeriesAggregations { - public static AggregatorFunction getAggregatorFunction(TimeSeriesAggregation.Function function) { + public static AggregatorFunction getAggregatorFunction(TimeSeriesAggregation.Function function) { switch (function) { case avg: return new AvgFunction(); @@ -42,7 +42,10 @@ public static AggregatorFunction getAggregatorFunction(TimeSeriesAggregation.Fun return new AvgFunction(); } - public static AggregatorBucketFunction getAggregatorBucketFunction(TimeSeriesAggregation.Aggregator aggregator, BigArrays bigArrays) { + public static AggregatorBucketFunction getAggregatorBucketFunction( + TimeSeriesAggregation.Aggregator aggregator, + BigArrays bigArrays + ) { if (aggregator == null) { return new NoAggregatorBucketFunction(); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java index 817fdc5559e64..08ce41150cff5 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java @@ -19,7 +19,7 @@ /** * The function is used to aggregator time series lines in the coordinate reduce phase. - * The _tsid may be exist in many indices, when the bucket ranges will flowout the range of the index, + * The _tsid may be exist in many indices, when the bucket ranges will overflow the range of the index, * it may be exist * e.g a index settings and query config is:
    *
  • time_series.start_time = 10 @@ -31,6 +31,7 @@ * can't compute in the datanode. the tsid bucket function gather all _tsid and the value, and aggregator * the result in the coordinate reduce phase. */ +@SuppressWarnings({ "unchecked", "rawtypes" }) public class TSIDBucketFunction implements AggregatorBucketFunction { private Map> values = new HashMap<>(); private final AggregatorBucketFunction aggregatorBucketFunction; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java index 78d7148a8648d..e458f9856d7f2 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java @@ -8,11 +8,6 @@ package org.elasticsearch.search.aggregations.timeseries.aggregation.internal; -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.Objects; - import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.search.DocValueFormat; @@ -22,6 +17,11 @@ import org.elasticsearch.search.aggregations.support.SamplingContext; import org.elasticsearch.xcontent.XContentBuilder; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Objects; + public class Last extends InternalNumericMetricsAggregation.SingleValue { public static final String NAME = "last"; @@ -88,10 +88,16 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th @Override public boolean equals(Object o) { - if (this == o) {return true;} - if (o == null || getClass() != o.getClass()) {return false;} - if (!super.equals(o)) {return false;} - Last last1 = (Last)o; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + Last last1 = (Last) o; return Double.compare(last1.last, last) == 0 && timestamp == last1.timestamp; } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java index d86ed89b3aa5a..a04858002eb14 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java @@ -18,6 +18,7 @@ import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation.SingleValue; import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregation; import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregations; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.TSIDBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; import org.elasticsearch.xcontent.XContentBuilder; @@ -26,6 +27,11 @@ import java.util.Map; import java.util.TreeMap; +/** + * The tsid internal aggregation is used to aggregator the tsid values in the coordinate reduce phase. + * The {@link TSIDBucketFunction} collect the data, and this internal aggregation compute the reduce result. + */ +@SuppressWarnings({ "unchecked", "rawtypes" }) public class TSIDInternalAggregation extends InternalAggregation { public static final String NAME = "tsid"; @@ -51,7 +57,6 @@ public TSIDInternalAggregation(StreamInput in) throws IOException { formatter = in.readNamedWriteable(DocValueFormat.class); aggreagator = in.readString(); values = in.readOrderedMap(StreamInput::readBytesRef, stream -> stream.readNamedWriteable(InternalAggregation.class)); - } @Override diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java index e8fd9c3cff78c..70aa2f034c585 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java @@ -19,10 +19,7 @@ import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; -import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregation; -import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregation.Avg; import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregation.Function; -import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregation.ValueCount; import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregationAggregator; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AvgFunction; @@ -34,6 +31,7 @@ import java.util.List; import java.util.Map; +@SuppressWarnings({ "unchecked", "rawtypes" }) public class AggregateMetricTimeSeriesAggregationAggregator extends TimeSeriesAggregationAggregator { private final AggregateMetricsValuesSource.AggregateDoubleMetric valuesSource; From 7255f4aaa0f1418d419a7be119e9f47e65dadcdf Mon Sep 17 00:00:00 2001 From: weizijun Date: Fri, 22 Apr 2022 19:35:26 +0800 Subject: [PATCH 05/53] optimize code --- .../TimeSeriesAggregationsIT.java | 8 +- .../timeseries/aggregation/Downsample.java | 90 ++++++++++++++ .../timeseries/aggregation/Function.java | 106 +++++++++++++++++ .../InternalTimeSeriesAggregation.java | 5 +- .../aggregation/TimeSeriesAggregation.java | 29 ----- ...meSeriesAggregationAggregationBuilder.java | 110 ++++++------------ ...meSeriesAggregationAggregationFactory.java | 19 ++- .../TimeSeriesAggregationAggregator.java | 24 ++-- ...meSeriesAggregationAggregatorSupplier.java | 5 +- .../aggregation/TimeSeriesAggregations.java | 65 ----------- .../timeseries/aggregation/internal/Last.java | 2 +- .../internal/TSIDInternalAggregation.java | 11 +- .../InternalTimeSeriesAggregationTests.java | 6 +- ...iesAggregationAggregationBuilderTests.java | 2 +- ...MetricTimeSeriesAggregationAggregator.java | 11 +- 15 files changed, 279 insertions(+), 214 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java delete mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregations.java diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java index bb977212eddf4..65660fed93850 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java @@ -32,6 +32,7 @@ import org.elasticsearch.search.aggregations.metrics.Sum; import org.elasticsearch.search.aggregations.pipeline.SimpleValue; import org.elasticsearch.search.aggregations.timeseries.TimeSeries; +import org.elasticsearch.search.aggregations.timeseries.aggregation.Function; import org.elasticsearch.search.aggregations.timeseries.aggregation.InternalTimeSeriesAggregation; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.test.ESIntegTestCase; @@ -533,7 +534,10 @@ public void testBasicTimeSeriesAggregations() { SearchResponse response = client().prepareSearch("index") .setSize(0) .addAggregation( - timeSeriesAggregation("by_ts").field("metric_0").interval(fixedInterval).downsampleFunction("sum").size(data.size()) + timeSeriesAggregation("by_ts").field("metric_0") + .interval(fixedInterval) + .downsample(fixedInterval, Function.sum) + .size(data.size()) ) .get(); Aggregations aggregations = response.getAggregations(); @@ -575,7 +579,7 @@ public void testTimeSeriesAggregationsGroupBy() { timeSeriesAggregation("by_ts").field("metric_0") .group(List.of("dim_0")) .interval(fixedInterval) - .downsampleFunction("max") + .downsample(fixedInterval, Function.max) .aggregator("sum") .size(data.size()) ) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java new file mode 100644 index 0000000000000..561ed67b0b0d1 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java @@ -0,0 +1,90 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; +import org.elasticsearch.xcontent.ConstructingObjectParser; +import org.elasticsearch.xcontent.ObjectParser; +import org.elasticsearch.xcontent.ParseField; +import org.elasticsearch.xcontent.ToXContentObject; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentParser; + +import java.io.IOException; + +public class Downsample implements ToXContentObject, Writeable { + + public static final String NAME = "downsample"; + public static final ParseField RANGE_FIELD = new ParseField("range"); + public static final ParseField FUNCTION_FIELD = new ParseField("function"); + + public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + NAME, + false, + (args, name) -> new Downsample((DateHistogramInterval) args[0], (Function) args[1]) + ); + + static { + PARSER.declareField( + ConstructingObjectParser.constructorArg(), + p -> new DateHistogramInterval(p.text()), + RANGE_FIELD, + ObjectParser.ValueType.STRING + ); + PARSER.declareField( + ConstructingObjectParser.constructorArg(), + p -> Function.resolve(p.text()), + FUNCTION_FIELD, + ObjectParser.ValueType.STRING + ); + } + + private final DateHistogramInterval range; + private final Function function; + + public Downsample(DateHistogramInterval range, Function function) { + this.range = range; + this.function = function; + } + + public Downsample(StreamInput in) throws IOException { + this.range = new DateHistogramInterval(in); + this.function = Function.resolve(in.readString()); + } + + public DateHistogramInterval getRange() { + return range; + } + + public Function getFunction() { + return function; + } + + public static Downsample fromXContent(XContentParser parser) { + return PARSER.apply(parser, null); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(RANGE_FIELD.getPreferredName(), range); + builder.field(FUNCTION_FIELD.getPreferredName(), function); + builder.endObject(); + return builder; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + range.writeTo(out); + out.writeString(function.name()); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java new file mode 100644 index 0000000000000..d0349e2c9e8da --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java @@ -0,0 +1,106 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.AggregatorBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.AvgBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.MaxBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.MinBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.SumBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AvgFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.LastFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MaxFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MinFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.SumFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.ValueCountFunction; + +public enum Function { + count { + @Override + public AggregatorFunction getAggregatorFunction() { + return new ValueCountFunction(); + } + + @Override + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { + throw new UnsupportedOperationException("count aggregator bucket function not support"); + } + }, + sum { + @Override + public AggregatorFunction getAggregatorFunction() { + return new SumFunction(); + } + + @Override + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { + return new SumBucketFunction(bigArrays); + } + }, + min { + @Override + public AggregatorFunction getAggregatorFunction() { + return new MinFunction(); + } + + @Override + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { + return new MinBucketFunction(bigArrays); + } + }, + max { + @Override + public AggregatorFunction getAggregatorFunction() { + return new MaxFunction(); + } + + @Override + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { + return new MaxBucketFunction(bigArrays); + } + }, + avg { + @Override + public AggregatorFunction getAggregatorFunction() { + return new AvgFunction(); + } + + @Override + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { + return new AvgBucketFunction(bigArrays); + } + }, + last { + @Override + public AggregatorFunction getAggregatorFunction() { + return new LastFunction(); + } + + @Override + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { + throw new UnsupportedOperationException("last aggregator bucket function not support"); + } + }; + + public static Function resolve(String name) { + return Function.valueOf(name); + } + + /** + * get the aggregator function + */ + public abstract AggregatorFunction getAggregatorFunction(); + + /** + * get the aggregator bucket function + */ + public abstract AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays); +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregation.java index 2c841ad9054cc..d5a1a352b0fac 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregation.java @@ -94,7 +94,7 @@ public InternalBucket(StreamInput in, boolean keyed, boolean showDocCountError) this.showDocCountError = showDocCountError; docCountError = -1; if (showDocCountError) { - docCountError = in.readLong(); + // docCountError = in.readLong(); } aggregations = InternalAggregations.readFrom(in); } @@ -105,7 +105,8 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVLong(docCount); out.writeMap(timeBucketValues, StreamOutput::writeLong, StreamOutput::writeNamedWriteable); if (showDocCountError) { - out.writeLong(docCountError); + // TODO recover -Dtests.seed=142C4BE4C242FF8B + // out.writeLong(docCountError); } aggregations.writeTo(out); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregation.java index b88b61d8bcb16..9ecd4c0364a22 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregation.java @@ -26,33 +26,4 @@ interface Bucket extends MultiBucketsAggregation.Bucket {} List getBuckets(); Bucket getBucketByKey(String key); - - class Downsample { - long range; - Function function; - } - - enum Function { - count, - sum, - min, - max, - avg, - last; - - public static Function resolve(String name) { - return Function.valueOf(name); - } - } - - enum Aggregator { - sum, - min, - max, - avg; - - public static Aggregator resolve(String name) { - return Aggregator.valueOf(name); - } - } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java index 313d4100c648d..44f5e5777ba26 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java @@ -44,8 +44,6 @@ public class TimeSeriesAggregationAggregationBuilder extends ValuesSourceAggrega public static final ParseField OFFSET_FIELD = new ParseField("offset"); public static final ParseField AGGREGATOR_FIELD = new ParseField("aggregator"); public static final ParseField DOWNSAMPLE_FIELD = new ParseField("downsample"); - public static final ParseField DOWNSAMPLE_RANGE_FIELD = new ParseField("range"); - public static final ParseField DOWNSAMPLE_FUNCTION_FIELD = new ParseField("function"); public static final ParseField ORDER_FIELD = new ParseField("order"); public static final ParseField SIZE_FIELD = new ParseField("size"); public static final ParseField SHARD_SIZE_FIELD = new ParseField("shard_size"); @@ -75,12 +73,11 @@ public class TimeSeriesAggregationAggregationBuilder extends ValuesSourceAggrega private DateHistogramInterval interval; private DateHistogramInterval offset; private String aggregator; - private DateHistogramInterval downsampleRange; - private String downsampleFunction; + private Downsample downsample; private TermsAggregator.BucketCountThresholds bucketCountThresholds = new TermsAggregator.BucketCountThresholds( DEFAULT_BUCKET_COUNT_THRESHOLDS ); - private BucketOrder order = BucketOrder.compound(BucketOrder.key(true)); + private BucketOrder order = BucketOrder.key(true); static { ValuesSourceAggregationBuilder.declareFields(PARSER, false, true, false); @@ -103,13 +100,7 @@ public class TimeSeriesAggregationAggregationBuilder extends ValuesSourceAggrega PARSER.declareStringArray(TimeSeriesAggregationAggregationBuilder::group, GROUP_FIELD); PARSER.declareStringArray(TimeSeriesAggregationAggregationBuilder::without, WITHOUT_FIELD); PARSER.declareString(TimeSeriesAggregationAggregationBuilder::aggregator, AGGREGATOR_FIELD); - PARSER.declareField( - TimeSeriesAggregationAggregationBuilder::downsampleRange, - p -> new DateHistogramInterval(p.text()), - DOWNSAMPLE_RANGE_FIELD, - ObjectParser.ValueType.STRING - ); - PARSER.declareString(TimeSeriesAggregationAggregationBuilder::downsampleFunction, DOWNSAMPLE_FUNCTION_FIELD); + PARSER.declareObject(TimeSeriesAggregationAggregationBuilder::downsample, (p, c) -> Downsample.fromXContent(p), DOWNSAMPLE_FIELD); PARSER.declareObjectArray( TimeSeriesAggregationAggregationBuilder::order, (p, c) -> InternalOrder.Parser.parseOrderParam(p), @@ -137,8 +128,8 @@ protected TimeSeriesAggregationAggregationBuilder( this.interval = clone.interval; this.offset = clone.offset; this.aggregator = clone.aggregator; - this.downsampleRange = clone.downsampleRange; - this.downsampleFunction = clone.downsampleFunction; + this.downsample = clone.downsample; + this.order = clone.order; this.bucketCountThresholds = clone.bucketCountThresholds; } @@ -150,8 +141,7 @@ public TimeSeriesAggregationAggregationBuilder(StreamInput in) throws IOExceptio interval = in.readOptionalWriteable(DateHistogramInterval::new); offset = in.readOptionalWriteable(DateHistogramInterval::new); aggregator = in.readOptionalString(); - downsampleRange = in.readOptionalWriteable(DateHistogramInterval::new); - downsampleFunction = in.readOptionalString(); + downsample = in.readOptionalWriteable(Downsample::new); order = InternalOrder.Streams.readOrder(in); bucketCountThresholds = new TermsAggregator.BucketCountThresholds(in); } @@ -164,8 +154,7 @@ protected void innerWriteTo(StreamOutput out) throws IOException { out.writeOptionalWriteable(interval); out.writeOptionalWriteable(offset); out.writeOptionalString(aggregator); - out.writeOptionalWriteable(downsampleRange); - out.writeOptionalString(downsampleFunction); + out.writeOptionalWriteable(downsample); order.writeTo(out); bucketCountThresholds.writeTo(out); } @@ -199,9 +188,8 @@ protected ValuesSourceAggregatorFactory innerBuild( without, interval, offset, - aggregator, - downsampleRange, - downsampleFunction, + aggregator != null ? Function.resolve(aggregator) : null, + downsample, bucketCountThresholds, order, config, @@ -231,24 +219,9 @@ protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) if (aggregator != null) { builder.field(AGGREGATOR_FIELD.getPreferredName(), aggregator); } - if (downsampleRange != null) { - builder.field(DOWNSAMPLE_RANGE_FIELD.getPreferredName(), downsampleRange.toString()); + if (downsample != null) { + builder.field(DOWNSAMPLE_FIELD.getPreferredName(), downsample); } - if (downsampleFunction != null) { - builder.field(DOWNSAMPLE_FUNCTION_FIELD.getPreferredName(), downsampleFunction); - } - - // if (downsampleRange != null || downsampleFunction != null) { - // Map downsample = new HashMap<>(); - // if (downsampleRange != null) { - // downsample.put(DOWNSAMPLE_RANGE_FIELD.getPreferredName(), downsampleRange.toString()); - // } - // if (downsampleFunction != null) { - // downsample.put(DOWNSAMPLE_FUNCTION_FIELD.getPreferredName(), downsampleFunction); - // } - // builder.field(DOWNSAMPLE_FIELD.getPreferredName(), downsample); - // } - bucketCountThresholds.toXContent(builder, params); builder.field(ORDER_FIELD.getPreferredName()); order.toXContent(builder, params); @@ -364,36 +337,6 @@ public TimeSeriesAggregationAggregationBuilder aggregator(String aggregator) { return this; } - /** - * Returns the downsample range value - */ - public DateHistogramInterval getDownsampleRange() { - return downsampleRange; - } - - /** - * Set the downsample range interval, if downsample range not set, the downsample range value is replace by interval - */ - public TimeSeriesAggregationAggregationBuilder downsampleRange(DateHistogramInterval downsampleRange) { - this.downsampleRange = downsampleRange; - return this; - } - - /** - * Returns the downsample function - */ - public String getDownsampleFunction() { - return downsampleFunction; - } - - /** - * Sets the downsample function - */ - public TimeSeriesAggregationAggregationBuilder downsampleFunction(String downsampleFunction) { - this.downsampleFunction = downsampleFunction; - return this; - } - /** * Sets the size - indicating how many term buckets should be returned * (defaults to 10) @@ -512,6 +455,29 @@ public BucketOrder order() { return order; } + /** + * Returns the downsample value + */ + public Downsample getDownsample() { + return downsample; + } + + /** + * Sets the downsample value + */ + public TimeSeriesAggregationAggregationBuilder downsample(Downsample downsample) { + this.downsample = downsample; + return this; + } + + /** + * Sets the downsample value + */ + public TimeSeriesAggregationAggregationBuilder downsample(DateHistogramInterval range, Function function) { + this.downsample = new Downsample(range, function); + return this; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -520,7 +486,7 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - if (!super.equals(o)) { + if (false == super.equals(o)) { return false; } TimeSeriesAggregationAggregationBuilder that = (TimeSeriesAggregationAggregationBuilder) o; @@ -530,8 +496,7 @@ public boolean equals(Object o) { && Objects.equals(interval, that.interval) && Objects.equals(offset, that.offset) && Objects.equals(aggregator, that.aggregator) - && Objects.equals(downsampleRange, that.downsampleRange) - && Objects.equals(downsampleFunction, that.downsampleFunction) + && Objects.equals(downsample, that.downsample) && Objects.equals(bucketCountThresholds, that.bucketCountThresholds) && Objects.equals(order, that.order); } @@ -546,8 +511,7 @@ public int hashCode() { interval, offset, aggregator, - downsampleRange, - downsampleFunction, + downsample, bucketCountThresholds, order ); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java index b82bd15148963..8984b42f98b61 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java @@ -34,9 +34,8 @@ public class TimeSeriesAggregationAggregationFactory extends ValuesSourceAggrega private final List without; private final DateHistogramInterval interval; private final DateHistogramInterval offset; - private final String aggregator; - private final DateHistogramInterval downsampleRange; - private final String downsampleFunction; + private final Function aggregator; + private final Downsample downsample; private final TermsAggregator.BucketCountThresholds bucketCountThresholds; private final BucketOrder order; private final ValuesSourceConfig config; @@ -49,9 +48,8 @@ public TimeSeriesAggregationAggregationFactory( List without, DateHistogramInterval interval, DateHistogramInterval offset, - String aggregator, - DateHistogramInterval downsampleRange, - String downsampleFunction, + Function aggregator, + Downsample downsample, TermsAggregator.BucketCountThresholds bucketCountThresholds, BucketOrder order, ValuesSourceConfig config, @@ -68,8 +66,7 @@ public TimeSeriesAggregationAggregationFactory( this.interval = interval; this.offset = offset; this.aggregator = aggregator; - this.downsampleRange = downsampleRange; - this.downsampleFunction = downsampleFunction; + this.downsample = downsample; this.bucketCountThresholds = bucketCountThresholds; this.order = order; this.config = config; @@ -105,8 +102,7 @@ protected Aggregator createUnmapped(Aggregator parent, Map metad interval, offset, aggregator, - downsampleRange, - downsampleFunction, + downsample, thresholds, order, config, @@ -138,8 +134,7 @@ protected Aggregator doCreateInternal(Aggregator parent, CardinalityUpperBound c interval, offset, aggregator, - downsampleRange, - downsampleFunction, + downsample, thresholds, order, config, diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index 01aefe57f6284..54c4e199b44be 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -39,7 +39,6 @@ import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; -import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregation.Function; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.AggregatorBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.TSIDBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; @@ -73,9 +72,9 @@ public class TimeSeriesAggregationAggregator extends BucketsAggregator { private Set without; private long interval; private long offset; - private TimeSeriesAggregation.Aggregator aggregator; + private Function aggregator; private long downsampleRange; - protected TimeSeriesAggregation.Function downsampleFunction; + protected Function downsampleFunction; private BucketOrder order; private TermsAggregator.BucketCountThresholds bucketCountThresholds; protected Comparator partiallyBuiltBucketComparator; @@ -100,9 +99,8 @@ public TimeSeriesAggregationAggregator( List without, DateHistogramInterval interval, DateHistogramInterval offset, - String aggregator, - DateHistogramInterval downsampleRange, - String downsampleFunction, + Function aggregator, + Downsample downsample, TermsAggregator.BucketCountThresholds bucketCountThresholds, BucketOrder order, ValuesSourceConfig valuesSourceConfig, @@ -123,9 +121,9 @@ public TimeSeriesAggregationAggregator( } this.rounding = Rounding.builder(new TimeValue(this.interval)).build().prepareForUnknown(); this.offset = offset != null ? offset.estimateMillis() : -1; - this.aggregator = aggregator != null ? TimeSeriesAggregation.Aggregator.resolve(aggregator) : null; - this.downsampleRange = downsampleRange != null ? downsampleRange.estimateMillis() : -1; - this.downsampleFunction = downsampleFunction != null ? TimeSeriesAggregation.Function.resolve(downsampleFunction) : Function.last; + this.aggregator = aggregator; + this.downsampleRange = downsample != null ? downsample.getRange().estimateMillis() : -1; + this.downsampleFunction = downsample != null ? downsample.getFunction() : Function.last; if (this.downsampleRange <= 0) { this.downsampleRange = this.interval; } @@ -308,7 +306,7 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBu protected AggregatorFunction getAggregatorFunction() throws IOException { AggregatorFunction function = timeBucketMetrics.get(preRounding); if (function == null) { - function = TimeSeriesAggregations.getAggregatorFunction(downsampleFunction); + function = downsampleFunction.getAggregatorFunction(); timeBucketMetrics.put(preRounding, function); } return function; @@ -378,11 +376,11 @@ private BytesRef packKey(BytesRef tsid) { Map groupMap = new LinkedHashMap<>(); tsidMap.forEach((key, value) -> { if (group.size() > 0) { - if (group.contains(key) && !without.contains(key)) { + if (group.contains(key) && false == without.contains(key)) { groupMap.put(key, value); } } else { - if (!without.contains(key)) { + if (false == without.contains(key)) { groupMap.put(key, value); } } @@ -397,7 +395,7 @@ public void collectTimeSeriesValues(long bucketOrd) throws IOException { if (needGroupBy) { AggregatorBucketFunction aggregatorBucketFunction = aggregatorCollectors.get(bucketOrd); if (aggregatorBucketFunction == null) { - AggregatorBucketFunction internal = TimeSeriesAggregations.getAggregatorBucketFunction(aggregator, bigArrays()); + AggregatorBucketFunction internal = aggregator.getAggregatorBucketFunction(bigArrays()); aggregatorBucketFunction = new TSIDBucketFunction(internal); aggregatorCollectors.put(bucketOrd, aggregatorBucketFunction); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java index 02898bf89893c..98ce531f3ef7e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java @@ -31,9 +31,8 @@ Aggregator build( List without, DateHistogramInterval interval, DateHistogramInterval offset, - String aggregator, - DateHistogramInterval downsampleRange, - String downsampleFunction, + Function aggregator, + Downsample downsample, TermsAggregator.BucketCountThresholds bucketCountThresholds, BucketOrder order, ValuesSourceConfig valuesSourceConfig, diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregations.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregations.java deleted file mode 100644 index a0a4b8397daa3..0000000000000 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregations.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0 and the Server Side Public License, v 1; you may not use this file except - * in compliance with, at your election, the Elastic License 2.0 or the Server - * Side Public License, v 1. - */ - -package org.elasticsearch.search.aggregations.timeseries.aggregation; - -import org.elasticsearch.common.util.BigArrays; -import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.AggregatorBucketFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.AvgBucketFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.MaxBucketFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.MinBucketFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.NoAggregatorBucketFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.SumBucketFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AvgFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.function.LastFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MaxFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MinFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.function.SumFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.function.ValueCountFunction; - -public class TimeSeriesAggregations { - public static AggregatorFunction getAggregatorFunction(TimeSeriesAggregation.Function function) { - switch (function) { - case avg: - return new AvgFunction(); - case max: - return new MaxFunction(); - case min: - return new MinFunction(); - case sum: - return new SumFunction(); - case count: - return new ValueCountFunction(); - case last: - return new LastFunction(); - } - return new AvgFunction(); - } - - public static AggregatorBucketFunction getAggregatorBucketFunction( - TimeSeriesAggregation.Aggregator aggregator, - BigArrays bigArrays - ) { - if (aggregator == null) { - return new NoAggregatorBucketFunction(); - } - - switch (aggregator) { - case max: - return new MaxBucketFunction(bigArrays); - case min: - return new MinBucketFunction(bigArrays); - case sum: - return new SumBucketFunction(bigArrays); - case avg: - return new AvgBucketFunction(bigArrays); - } - return new AvgBucketFunction(bigArrays); - } -} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java index e458f9856d7f2..9f9a1794f3368 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java @@ -94,7 +94,7 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - if (!super.equals(o)) { + if (false == super.equals(o)) { return false; } Last last1 = (Last) o; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java index a04858002eb14..977036cd2b354 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java @@ -16,8 +16,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation.SingleValue; -import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregation; -import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregations; +import org.elasticsearch.search.aggregations.timeseries.aggregation.Function; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.TSIDBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; import org.elasticsearch.xcontent.XContentBuilder; @@ -76,8 +75,8 @@ public InternalAggregation reduce(List aggregations, Aggreg if (aggregations.size() == 1) { TSIDInternalAggregation tsidAgg = (TSIDInternalAggregation) aggregations.get(0); if (reduceContext.isFinalReduce()) { - TimeSeriesAggregation.Function function = TimeSeriesAggregation.Function.valueOf(aggreagator); - final AggregatorFunction aggregatorFunction = TimeSeriesAggregations.getAggregatorFunction(function); + Function function = Function.valueOf(aggreagator); + final AggregatorFunction aggregatorFunction = function.getAggregatorFunction(); tsidAgg.values.forEach( (tsid, agg) -> { aggregatorFunction.collect(((InternalNumericMetricsAggregation.SingleValue) agg).value()); } ); @@ -100,8 +99,8 @@ public InternalAggregation reduce(List aggregations, Aggreg } if (reduceContext.isFinalReduce()) { - TimeSeriesAggregation.Function function = TimeSeriesAggregation.Function.valueOf(aggreagator); - final AggregatorFunction aggregatorFunction = TimeSeriesAggregations.getAggregatorFunction(function); + Function function = Function.valueOf(aggreagator); + final AggregatorFunction aggregatorFunction = function.getAggregatorFunction(); reduced.forEach((tsid, aggs) -> { if (aggs.size() > 0) { InternalAggregation first = aggs.get(0); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregationTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregationTests.java index 68c50414c034d..88abef38e922e 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregationTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregationTests.java @@ -30,7 +30,11 @@ protected void assertReduced(InternalTimeSeriesAggregation reduced, List metadata, + InternalAggregations aggregations + ) { BucketOrder order = BucketOrder.count(false); long minDocCount = 1; int requiredSize = 3; diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilderTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilderTests.java index 132323769c536..7c9d26b075a9f 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilderTests.java @@ -45,7 +45,7 @@ protected Writeable.Reader instanceRead @Override protected TimeSeriesAggregationAggregationBuilder createTestInstance() { - return new TimeSeriesAggregationAggregationBuilder(randomAlphaOfLength(10)); + return new TimeSeriesAggregationAggregationBuilder(randomAlphaOfLength(10)).field(randomAlphaOfLength(5)); } @Override diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java index 70aa2f034c585..9542c42b5a288 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java @@ -19,7 +19,8 @@ import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; -import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregation.Function; +import org.elasticsearch.search.aggregations.timeseries.aggregation.Downsample; +import org.elasticsearch.search.aggregations.timeseries.aggregation.Function; import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregationAggregator; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AvgFunction; @@ -43,9 +44,8 @@ public AggregateMetricTimeSeriesAggregationAggregator( List without, DateHistogramInterval interval, DateHistogramInterval offset, - String aggregator, - DateHistogramInterval downsampleRange, - String downsampleFunction, + Function aggregator, + Downsample downsample, BucketCountThresholds bucketCountThresholds, BucketOrder order, ValuesSourceConfig valuesSourceConfig, @@ -63,8 +63,7 @@ public AggregateMetricTimeSeriesAggregationAggregator( interval, offset, aggregator, - downsampleRange, - downsampleFunction, + downsample, bucketCountThresholds, order, null, From 9568953f1c67aa0effd8579a7a4b4dc445595048 Mon Sep 17 00:00:00 2001 From: weizijun Date: Mon, 25 Apr 2022 21:07:28 +0800 Subject: [PATCH 06/53] support down sample range --- .../TimeSeriesAggregationAggregator.java | 29 ++++++++++++++----- 1 file changed, 22 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index 54c4e199b44be..76d79e0464eef 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -54,6 +54,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Set; +import java.util.TreeMap; import static org.elasticsearch.search.DocValueFormat.TIME_SERIES_ID; import static org.elasticsearch.search.aggregations.InternalOrder.isKeyOrder; @@ -120,7 +121,7 @@ public TimeSeriesAggregationAggregator( throw new IllegalArgumentException("time_series_aggregation invalid interval [" + interval + "]"); } this.rounding = Rounding.builder(new TimeValue(this.interval)).build().prepareForUnknown(); - this.offset = offset != null ? offset.estimateMillis() : -1; + this.offset = offset != null ? offset.estimateMillis() : 0; this.aggregator = aggregator; this.downsampleRange = downsample != null ? downsample.getRange().estimateMillis() : -1; this.downsampleFunction = downsample != null ? downsample.getFunction() : Function.last; @@ -272,6 +273,9 @@ public void collect(int doc, long bucket) throws IOException { if (preRounding < 0 || aggCtx.getTimestamp() <= preRounding - interval) { preRounding = rounding.nextRoundingValue(aggCtx.getTimestamp()); + if (false == timeBucketMetrics.containsKey(preRounding)) { + timeBucketMetrics.put(preRounding, downsampleFunction.getAggregatorFunction()); + } } // calculate the value of the current doc @@ -280,7 +284,7 @@ public void collect(int doc, long bucket) throws IOException { } private void reset(BytesRef tsid, long bucket) { - timeBucketMetrics = new LinkedHashMap<>(); + timeBucketMetrics = new TreeMap<>(); preTsid = BytesRef.deepCopyOf(tsid); preRounding = -1; @@ -319,15 +323,26 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBu } final SortedNumericDoubleValues values = valuesSource.doubleValues(context); return new Collector(sub, values, aggCtx, (doc) -> { + if (aggCtx.getTimestamp() + downsampleRange < preRounding) { + return; + } + if (values.advanceExact(doc)) { final int valuesCount = values.docValueCount(); for (int i = 0; i < valuesCount; i++) { double value = values.nextValue(); - AggregatorFunction function = getAggregatorFunction(); - if (function instanceof LastFunction last) { - last.collectExact(value, aggCtx.getTimestamp()); - } else { - function.collect(value); + for (Entry entry : timeBucketMetrics.entrySet()) { + Long timestamp = entry.getKey(); + AggregatorFunction function = entry.getValue(); + if (aggCtx.getTimestamp() + downsampleRange > timestamp) { + if (function instanceof LastFunction last) { + last.collectExact(value, aggCtx.getTimestamp()); + } else { + function.collect(value); + } + } else { + break; + } } } } From 75773687b3ee49403cdd9621be2b8915a107ec7d Mon Sep 17 00:00:00 2001 From: weizijun Date: Tue, 26 Apr 2022 11:53:41 +0800 Subject: [PATCH 07/53] support offset --- .../TimeSeriesAggregationAggregator.java | 22 +++------- .../internal/TSIDInternalAggregation.java | 11 +++-- ...MetricTimeSeriesAggregationAggregator.java | 42 ++++++++++++------- 3 files changed, 39 insertions(+), 36 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index 76d79e0464eef..86ef1fda43766 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -74,7 +74,7 @@ public class TimeSeriesAggregationAggregator extends BucketsAggregator { private long interval; private long offset; private Function aggregator; - private long downsampleRange; + protected long downsampleRange; protected Function downsampleFunction; private BucketOrder order; private TermsAggregator.BucketCountThresholds bucketCountThresholds; @@ -84,10 +84,10 @@ public class TimeSeriesAggregationAggregator extends BucketsAggregator { private BytesRef preTsid; private long preBucketOrdinal; - private long preRounding = -1; + protected long preRounding = -1; private Rounding.Prepared rounding; private boolean needGroupBy; - private Map timeBucketMetrics; // TODO replace map + protected Map timeBucketMetrics; // TODO replace map private Map> groupBucketValues; // TODO replace map private Map aggregatorCollectors; // TODO replace map @@ -196,7 +196,7 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I AggregatorBucketFunction aggregatorBucketFunction = aggregatorCollectors.get(ord); LongKeyedBucketOrds.BucketOrdsEnum timeOrdsEnum = timestampOrds.ordsEnum(ord); while (timeOrdsEnum.next()) { - values.put(timeOrdsEnum.value(), aggregatorBucketFunction.getAggregation(timeOrdsEnum.ord(), format, metadata())); + values.put(timeOrdsEnum.value() + offset, aggregatorBucketFunction.getAggregation(timeOrdsEnum.ord(), format, metadata())); } } else { values = groupBucketValues.get(ord); @@ -304,18 +304,6 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBu throw new UnsupportedOperationException("Shouldn't be here"); } - /** - * get the downsample function of the current rouding - */ - protected AggregatorFunction getAggregatorFunction() throws IOException { - AggregatorFunction function = timeBucketMetrics.get(preRounding); - if (function == null) { - function = downsampleFunction.getAggregatorFunction(); - timeBucketMetrics.put(preRounding, function); - } - return function; - } - protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBucketCollector sub, AggregationExecutionContext aggCtx) throws IOException { if (valuesSource == null) { @@ -443,7 +431,7 @@ public void collectTimeSeriesValues(long bucketOrd) throws IOException { } } else { Map tsids = new LinkedHashMap<>(); - timeBucketMetrics.forEach((k, v) -> { tsids.put(k, v.getAggregation(format, metadata())); }); + timeBucketMetrics.forEach((k, v) -> { tsids.put(k + offset, v.getAggregation(format, metadata())); }); groupBucketValues.put(bucketOrd, tsids); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java index 977036cd2b354..b4fab7073a04e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java @@ -22,6 +22,8 @@ import org.elasticsearch.xcontent.XContentBuilder; import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.TreeMap; @@ -90,11 +92,12 @@ public InternalAggregation reduce(List aggregations, Aggreg for (InternalAggregation aggregation : aggregations) { TSIDInternalAggregation tsidAgg = (TSIDInternalAggregation) aggregation; tsidAgg.values.forEach((tsid, value) -> { - if (false == reduced.containsKey(tsid)) { - reduced.put(tsid, List.of(value)); - } else { - reduced.get(tsid).add(value); + List values = reduced.get(tsid); + if (values == null) { + values = new ArrayList<>(); + reduced.put(tsid, values); } + values.add(value); }); } diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java index 9542c42b5a288..77800fc036f25 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java @@ -24,6 +24,7 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregationAggregator; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AvgFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.LastFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.ValueCountFunction; import org.elasticsearch.xpack.aggregatemetric.aggregations.support.AggregateMetricsValuesSource; import org.elasticsearch.xpack.aggregatemetric.mapper.AggregateDoubleMetricFieldMapper.Metric; @@ -31,6 +32,7 @@ import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.Map.Entry; @SuppressWarnings({ "unchecked", "rawtypes" }) public class AggregateMetricTimeSeriesAggregationAggregator extends TimeSeriesAggregationAggregator { @@ -91,24 +93,27 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBu if (metricType != null) { final SortedNumericDoubleValues values = valuesSource.getAggregateMetricValues(context, metricType); return new Collector(sub, values, aggCtx, (doc) -> { - if (values.advanceExact(doc)) { - final int valuesCount = values.docValueCount(); - for (int i = 0; i < valuesCount; i++) { - double value = values.nextValue(); - AggregatorFunction function = getAggregatorFunction(); - function.collect(value); - } + if (aggCtx.getTimestamp() + downsampleRange < preRounding) { + return; } - }); - } else if (metricType == Metric.value_count) { - final SortedNumericDoubleValues values = valuesSource.getAggregateMetricValues(context, metricType); - return new Collector(sub, values, aggCtx, (doc) -> { + if (values.advanceExact(doc)) { final int valuesCount = values.docValueCount(); for (int i = 0; i < valuesCount; i++) { double value = values.nextValue(); - AggregatorFunction function = getAggregatorFunction(); - ((ValueCountFunction) function).collectExact((long) value); + for (Entry entry : timeBucketMetrics.entrySet()) { + Long timestamp = entry.getKey(); + AggregatorFunction function = entry.getValue(); + if (aggCtx.getTimestamp() + downsampleRange > timestamp) { + if (function instanceof ValueCountFunction) { + ((ValueCountFunction) function).collectExact((long) value); + } else { + function.collect(value); + } + } else { + break; + } + } } } }); @@ -134,8 +139,15 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBu } } - AggregatorFunction function = getAggregatorFunction(); - ((AvgFunction) function).collectExact(sum, valueCount); + for (Entry entry : timeBucketMetrics.entrySet()) { + Long timestamp = entry.getKey(); + AggregatorFunction function = entry.getValue(); + if (aggCtx.getTimestamp() + downsampleRange > timestamp) { + ((AvgFunction) function).collectExact(sum, valueCount); + } else { + break; + } + } }); } } From 984fc2d9560c7c2728dc6f101c5e46b205d2c023 Mon Sep 17 00:00:00 2001 From: weizijun Date: Wed, 27 Apr 2022 15:59:38 +0800 Subject: [PATCH 08/53] fixed last exception and support without group --- .../TimeSeriesAggregationsIT.java | 100 ++++++++++++++++++ .../TimeSeriesAggregationAggregator.java | 27 +++-- .../aggregation/function/LastFunction.java | 1 + .../timeseries/aggregation/internal/Last.java | 5 + .../internal/TSIDInternalAggregation.java | 9 +- .../InternalTimeSeriesAggregationTests.java | 2 +- 6 files changed, 131 insertions(+), 13 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java index 65660fed93850..a093c8af7af6d 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java @@ -17,6 +17,7 @@ import org.elasticsearch.common.Rounding; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.TimeValue; +import org.elasticsearch.core.Tuple; import org.elasticsearch.index.mapper.DateFieldMapper; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; @@ -529,6 +530,49 @@ public void testGetHitsFailure() throws Exception { } public void testBasicTimeSeriesAggregations() { + DateHistogramInterval fixedInterval = DateHistogramInterval.days(randomIntBetween(10, 100)); + Rounding.Prepared rounding = Rounding.builder(new TimeValue(fixedInterval.estimateMillis())).build().prepareForUnknown(); + SearchResponse response = client().prepareSearch("index") + .setSize(0) + .addAggregation( + timeSeriesAggregation("by_ts").field("metric_0") + .interval(fixedInterval) + .size(data.size()) + ) + .get(); + Aggregations aggregations = response.getAggregations(); + assertNotNull(aggregations); + InternalTimeSeriesAggregation timeSeries = aggregations.get("by_ts"); + assertThat( + timeSeries.getBuckets().stream().map(MultiBucketsAggregation.Bucket::getKey).collect(Collectors.toSet()), + equalTo(data.keySet()) + ); + for (InternalTimeSeriesAggregation.InternalBucket bucket : timeSeries.getBuckets()) { + Map key = bucket.getKey(); + Map> dataValues = data.get(key); + assertThat((long) dataValues.size(), equalTo(bucket.getDocCount())); + Map> dataBucketValues = new HashMap<>(); + dataValues.forEach((timestamp, metrics) -> { + long roundValue = rounding.nextRoundingValue(timestamp); + if (dataBucketValues.containsKey(roundValue)) { + if (timestamp > dataBucketValues.get(roundValue).v2()) { + dataBucketValues.put(roundValue, new Tuple<>(metrics.get("metric_0"), timestamp)); + } + } else { + dataBucketValues.put(roundValue, new Tuple<>(metrics.get("metric_0"), timestamp)); + } + }); + + dataBucketValues.forEach((timestamp, value) -> { + assertTrue(bucket.getTimeBucketValues().containsKey(timestamp)); + InternalAggregation aggregation = bucket.getTimeBucketValues().get(timestamp); + assertTrue(aggregation instanceof InternalNumericMetricsAggregation.SingleValue); + assertThat(((InternalNumericMetricsAggregation.SingleValue) aggregation).value(), closeTo(value.v1(), 0.0001d)); + }); + } + } + + public void testTimeSeriesAggregationsDownsample() { DateHistogramInterval fixedInterval = DateHistogramInterval.days(randomIntBetween(10, 100)); Rounding.Prepared rounding = Rounding.builder(new TimeValue(fixedInterval.estimateMillis())).build().prepareForUnknown(); SearchResponse response = client().prepareSearch("index") @@ -570,6 +614,62 @@ public void testBasicTimeSeriesAggregations() { } } + public void testTimeSeriesAggregationsAggregator() { + DateHistogramInterval fixedInterval = DateHistogramInterval.days(randomIntBetween(10, 100)); + Rounding.Prepared rounding = Rounding.builder(new TimeValue(fixedInterval.estimateMillis())).build().prepareForUnknown(); + SearchResponse response = client().prepareSearch("index") + .setSize(0) + .addAggregation( + timeSeriesAggregation("by_ts").field("metric_0") + .interval(fixedInterval) + .aggregator("sum") + .size(data.size()) + ) + .get(); + Aggregations aggregations = response.getAggregations(); + assertNotNull(aggregations); + InternalTimeSeriesAggregation timeSeries = aggregations.get("by_ts"); + assertThat(timeSeries.getBuckets().size(), equalTo(1)); + assertThat( + timeSeries.getBuckets().stream().map(MultiBucketsAggregation.Bucket::getKey).collect(Collectors.toSet()), + equalTo(Set.of(Map.of())) + ); + + Map aggResults = new HashMap<>(); + data.forEach((key, value) -> { + Map> downsampleResult = new HashMap<>(); + value.forEach((timestamp, metrics) -> { + long roundValue = rounding.nextRoundingValue(timestamp); + if (downsampleResult.containsKey(roundValue)) { + if (timestamp > downsampleResult.get(roundValue).v2()) { + downsampleResult.put(roundValue, new Tuple<>(metrics.get("metric_0"), timestamp)); + } + } else { + downsampleResult.put(roundValue, new Tuple<>(metrics.get("metric_0"), timestamp)); + } + }); + + for (Entry> entry : downsampleResult.entrySet()) { + Long timestamp = entry.getKey(); + Double downsampleValue = entry.getValue().v1(); + if (aggResults.containsKey(timestamp)) { + aggResults.put(timestamp, aggResults.get(timestamp) + downsampleValue); + } else { + aggResults.put(timestamp, downsampleValue); + } + } + }); + + for (InternalTimeSeriesAggregation.InternalBucket bucket : timeSeries.getBuckets()) { + aggResults.forEach((timestamp, metric) -> { + assertTrue(bucket.getTimeBucketValues().containsKey(timestamp)); + InternalAggregation aggregation = bucket.getTimeBucketValues().get(timestamp); + assertTrue(aggregation instanceof InternalNumericMetricsAggregation.SingleValue); + assertThat(((InternalNumericMetricsAggregation.SingleValue) aggregation).value(), closeTo(metric, 0.0001d)); + }); + } + } + public void testTimeSeriesAggregationsGroupBy() { DateHistogramInterval fixedInterval = DateHistogramInterval.days(randomIntBetween(10, 100)); Rounding.Prepared rounding = Rounding.builder(new TimeValue(fixedInterval.estimateMillis())).build().prepareForUnknown(); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index 86ef1fda43766..3006efe7e7a4b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -86,7 +86,7 @@ public class TimeSeriesAggregationAggregator extends BucketsAggregator { private long preBucketOrdinal; protected long preRounding = -1; private Rounding.Prepared rounding; - private boolean needGroupBy; + private boolean needAggregator; protected Map timeBucketMetrics; // TODO replace map private Map> groupBucketValues; // TODO replace map private Map aggregatorCollectors; // TODO replace map @@ -114,8 +114,6 @@ public TimeSeriesAggregationAggregator( this.keyed = keyed; this.group = group != null ? Sets.newHashSet(group) : Set.of(); this.without = without != null ? Sets.newHashSet(without) : Set.of(); - this.needGroupBy = this.group.size() > 0 || this.without.size() > 0; - this.interval = interval != null ? interval.estimateMillis() : -1; if (this.interval <= 0) { throw new IllegalArgumentException("time_series_aggregation invalid interval [" + interval + "]"); @@ -123,6 +121,7 @@ public TimeSeriesAggregationAggregator( this.rounding = Rounding.builder(new TimeValue(this.interval)).build().prepareForUnknown(); this.offset = offset != null ? offset.estimateMillis() : 0; this.aggregator = aggregator; + this.needAggregator = this.aggregator != null; this.downsampleRange = downsample != null ? downsample.getRange().estimateMillis() : -1; this.downsampleFunction = downsample != null ? downsample.getFunction() : Function.last; if (this.downsampleRange <= 0) { @@ -142,8 +141,8 @@ public TimeSeriesAggregationAggregator( } this.timestampBounds = context.getIndexSettings().getTimestampBounds(); - if (needGroupBy && this.aggregator == null) { - throw new IllegalArgumentException("time_series_aggregation invalid aggregator [" + aggregator + "]"); + if ((this.group.size() > 0 || this.without.size() > 0) && this.aggregator == null) { + throw new IllegalArgumentException("time_series_aggregation group by must have an aggregator"); } groupBucketValues = new LinkedHashMap<>(); @@ -192,11 +191,14 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I InternalTimeSeriesAggregation.InternalBucket bucket = ordered.pop(); long ord = bucket.bucketOrd; Map values = new LinkedHashMap<>(); - if (needGroupBy) { + if (needAggregator) { AggregatorBucketFunction aggregatorBucketFunction = aggregatorCollectors.get(ord); LongKeyedBucketOrds.BucketOrdsEnum timeOrdsEnum = timestampOrds.ordsEnum(ord); while (timeOrdsEnum.next()) { - values.put(timeOrdsEnum.value() + offset, aggregatorBucketFunction.getAggregation(timeOrdsEnum.ord(), format, metadata())); + values.put( + timeOrdsEnum.value() + offset, + aggregatorBucketFunction.getAggregation(timeOrdsEnum.ord(), format, metadata()) + ); } } else { values = groupBucketValues.get(ord); @@ -288,7 +290,7 @@ private void reset(BytesRef tsid, long bucket) { preTsid = BytesRef.deepCopyOf(tsid); preRounding = -1; - BytesRef bucketValue = needGroupBy ? packKey(preTsid) : preTsid; + BytesRef bucketValue = needAggregator ? packKey(preTsid) : preTsid; long bucketOrdinal = bucketOrds.add(bucket, bucketValue); if (bucketOrdinal < 0) { // already seen bucketOrdinal = -1 - bucketOrdinal; @@ -323,6 +325,7 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBu Long timestamp = entry.getKey(); AggregatorFunction function = entry.getValue(); if (aggCtx.getTimestamp() + downsampleRange > timestamp) { + // TODO replace this instanceof logic if (function instanceof LastFunction last) { last.collectExact(value, aggCtx.getTimestamp()); } else { @@ -375,6 +378,10 @@ InternalTimeSeriesAggregation buildResult(long otherDocCount, InternalTimeSeries * decode the tsid and pack the bucket key from the group and without config */ private BytesRef packKey(BytesRef tsid) { + if (group.size() == 0 && without.size() == 0) { + return new BytesRef(new byte[] { 0 }); + } + Map tsidMap = TimeSeriesIdFieldMapper.decodeTsid(tsid); Map groupMap = new LinkedHashMap<>(); tsidMap.forEach((key, value) -> { @@ -382,7 +389,7 @@ private BytesRef packKey(BytesRef tsid) { if (group.contains(key) && false == without.contains(key)) { groupMap.put(key, value); } - } else { + } else if (without.size() > 0) { if (false == without.contains(key)) { groupMap.put(key, value); } @@ -395,7 +402,7 @@ private BytesRef packKey(BytesRef tsid) { * collect the value of one time series line */ public void collectTimeSeriesValues(long bucketOrd) throws IOException { - if (needGroupBy) { + if (needAggregator) { AggregatorBucketFunction aggregatorBucketFunction = aggregatorCollectors.get(bucketOrd); if (aggregatorBucketFunction == null) { AggregatorBucketFunction internal = aggregator.getAggregatorBucketFunction(bigArrays()); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java index 1db44cff9c8fb..c0b5e5cd76b40 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java @@ -27,6 +27,7 @@ public void collect(Double value) { public void collectExact(double number, long timestamp) { if (timestamp > this.timestamp) { + this.timestamp = timestamp; last = number; } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java index 9f9a1794f3368..7c082ffa98553 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java @@ -64,6 +64,10 @@ public double value() { return last; } + public long getTimestamp() { + return timestamp; + } + @Override public Last reduce(List aggregations, AggregationReduceContext reduceContext) { double last = Double.NEGATIVE_INFINITY; @@ -71,6 +75,7 @@ public Last reduce(List aggregations, AggregationReduceCont for (InternalAggregation aggregation : aggregations) { if (((Last) aggregation).timestamp > timestamp) { last = ((Last) aggregation).last; + timestamp = ((Last) aggregation).timestamp; } } return new Last(name, last, timestamp, format, getMetadata()); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java index b4fab7073a04e..5ffab5abc4464 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java @@ -19,11 +19,11 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.Function; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.TSIDBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.LastFunction; import org.elasticsearch.xcontent.XContentBuilder; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.TreeMap; @@ -108,7 +108,12 @@ public InternalAggregation reduce(List aggregations, Aggreg if (aggs.size() > 0) { InternalAggregation first = aggs.get(0); InternalNumericMetricsAggregation.SingleValue internalAggregation = (SingleValue) first.reduce(aggs, reduceContext); - aggregatorFunction.collect(internalAggregation.value()); + // TODO replace this instanceof logic + if (internalAggregation instanceof Last last && aggregatorFunction instanceof LastFunction lastFunction) { + lastFunction.collectExact(last.value(), last.getTimestamp()); + } else { + aggregatorFunction.collect(internalAggregation.value()); + } } }); return aggregatorFunction.getAggregation(formatter, getMetadata()); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregationTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregationTests.java index 88abef38e922e..f86a9c87bd0c5 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregationTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregationTests.java @@ -38,7 +38,7 @@ protected InternalTimeSeriesAggregation createTestInstance( BucketOrder order = BucketOrder.count(false); long minDocCount = 1; int requiredSize = 3; - int shardSize = requiredSize + 2; + int shardSize = requiredSize; long otherDocCount = 0; boolean keyed = randomBoolean(); boolean showTermDocCountError = randomBoolean(); From 837b32f88f06e617b481d6ae3e56ce8af5bd0e6a Mon Sep 17 00:00:00 2001 From: weizijun Date: Tue, 10 May 2022 15:45:12 +0800 Subject: [PATCH 09/53] fixed empty bucket --- .../aggregation/TimeSeriesAggregationAggregator.java | 6 +++--- .../AggregateMetricTimeSeriesAggregationAggregator.java | 6 ++++++ 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index 3006efe7e7a4b..5bb0e23f8a60d 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -275,9 +275,6 @@ public void collect(int doc, long bucket) throws IOException { if (preRounding < 0 || aggCtx.getTimestamp() <= preRounding - interval) { preRounding = rounding.nextRoundingValue(aggCtx.getTimestamp()); - if (false == timeBucketMetrics.containsKey(preRounding)) { - timeBucketMetrics.put(preRounding, downsampleFunction.getAggregatorFunction()); - } } // calculate the value of the current doc @@ -321,6 +318,9 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBu final int valuesCount = values.docValueCount(); for (int i = 0; i < valuesCount; i++) { double value = values.nextValue(); + if (false == timeBucketMetrics.containsKey(preRounding)) { + timeBucketMetrics.put(preRounding, downsampleFunction.getAggregatorFunction()); + } for (Entry entry : timeBucketMetrics.entrySet()) { Long timestamp = entry.getKey(); AggregatorFunction function = entry.getValue(); diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java index 77800fc036f25..180da82628ff0 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java @@ -101,6 +101,9 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBu final int valuesCount = values.docValueCount(); for (int i = 0; i < valuesCount; i++) { double value = values.nextValue(); + if (false == timeBucketMetrics.containsKey(preRounding)) { + timeBucketMetrics.put(preRounding, downsampleFunction.getAggregatorFunction()); + } for (Entry entry : timeBucketMetrics.entrySet()) { Long timestamp = entry.getKey(); AggregatorFunction function = entry.getValue(); @@ -139,6 +142,9 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBu } } + if (false == timeBucketMetrics.containsKey(preRounding)) { + timeBucketMetrics.put(preRounding, downsampleFunction.getAggregatorFunction()); + } for (Entry entry : timeBucketMetrics.entrySet()) { Long timestamp = entry.getKey(); AggregatorFunction function = entry.getValue(); From 23f3a8c6dca4e76eb2e306a74be5bb4eea59b004 Mon Sep 17 00:00:00 2001 From: weizijun Date: Thu, 12 May 2022 11:27:03 +0800 Subject: [PATCH 10/53] [WIP] add rate support --- .../timeseries/aggregation/Function.java | 26 ++- .../timeseries/aggregation/TimePoint.java | 72 ++++++ .../TimeSeriesAggregationAggregator.java | 2 +- .../aggregation/function/RateFunction.java | 126 ++++++++++ .../internal/TSIDInternalAggregation.java | 4 +- .../aggregation/internal/TimeSeriesRate.java | 218 ++++++++++++++++++ ...MetricTimeSeriesAggregationAggregator.java | 4 +- 7 files changed, 440 insertions(+), 12 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimePoint.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/RateFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesRate.java diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java index d0349e2c9e8da..33ac7c798b060 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java @@ -19,13 +19,14 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.function.LastFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MaxFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MinFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.RateFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.SumFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.ValueCountFunction; public enum Function { count { @Override - public AggregatorFunction getAggregatorFunction() { + public AggregatorFunction getAggregatorFunction(TimeSeriesAggregationAggregator aggregator) { return new ValueCountFunction(); } @@ -36,7 +37,7 @@ public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArra }, sum { @Override - public AggregatorFunction getAggregatorFunction() { + public AggregatorFunction getAggregatorFunction(TimeSeriesAggregationAggregator aggregator) { return new SumFunction(); } @@ -47,7 +48,7 @@ public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArra }, min { @Override - public AggregatorFunction getAggregatorFunction() { + public AggregatorFunction getAggregatorFunction(TimeSeriesAggregationAggregator aggregator) { return new MinFunction(); } @@ -58,7 +59,7 @@ public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArra }, max { @Override - public AggregatorFunction getAggregatorFunction() { + public AggregatorFunction getAggregatorFunction(TimeSeriesAggregationAggregator aggregator) { return new MaxFunction(); } @@ -69,7 +70,7 @@ public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArra }, avg { @Override - public AggregatorFunction getAggregatorFunction() { + public AggregatorFunction getAggregatorFunction(TimeSeriesAggregationAggregator aggregator) { return new AvgFunction(); } @@ -80,10 +81,21 @@ public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArra }, last { @Override - public AggregatorFunction getAggregatorFunction() { + public AggregatorFunction getAggregatorFunction(TimeSeriesAggregationAggregator aggregator) { return new LastFunction(); } + @Override + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { + throw new UnsupportedOperationException("last aggregator bucket function not support"); + } + }, + rate { + @Override + public AggregatorFunction getAggregatorFunction(TimeSeriesAggregationAggregator aggregator) { + return new RateFunction(0, 0, true, true); + } + @Override public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { throw new UnsupportedOperationException("last aggregator bucket function not support"); @@ -97,7 +109,7 @@ public static Function resolve(String name) { /** * get the aggregator function */ - public abstract AggregatorFunction getAggregatorFunction(); + public abstract AggregatorFunction getAggregatorFunction(TimeSeriesAggregationAggregator aggregator); /** * get the aggregator bucket function diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimePoint.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimePoint.java new file mode 100644 index 0000000000000..e30f124dec705 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimePoint.java @@ -0,0 +1,72 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; + +import java.io.IOException; +import java.util.Objects; + +public class TimePoint implements Comparable, Writeable { + private long timestamp; + private double value; + + public TimePoint(long timestamp, double value) { + this.timestamp = timestamp; + this.value = value; + } + + public TimePoint(StreamInput in) throws IOException { + this.timestamp = in.readLong(); + this.value = in.readDouble(); + } + + public long getTimestamp() { + return timestamp; + } + + public double getValue() { + return value; + } + + @Override + public String toString() { + return "Point{" + "timestamp=" + timestamp + ", value=" + value + '}'; + } + + @Override + public int compareTo(TimePoint o) { + return (int) (this.timestamp - o.timestamp); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TimePoint timePoint = (TimePoint) o; + return timestamp == timePoint.timestamp && Objects.equals(value, timePoint.value); + } + + @Override + public int hashCode() { + return Objects.hash(timestamp, value); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeLong(timestamp); + out.writeDouble(value); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index 5bb0e23f8a60d..a7b44f1fed84c 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -319,7 +319,7 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBu for (int i = 0; i < valuesCount; i++) { double value = values.nextValue(); if (false == timeBucketMetrics.containsKey(preRounding)) { - timeBucketMetrics.put(preRounding, downsampleFunction.getAggregatorFunction()); + timeBucketMetrics.put(preRounding, downsampleFunction.getAggregatorFunction(this)); } for (Entry entry : timeBucketMetrics.entrySet()) { Long timestamp = entry.getKey(); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/RateFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/RateFunction.java new file mode 100644 index 0000000000000..578f8c92d796c --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/RateFunction.java @@ -0,0 +1,126 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.function; + +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesRate; + +import java.util.Map; + +public class RateFunction implements AggregatorFunction { + + private final long range; + private final long timestamp; + private final boolean isCounter; + private final boolean isRate; + private TimePoint lastSample; + private TimePoint firstSample; + private TimePoint currentSample; + private long count; + private double totalRevertValue = 0; + + public RateFunction(long range, long timestamp, boolean isCounter, boolean isRate) { + this.timestamp = timestamp; + this.range = range; + this.isCounter = isCounter; + this.isRate = isRate; + } + + @Override + public void collect(TimePoint value) { + if (value.compareTo(lastSample) > 0) { + lastSample = value; + } + if (value.compareTo(firstSample) < 0) { + firstSample = value; + } + count += 1; + + if (currentSample.compareTo(value) > 0 && currentSample.getValue() < value.getValue()) { + totalRevertValue += value.getValue(); + } + + currentSample = value; + } + + @Override + public TimePoint get() { + long rangeStart = timestamp - range; + long rangeEnd = timestamp; + + if (count < 2) { + return new TimePoint(timestamp, 0d); + } + + double resultValue = lastSample.getValue() - firstSample.getValue(); + if (isCounter) { + // TODO + // double lastValue = 0; + // for (Tuple sample : samples) { + // if (sample.v2() < lastValue) { + // resultValue += lastValue; + // } + // lastValue = sample.v2(); + // } + resultValue += totalRevertValue; + } + + double durationToStart = (firstSample.getTimestamp() - rangeStart) / 1000; + double durationToEnd = (rangeEnd - lastSample.getTimestamp()) / 1000; + + double sampledInterval = (lastSample.getTimestamp() - firstSample.getTimestamp()) / 1000; + double averageDurationBetweenSamples = sampledInterval / (count - 1); + + if (isCounter && resultValue > 0 && firstSample.getValue() >= 0) { + double durationToZero = sampledInterval * (firstSample.getValue() / resultValue); + if (durationToZero < durationToStart) { + durationToStart = durationToZero; + } + } + + double extrapolationThreshold = averageDurationBetweenSamples * 1.1; + double extrapolateToInterval = sampledInterval; + + if (durationToStart < extrapolationThreshold) { + extrapolateToInterval += durationToStart; + } else { + extrapolateToInterval += averageDurationBetweenSamples / 2; + } + if (durationToEnd < extrapolationThreshold) { + extrapolateToInterval += durationToEnd; + } else { + extrapolateToInterval += averageDurationBetweenSamples / 2; + } + resultValue = resultValue * (extrapolateToInterval / sampledInterval); + if (isRate) { + resultValue = resultValue / range; + } + + return new TimePoint(timestamp, resultValue); + } + + @Override + public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { + return new TimeSeriesRate( + TimeSeriesRate.NAME, + range, + timestamp, + isCounter, + isRate, + lastSample, + firstSample, + count, + totalRevertValue, + formatter, + metadata + ); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java index 5ffab5abc4464..d9a2ee87fcf5c 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java @@ -78,7 +78,7 @@ public InternalAggregation reduce(List aggregations, Aggreg TSIDInternalAggregation tsidAgg = (TSIDInternalAggregation) aggregations.get(0); if (reduceContext.isFinalReduce()) { Function function = Function.valueOf(aggreagator); - final AggregatorFunction aggregatorFunction = function.getAggregatorFunction(); + final AggregatorFunction aggregatorFunction = function.getAggregatorFunction(null); tsidAgg.values.forEach( (tsid, agg) -> { aggregatorFunction.collect(((InternalNumericMetricsAggregation.SingleValue) agg).value()); } ); @@ -103,7 +103,7 @@ public InternalAggregation reduce(List aggregations, Aggreg if (reduceContext.isFinalReduce()) { Function function = Function.valueOf(aggreagator); - final AggregatorFunction aggregatorFunction = function.getAggregatorFunction(); + final AggregatorFunction aggregatorFunction = function.getAggregatorFunction(null); reduced.forEach((tsid, aggs) -> { if (aggs.size() > 0) { InternalAggregation first = aggs.get(0); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesRate.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesRate.java new file mode 100644 index 0000000000000..b4d09f3340eb5 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesRate.java @@ -0,0 +1,218 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.internal; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.AggregationReduceContext; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; +import org.elasticsearch.search.aggregations.support.SamplingContext; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; + +public class TimeSeriesRate extends InternalNumericMetricsAggregation.SingleValue implements Comparable { + public static final String NAME = "rate"; + + private final long range; + private final long timestamp; + private final boolean isCounter; + private final boolean isRate; + private TimePoint lastSample; + private TimePoint firstSample; + private long count; + private double totalRevertValue; + private double resultValue = -1d; + + public TimeSeriesRate( + String name, + long range, + long timestamp, + boolean isCounter, + boolean isRate, + TimePoint lastSample, + TimePoint firstSample, + long count, + double totalRevertValue, + DocValueFormat formatter, + Map metadata + ) { + super(name, formatter, metadata); + this.range = range; + this.timestamp = timestamp; + this.isCounter = isCounter; + this.isRate = isRate; + this.lastSample = lastSample; + this.firstSample = firstSample; + this.count = count; + this.totalRevertValue = totalRevertValue; + } + + /** + * Read from a stream. + */ + public TimeSeriesRate(StreamInput in) throws IOException { + super(in); + range = in.readLong(); + timestamp = in.readLong(); + isCounter = in.readBoolean(); + isRate = in.readBoolean(); + lastSample = new TimePoint(in); + firstSample = new TimePoint(in); + count = in.readLong(); + totalRevertValue = in.readDouble(); + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + out.writeNamedWriteable(format); + out.writeLong(range); + out.writeLong(timestamp); + out.writeBoolean(isCounter); + out.writeBoolean(isRate); + lastSample.writeTo(out); + firstSample.writeTo(out); + out.writeLong(count); + out.writeDouble(totalRevertValue); + } + + @Override + public String getWriteableName() { + return NAME; + } + + public InternalAggregation finalizeSampling(SamplingContext samplingContext) { + return this; + } + + @Override + public double value() { + if (resultValue < 0) { + resultValue = calc(); + } + return resultValue; + } + + public double calc() { + long rangeStart = timestamp - range; + long rangeEnd = timestamp; + + if (count < 2) { + return 0d; + } + + double resultValue = lastSample.getValue() - firstSample.getValue(); + if (isCounter) { + resultValue += totalRevertValue; + } + + double durationToStart = (firstSample.getTimestamp() - rangeStart) / 1000; + double durationToEnd = (rangeEnd - lastSample.getTimestamp()) / 1000; + + double sampledInterval = (lastSample.getTimestamp() - firstSample.getTimestamp()) / 1000; + double averageDurationBetweenSamples = sampledInterval / (count - 1); + + if (isCounter && resultValue > 0 && firstSample.getValue() >= 0) { + double durationToZero = sampledInterval * (firstSample.getValue() / resultValue); + if (durationToZero < durationToStart) { + durationToStart = durationToZero; + } + } + + double extrapolationThreshold = averageDurationBetweenSamples * 1.1; + double extrapolateToInterval = sampledInterval; + + if (durationToStart < extrapolationThreshold) { + extrapolateToInterval += durationToStart; + } else { + extrapolateToInterval += averageDurationBetweenSamples / 2; + } + if (durationToEnd < extrapolationThreshold) { + extrapolateToInterval += durationToEnd; + } else { + extrapolateToInterval += averageDurationBetweenSamples / 2; + } + resultValue = resultValue * (extrapolateToInterval / sampledInterval); + if (isRate) { + resultValue = resultValue / range; + } + + return resultValue; + } + + @Override + public TimeSeriesRate reduce(List aggregations, AggregationReduceContext reduceContext) { + if (aggregations.size() == 1) { + return (TimeSeriesRate)aggregations.get(0); + } + + List timeSeriesRates = aggregations.stream().map(c->(TimeSeriesRate)c).sorted().collect(Collectors.toList()); + + TimeSeriesRate reduced = timeSeriesRates.get(0); + for (int i = 1; i < timeSeriesRates.size(); i++) { + TimeSeriesRate timeSeriesRate = timeSeriesRates.get(i); + reduced.count += timeSeriesRate.count; + reduced.lastSample = timeSeriesRate.lastSample; + reduced.totalRevertValue += timeSeriesRate.totalRevertValue; + if (timeSeriesRate.firstSample.getValue() < reduced.lastSample.getValue()) { + reduced.totalRevertValue += reduced.lastSample.getValue() - timeSeriesRate.firstSample.getValue(); + } + } + return reduced; + } + + @Override + public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + boolean hasValue = Double.isInfinite(value()) == false; + builder.field(CommonFields.VALUE.getPreferredName(), hasValue ? value() : null); + if (hasValue && format != DocValueFormat.RAW) { + builder.field(CommonFields.VALUE_AS_STRING.getPreferredName(), format.format(value()).toString()); + } + return builder; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + TimeSeriesRate rate = (TimeSeriesRate) o; + return range == rate.range + && timestamp == rate.timestamp + && isCounter == rate.isCounter + && isRate == rate.isRate + && count == rate.count + && Double.compare(rate.totalRevertValue, totalRevertValue) == 0 + && Objects.equals(lastSample, rate.lastSample) + && Objects.equals(firstSample, rate.firstSample); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), range, timestamp, isCounter, isRate, lastSample, firstSample, count, totalRevertValue); + } + + @Override + public int compareTo(TimeSeriesRate o) { + return firstSample.compareTo(o.firstSample); + } +} diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java index 180da82628ff0..32c1111e45966 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java @@ -102,7 +102,7 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBu for (int i = 0; i < valuesCount; i++) { double value = values.nextValue(); if (false == timeBucketMetrics.containsKey(preRounding)) { - timeBucketMetrics.put(preRounding, downsampleFunction.getAggregatorFunction()); + timeBucketMetrics.put(preRounding, downsampleFunction.getAggregatorFunction(this)); } for (Entry entry : timeBucketMetrics.entrySet()) { Long timestamp = entry.getKey(); @@ -143,7 +143,7 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBu } if (false == timeBucketMetrics.containsKey(preRounding)) { - timeBucketMetrics.put(preRounding, downsampleFunction.getAggregatorFunction()); + timeBucketMetrics.put(preRounding, downsampleFunction.getAggregatorFunction(this)); } for (Entry entry : timeBucketMetrics.entrySet()) { Long timestamp = entry.getKey(); From 9caaad6a448604e92d4261c2d37ae47dc4cc201f Mon Sep 17 00:00:00 2001 From: weizijun Date: Thu, 12 May 2022 11:51:08 +0800 Subject: [PATCH 11/53] add rate support --- .../timeseries/aggregation/Function.java | 67 +++++++++++++------ .../TimeSeriesAggregationAggregator.java | 2 +- .../internal/TSIDInternalAggregation.java | 4 +- ...MetricTimeSeriesAggregationAggregator.java | 5 +- 4 files changed, 51 insertions(+), 27 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java index 33ac7c798b060..86dda654082b2 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java @@ -14,6 +14,7 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.MaxBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.MinBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.SumBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.ValueCountBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AvgFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.LastFunction; @@ -26,18 +27,28 @@ public enum Function { count { @Override - public AggregatorFunction getAggregatorFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new ValueCountFunction(); + } + + @Override + public AggregatorFunction getAggregatorFunction() { return new ValueCountFunction(); } @Override public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { - throw new UnsupportedOperationException("count aggregator bucket function not support"); + return new ValueCountBucketFunction(bigArrays); } }, sum { @Override - public AggregatorFunction getAggregatorFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new SumFunction(); + } + + @Override + public AggregatorFunction getAggregatorFunction() { return new SumFunction(); } @@ -48,7 +59,12 @@ public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArra }, min { @Override - public AggregatorFunction getAggregatorFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new MinFunction(); + } + + @Override + public AggregatorFunction getAggregatorFunction() { return new MinFunction(); } @@ -59,7 +75,12 @@ public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArra }, max { @Override - public AggregatorFunction getAggregatorFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new MaxFunction(); + } + + @Override + public AggregatorFunction getAggregatorFunction() { return new MaxFunction(); } @@ -70,7 +91,12 @@ public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArra }, avg { @Override - public AggregatorFunction getAggregatorFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new AvgFunction(); + } + + @Override + public AggregatorFunction getAggregatorFunction() { return new AvgFunction(); } @@ -81,24 +107,14 @@ public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArra }, last { @Override - public AggregatorFunction getAggregatorFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { return new LastFunction(); } - - @Override - public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { - throw new UnsupportedOperationException("last aggregator bucket function not support"); - } }, rate { @Override - public AggregatorFunction getAggregatorFunction(TimeSeriesAggregationAggregator aggregator) { - return new RateFunction(0, 0, true, true); - } - - @Override - public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { - throw new UnsupportedOperationException("last aggregator bucket function not support"); + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new RateFunction(aggregator.downsampleRange, aggregator.preRounding, true, true); } }; @@ -106,13 +122,22 @@ public static Function resolve(String name) { return Function.valueOf(name); } + /** + * get the function + */ + public abstract AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator); + /** * get the aggregator function */ - public abstract AggregatorFunction getAggregatorFunction(TimeSeriesAggregationAggregator aggregator); + public AggregatorFunction getAggregatorFunction() { + throw new UnsupportedOperationException(name() + " aggregator function not support"); + }; /** * get the aggregator bucket function */ - public abstract AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays); + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { + throw new UnsupportedOperationException(name() + " aggregator bucket function not support"); + }; } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index a7b44f1fed84c..3d25c3d162a74 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -319,7 +319,7 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBu for (int i = 0; i < valuesCount; i++) { double value = values.nextValue(); if (false == timeBucketMetrics.containsKey(preRounding)) { - timeBucketMetrics.put(preRounding, downsampleFunction.getAggregatorFunction(this)); + timeBucketMetrics.put(preRounding, downsampleFunction.getFunction(this)); } for (Entry entry : timeBucketMetrics.entrySet()) { Long timestamp = entry.getKey(); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java index d9a2ee87fcf5c..5ffab5abc4464 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java @@ -78,7 +78,7 @@ public InternalAggregation reduce(List aggregations, Aggreg TSIDInternalAggregation tsidAgg = (TSIDInternalAggregation) aggregations.get(0); if (reduceContext.isFinalReduce()) { Function function = Function.valueOf(aggreagator); - final AggregatorFunction aggregatorFunction = function.getAggregatorFunction(null); + final AggregatorFunction aggregatorFunction = function.getAggregatorFunction(); tsidAgg.values.forEach( (tsid, agg) -> { aggregatorFunction.collect(((InternalNumericMetricsAggregation.SingleValue) agg).value()); } ); @@ -103,7 +103,7 @@ public InternalAggregation reduce(List aggregations, Aggreg if (reduceContext.isFinalReduce()) { Function function = Function.valueOf(aggreagator); - final AggregatorFunction aggregatorFunction = function.getAggregatorFunction(null); + final AggregatorFunction aggregatorFunction = function.getAggregatorFunction(); reduced.forEach((tsid, aggs) -> { if (aggs.size() > 0) { InternalAggregation first = aggs.get(0); diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java index 32c1111e45966..b7741c01bd25e 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java @@ -24,7 +24,6 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregationAggregator; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AvgFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.function.LastFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.ValueCountFunction; import org.elasticsearch.xpack.aggregatemetric.aggregations.support.AggregateMetricsValuesSource; import org.elasticsearch.xpack.aggregatemetric.mapper.AggregateDoubleMetricFieldMapper.Metric; @@ -102,7 +101,7 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBu for (int i = 0; i < valuesCount; i++) { double value = values.nextValue(); if (false == timeBucketMetrics.containsKey(preRounding)) { - timeBucketMetrics.put(preRounding, downsampleFunction.getAggregatorFunction(this)); + timeBucketMetrics.put(preRounding, downsampleFunction.getFunction(this)); } for (Entry entry : timeBucketMetrics.entrySet()) { Long timestamp = entry.getKey(); @@ -143,7 +142,7 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBu } if (false == timeBucketMetrics.containsKey(preRounding)) { - timeBucketMetrics.put(preRounding, downsampleFunction.getAggregatorFunction(this)); + timeBucketMetrics.put(preRounding, downsampleFunction.getFunction(this)); } for (Entry entry : timeBucketMetrics.entrySet()) { Long timestamp = entry.getKey(); From 9ad347b016884ba02a4dd4fd92e2c9c0f67431ee Mon Sep 17 00:00:00 2001 From: weizijun Date: Thu, 12 May 2022 14:37:13 +0800 Subject: [PATCH 12/53] add rate support --- .../elasticsearch/search/SearchModule.java | 2 ++ .../TimeSeriesAggregationAggregator.java | 7 +----- .../aggregation/function/AvgFunction.java | 7 +++--- .../aggregation/function/LastFunction.java | 23 +++++++------------ .../aggregation/function/MaxFunction.java | 7 +++--- .../aggregation/function/MinFunction.java | 7 +++--- .../aggregation/function/RateFunction.java | 9 +++++++- .../aggregation/function/SumFunction.java | 7 +++--- .../function/ValueCountFunction.java | 5 ++-- .../internal/TSIDInternalAggregation.java | 13 ++++------- .../aggregation/internal/TimeSeriesRate.java | 2 +- 11 files changed, 44 insertions(+), 45 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/SearchModule.java b/server/src/main/java/org/elasticsearch/search/SearchModule.java index 4296ec6f66e04..ee75e2b35a8e3 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchModule.java +++ b/server/src/main/java/org/elasticsearch/search/SearchModule.java @@ -219,6 +219,7 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregationAggregationBuilder; import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.Last; import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TSIDInternalAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesRate; import org.elasticsearch.search.fetch.FetchPhase; import org.elasticsearch.search.fetch.FetchSubPhase; import org.elasticsearch.search.fetch.subphase.ExplainPhase; @@ -677,6 +678,7 @@ private ValuesSourceRegistry registerAggregations(List plugins) { ).addResultReader(InternalTimeSeriesAggregation::new) .addResultReader(Last.NAME, Last::new) .addResultReader(TSIDInternalAggregation.NAME, TSIDInternalAggregation::new) + .addResultReader(TimeSeriesRate.NAME, TimeSeriesRate::new) .setAggregatorRegistrar(TimeSeriesAggregationAggregationBuilder::registerAggregators), builder ); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index 3d25c3d162a74..0e668b049a9fd 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -325,12 +325,7 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBu Long timestamp = entry.getKey(); AggregatorFunction function = entry.getValue(); if (aggCtx.getTimestamp() + downsampleRange > timestamp) { - // TODO replace this instanceof logic - if (function instanceof LastFunction last) { - last.collectExact(value, aggCtx.getTimestamp()); - } else { - function.collect(value); - } + function.collect(new TimePoint(aggCtx.getTimestamp(), value)); } else { break; } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AvgFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AvgFunction.java index e240092ddd402..4bd22d3e11b2d 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AvgFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AvgFunction.java @@ -11,16 +11,17 @@ import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.AvgAggregationBuilder; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; import java.util.Map; -public class AvgFunction implements AggregatorFunction { +public class AvgFunction implements AggregatorFunction { private double sum = 0; private long count = 0; @Override - public void collect(Double value) { - sum += value; + public void collect(TimePoint value) { + sum += value.getValue(); count += 1; } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java index c0b5e5cd76b40..d9c7cf9cab91d 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java @@ -10,35 +10,28 @@ import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.Last; import java.util.Map; -public class LastFunction implements AggregatorFunction { - private double last = Double.NEGATIVE_INFINITY; - private long timestamp = Long.MIN_VALUE; +public class LastFunction implements AggregatorFunction { + private TimePoint point; @Override - public void collect(Double value) { - if (last != Double.NEGATIVE_INFINITY) { - last = value; - } - } - - public void collectExact(double number, long timestamp) { - if (timestamp > this.timestamp) { - this.timestamp = timestamp; - last = number; + public void collect(TimePoint value) { + if (this.point == null || value.getTimestamp() > this.point.getTimestamp()) { + this.point = value; } } @Override public Double get() { - return last; + return point.getValue(); } @Override public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { - return new Last("last", last, timestamp, formatter, metadata); + return new Last("last", point.getValue(), point.getTimestamp(), formatter, metadata); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/MaxFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/MaxFunction.java index e443eaed800fa..261e01a20ad06 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/MaxFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/MaxFunction.java @@ -11,15 +11,16 @@ import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; import java.util.Map; -public class MaxFunction implements AggregatorFunction { +public class MaxFunction implements AggregatorFunction { private Double max = Double.NEGATIVE_INFINITY; @Override - public void collect(Double value) { - this.max = Math.max(value, max); + public void collect(TimePoint value) { + this.max = Math.max(value.getValue(), max); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/MinFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/MinFunction.java index e2600654d8579..2153ce9310053 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/MinFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/MinFunction.java @@ -11,15 +11,16 @@ import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.MinAggregationBuilder; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; import java.util.Map; -public class MinFunction implements AggregatorFunction { +public class MinFunction implements AggregatorFunction { private Double min = Double.POSITIVE_INFINITY; @Override - public void collect(Double value) { - this.min = Math.min(value, min); + public void collect(TimePoint value) { + this.min = Math.min(value.getValue(), min); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/RateFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/RateFunction.java index 578f8c92d796c..bdb59068587e2 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/RateFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/RateFunction.java @@ -36,13 +36,20 @@ public RateFunction(long range, long timestamp, boolean isCounter, boolean isRat @Override public void collect(TimePoint value) { + count += 1; + if (firstSample == null) { + firstSample = value; + lastSample = value; + currentSample = value; + return; + } + if (value.compareTo(lastSample) > 0) { lastSample = value; } if (value.compareTo(firstSample) < 0) { firstSample = value; } - count += 1; if (currentSample.compareTo(value) > 0 && currentSample.getValue() < value.getValue()) { totalRevertValue += value.getValue(); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/SumFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/SumFunction.java index 51eb7906fb7ca..f9984d415a9c8 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/SumFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/SumFunction.java @@ -11,15 +11,16 @@ import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; import java.util.Map; -public class SumFunction implements AggregatorFunction { +public class SumFunction implements AggregatorFunction { private double sum = 0; @Override - public void collect(Double value) { - this.sum += value; + public void collect(TimePoint value) { + this.sum += value.getValue(); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountFunction.java index c1d62a79960e4..dee2d2da40ce1 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountFunction.java @@ -11,14 +11,15 @@ import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; import java.util.Map; -public class ValueCountFunction implements AggregatorFunction { +public class ValueCountFunction implements AggregatorFunction { private long count = 0; @Override - public void collect(Double value) { + public void collect(TimePoint value) { count += 1; } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java index 5ffab5abc4464..2d39f6801f3e1 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java @@ -17,9 +17,9 @@ import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation.SingleValue; import org.elasticsearch.search.aggregations.timeseries.aggregation.Function; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.TSIDBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.function.LastFunction; import org.elasticsearch.xcontent.XContentBuilder; import java.io.IOException; @@ -80,7 +80,9 @@ public InternalAggregation reduce(List aggregations, Aggreg Function function = Function.valueOf(aggreagator); final AggregatorFunction aggregatorFunction = function.getAggregatorFunction(); tsidAgg.values.forEach( - (tsid, agg) -> { aggregatorFunction.collect(((InternalNumericMetricsAggregation.SingleValue) agg).value()); } + (tsid, agg) -> { + aggregatorFunction.collect(new TimePoint(0, ((InternalNumericMetricsAggregation.SingleValue) agg).value())); + } ); return aggregatorFunction.getAggregation(formatter, getMetadata()); } else { @@ -108,12 +110,7 @@ public InternalAggregation reduce(List aggregations, Aggreg if (aggs.size() > 0) { InternalAggregation first = aggs.get(0); InternalNumericMetricsAggregation.SingleValue internalAggregation = (SingleValue) first.reduce(aggs, reduceContext); - // TODO replace this instanceof logic - if (internalAggregation instanceof Last last && aggregatorFunction instanceof LastFunction lastFunction) { - lastFunction.collectExact(last.value(), last.getTimestamp()); - } else { - aggregatorFunction.collect(internalAggregation.value()); - } + aggregatorFunction.collect(new TimePoint(0, internalAggregation.value())); } }); return aggregatorFunction.getAggregation(formatter, getMetadata()); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesRate.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesRate.java index b4d09f3340eb5..49a0ab7549dd1 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesRate.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesRate.java @@ -25,7 +25,7 @@ import java.util.stream.Collectors; public class TimeSeriesRate extends InternalNumericMetricsAggregation.SingleValue implements Comparable { - public static final String NAME = "rate"; + public static final String NAME = "time_series_rate"; private final long range; private final long timestamp; From 354cdb8672347c1a9a1f72f9da9dd6b150169ea7 Mon Sep 17 00:00:00 2001 From: weizijun Date: Thu, 12 May 2022 15:20:13 +0800 Subject: [PATCH 13/53] add rate support --- .../timeseries/aggregation/Function.java | 12 ++++ .../aggregation/function/RateFunction.java | 63 ++++++++++--------- .../aggregation/internal/TimeSeriesRate.java | 63 ++++--------------- 3 files changed, 59 insertions(+), 79 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java index 86dda654082b2..44ca35585c790 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java @@ -116,6 +116,18 @@ public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArra public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { return new RateFunction(aggregator.downsampleRange, aggregator.preRounding, true, true); } + }, + delta { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new RateFunction(aggregator.downsampleRange, aggregator.preRounding, false, false); + } + }, + increase { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new RateFunction(aggregator.downsampleRange, aggregator.preRounding, true, false); + } }; public static Function resolve(String name) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/RateFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/RateFunction.java index bdb59068587e2..de5c89fe2bd4c 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/RateFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/RateFunction.java @@ -15,7 +15,7 @@ import java.util.Map; -public class RateFunction implements AggregatorFunction { +public class RateFunction implements AggregatorFunction { private final long range; private final long timestamp; @@ -59,24 +59,46 @@ public void collect(TimePoint value) { } @Override - public TimePoint get() { + public Double get() { + return extrapolatedRate(range, timestamp, isCounter, isRate, lastSample, firstSample, count, totalRevertValue); + } + + @Override + public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { + return new TimeSeriesRate( + TimeSeriesRate.NAME, + range, + timestamp, + isCounter, + isRate, + lastSample, + firstSample, + count, + totalRevertValue, + formatter, + metadata + ); + } + + public static double extrapolatedRate( + long range, + long timestamp, + boolean isCounter, + boolean isRate, + TimePoint lastSample, + TimePoint firstSample, + long count, + double totalRevertValue + ) { long rangeStart = timestamp - range; long rangeEnd = timestamp; if (count < 2) { - return new TimePoint(timestamp, 0d); + return 0d; } double resultValue = lastSample.getValue() - firstSample.getValue(); if (isCounter) { - // TODO - // double lastValue = 0; - // for (Tuple sample : samples) { - // if (sample.v2() < lastValue) { - // resultValue += lastValue; - // } - // lastValue = sample.v2(); - // } resultValue += totalRevertValue; } @@ -111,23 +133,6 @@ public TimePoint get() { resultValue = resultValue / range; } - return new TimePoint(timestamp, resultValue); - } - - @Override - public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { - return new TimeSeriesRate( - TimeSeriesRate.NAME, - range, - timestamp, - isCounter, - isRate, - lastSample, - firstSample, - count, - totalRevertValue, - formatter, - metadata - ); + return resultValue; } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesRate.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesRate.java index 49a0ab7549dd1..7eb08c0373bb4 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesRate.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesRate.java @@ -16,6 +16,7 @@ import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; import org.elasticsearch.search.aggregations.support.SamplingContext; import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.RateFunction; import org.elasticsearch.xcontent.XContentBuilder; import java.io.IOException; @@ -101,65 +102,27 @@ public InternalAggregation finalizeSampling(SamplingContext samplingContext) { @Override public double value() { if (resultValue < 0) { - resultValue = calc(); + resultValue = RateFunction.extrapolatedRate( + range, + timestamp, + isCounter, + isRate, + lastSample, + firstSample, + count, + totalRevertValue + ); } return resultValue; } - public double calc() { - long rangeStart = timestamp - range; - long rangeEnd = timestamp; - - if (count < 2) { - return 0d; - } - - double resultValue = lastSample.getValue() - firstSample.getValue(); - if (isCounter) { - resultValue += totalRevertValue; - } - - double durationToStart = (firstSample.getTimestamp() - rangeStart) / 1000; - double durationToEnd = (rangeEnd - lastSample.getTimestamp()) / 1000; - - double sampledInterval = (lastSample.getTimestamp() - firstSample.getTimestamp()) / 1000; - double averageDurationBetweenSamples = sampledInterval / (count - 1); - - if (isCounter && resultValue > 0 && firstSample.getValue() >= 0) { - double durationToZero = sampledInterval * (firstSample.getValue() / resultValue); - if (durationToZero < durationToStart) { - durationToStart = durationToZero; - } - } - - double extrapolationThreshold = averageDurationBetweenSamples * 1.1; - double extrapolateToInterval = sampledInterval; - - if (durationToStart < extrapolationThreshold) { - extrapolateToInterval += durationToStart; - } else { - extrapolateToInterval += averageDurationBetweenSamples / 2; - } - if (durationToEnd < extrapolationThreshold) { - extrapolateToInterval += durationToEnd; - } else { - extrapolateToInterval += averageDurationBetweenSamples / 2; - } - resultValue = resultValue * (extrapolateToInterval / sampledInterval); - if (isRate) { - resultValue = resultValue / range; - } - - return resultValue; - } - @Override public TimeSeriesRate reduce(List aggregations, AggregationReduceContext reduceContext) { if (aggregations.size() == 1) { - return (TimeSeriesRate)aggregations.get(0); + return (TimeSeriesRate) aggregations.get(0); } - List timeSeriesRates = aggregations.stream().map(c->(TimeSeriesRate)c).sorted().collect(Collectors.toList()); + List timeSeriesRates = aggregations.stream().map(c -> (TimeSeriesRate) c).sorted().collect(Collectors.toList()); TimeSeriesRate reduced = timeSeriesRates.get(0); for (int i = 1; i < timeSeriesRates.size(); i++) { From 993f6f7f4735d943a97c64105b3c003d2f5f3075 Mon Sep 17 00:00:00 2001 From: weizijun Date: Thu, 12 May 2022 16:02:27 +0800 Subject: [PATCH 14/53] add irate support --- .../TimeSeriesAggregationsIT.java | 2 +- .../elasticsearch/search/SearchModule.java | 2 + .../timeseries/aggregation/Function.java | 13 ++ .../aggregation/function/IRateFunction.java | 80 +++++++++ .../aggregation/function/LastFunction.java | 2 +- .../aggregation/internal/TimeSeriesIRate.java | 157 ++++++++++++++++++ .../aggregation/internal/TimeSeriesRate.java | 22 ++- 7 files changed, 270 insertions(+), 8 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/IRateFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesIRate.java diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java index a093c8af7af6d..aac6f700a43e2 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java @@ -580,7 +580,7 @@ public void testTimeSeriesAggregationsDownsample() { .addAggregation( timeSeriesAggregation("by_ts").field("metric_0") .interval(fixedInterval) - .downsample(fixedInterval, Function.sum) + .downsample(fixedInterval, Function.irate) .size(data.size()) ) .get(); diff --git a/server/src/main/java/org/elasticsearch/search/SearchModule.java b/server/src/main/java/org/elasticsearch/search/SearchModule.java index ee75e2b35a8e3..fb25cc05ef632 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchModule.java +++ b/server/src/main/java/org/elasticsearch/search/SearchModule.java @@ -219,6 +219,7 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregationAggregationBuilder; import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.Last; import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TSIDInternalAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesIRate; import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesRate; import org.elasticsearch.search.fetch.FetchPhase; import org.elasticsearch.search.fetch.FetchSubPhase; @@ -679,6 +680,7 @@ private ValuesSourceRegistry registerAggregations(List plugins) { .addResultReader(Last.NAME, Last::new) .addResultReader(TSIDInternalAggregation.NAME, TSIDInternalAggregation::new) .addResultReader(TimeSeriesRate.NAME, TimeSeriesRate::new) + .addResultReader(TimeSeriesIRate.NAME, TimeSeriesIRate::new) .setAggregatorRegistrar(TimeSeriesAggregationAggregationBuilder::registerAggregators), builder ); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java index 44ca35585c790..e5c499f807627 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java @@ -17,6 +17,7 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.ValueCountBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AvgFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.IRateFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.LastFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MaxFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MinFunction; @@ -128,6 +129,18 @@ public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArra public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { return new RateFunction(aggregator.downsampleRange, aggregator.preRounding, true, false); } + }, + irate { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new IRateFunction(true); + } + }, + idelta { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new IRateFunction(true); + } }; public static Function resolve(String name) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/IRateFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/IRateFunction.java new file mode 100644 index 0000000000000..e9dc8c16d644f --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/IRateFunction.java @@ -0,0 +1,80 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.function; + +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesIRate; + +import java.util.Map; + +/** + * @author weizijun.wzj + * @date 2022/5/12 + */ +public class IRateFunction implements AggregatorFunction { + + private final boolean isRate; + private TimePoint lastSample; + private TimePoint previousSample; + private long count; + + public IRateFunction(boolean isRate) { + this.isRate = isRate; + } + + @Override + public void collect(TimePoint value) { + count += 1; + if (lastSample == null) { + lastSample = value; + return; + } + + if (previousSample == null) { + previousSample = value; + return; + } + } + + @Override + public Double get() { + return instantValue(isRate, lastSample, previousSample, count); + } + + @Override + public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { + return new TimeSeriesIRate(TimeSeriesIRate.NAME, isRate, lastSample, previousSample, count, formatter, metadata); + } + + public static double instantValue(boolean isRate, TimePoint lastSample, TimePoint previousSample, long count) { + if (count < 2) { + return 0d; + } + + double resultValue; + if (isRate && lastSample.getValue() < previousSample.getValue()) { + resultValue = lastSample.getValue(); + } else { + resultValue = lastSample.getValue() - previousSample.getValue(); + } + + long sampledInterval = lastSample.getTimestamp() - previousSample.getTimestamp(); + if (sampledInterval == 0) { + return 0d; + } + + if (isRate) { + resultValue /= (double) sampledInterval / 1000; + } + + return resultValue; + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java index d9c7cf9cab91d..415f84b6302cd 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java @@ -32,6 +32,6 @@ public Double get() { @Override public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { - return new Last("last", point.getValue(), point.getTimestamp(), formatter, metadata); + return new Last(Last.NAME, point.getValue(), point.getTimestamp(), formatter, metadata); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesIRate.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesIRate.java new file mode 100644 index 0000000000000..dadf4a94657e7 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesIRate.java @@ -0,0 +1,157 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.internal; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.AggregationReduceContext; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; +import org.elasticsearch.search.aggregations.support.SamplingContext; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.IRateFunction; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; + +/** + * @author weizijun.wzj + * @date 2022/5/12 + */ +public class TimeSeriesIRate extends InternalNumericMetricsAggregation.SingleValue implements Comparable { + public static final String NAME = "time_series_irate"; + + private final boolean isRate; + private TimePoint lastSample; + private TimePoint previousSample; + private long count; + private double resultValue = -1d; + + public TimeSeriesIRate( + String name, + boolean isRate, + TimePoint lastSample, + TimePoint previousSample, + long count, + DocValueFormat formatter, + Map metadata + ) { + super(name, formatter, metadata); + this.isRate = isRate; + this.lastSample = lastSample; + this.previousSample = previousSample; + this.count = count; + } + + /** + * Read from a stream. + */ + public TimeSeriesIRate(StreamInput in) throws IOException { + super(in); + isRate = in.readBoolean(); + lastSample = in.readOptionalWriteable(TimePoint::new); + previousSample = in.readOptionalWriteable(TimePoint::new); + count = in.readLong(); + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + out.writeNamedWriteable(format); + out.writeBoolean(isRate); + out.writeOptionalWriteable(lastSample); + out.writeOptionalWriteable(previousSample); + out.writeLong(count); + } + + @Override + public String getWriteableName() { + return NAME; + } + + public InternalAggregation finalizeSampling(SamplingContext samplingContext) { + return this; + } + + @Override + public double value() { + if (resultValue < 0) { + resultValue = IRateFunction.instantValue(isRate, lastSample, previousSample, count); + } + return resultValue; + } + + @Override + public TimeSeriesIRate reduce(List aggregations, AggregationReduceContext reduceContext) { + if (aggregations.size() == 1) { + return (TimeSeriesIRate) aggregations.get(0); + } + + List timeSeriesIRates = aggregations.stream().map(c -> (TimeSeriesIRate) c).sorted().collect(Collectors.toList()); + + TimeSeriesIRate reduced = timeSeriesIRates.get(0); + for (int i = 1; i < timeSeriesIRates.size(); i++) { + TimeSeriesIRate timeSeriesIRate = timeSeriesIRates.get(i); + if (timeSeriesIRate.count == 0) { + continue; + } + reduced.count += timeSeriesIRate.count; + if (timeSeriesIRate.count == 1) { + reduced.previousSample = reduced.lastSample; + } else { + reduced.previousSample = timeSeriesIRate.previousSample; + } + reduced.lastSample = timeSeriesIRate.lastSample; + } + return reduced; + } + + @Override + public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + boolean hasValue = Double.isInfinite(value()) == false; + builder.field(CommonFields.VALUE.getPreferredName(), hasValue ? value() : null); + if (hasValue && format != DocValueFormat.RAW) { + builder.field(CommonFields.VALUE_AS_STRING.getPreferredName(), format.format(value()).toString()); + } + return builder; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + TimeSeriesIRate that = (TimeSeriesIRate) o; + return isRate == that.isRate + && count == that.count + && Double.compare(that.resultValue, resultValue) == 0 + && Objects.equals(lastSample, that.lastSample) + && Objects.equals(previousSample, that.previousSample); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), isRate, lastSample, previousSample, count, resultValue); + } + + @Override + public int compareTo(TimeSeriesIRate o) { + return lastSample.compareTo(o.lastSample); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesRate.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesRate.java index 7eb08c0373bb4..5be1cf0bcbbd4 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesRate.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesRate.java @@ -71,8 +71,8 @@ public TimeSeriesRate(StreamInput in) throws IOException { timestamp = in.readLong(); isCounter = in.readBoolean(); isRate = in.readBoolean(); - lastSample = new TimePoint(in); - firstSample = new TimePoint(in); + lastSample = in.readOptionalWriteable(TimePoint::new); + firstSample = in.readOptionalWriteable(TimePoint::new); count = in.readLong(); totalRevertValue = in.readDouble(); } @@ -84,8 +84,8 @@ protected void doWriteTo(StreamOutput out) throws IOException { out.writeLong(timestamp); out.writeBoolean(isCounter); out.writeBoolean(isRate); - lastSample.writeTo(out); - firstSample.writeTo(out); + out.writeOptionalWriteable(lastSample); + out.writeOptionalWriteable(firstSample); out.writeLong(count); out.writeDouble(totalRevertValue); } @@ -123,10 +123,12 @@ public TimeSeriesRate reduce(List aggregations, Aggregation } List timeSeriesRates = aggregations.stream().map(c -> (TimeSeriesRate) c).sorted().collect(Collectors.toList()); - TimeSeriesRate reduced = timeSeriesRates.get(0); for (int i = 1; i < timeSeriesRates.size(); i++) { TimeSeriesRate timeSeriesRate = timeSeriesRates.get(i); + if (timeSeriesRate.count == 0) { + continue; + } reduced.count += timeSeriesRate.count; reduced.lastSample = timeSeriesRate.lastSample; reduced.totalRevertValue += timeSeriesRate.totalRevertValue; @@ -176,6 +178,14 @@ public int hashCode() { @Override public int compareTo(TimeSeriesRate o) { - return firstSample.compareTo(o.firstSample); + if (firstSample == null && o.firstSample == null) { + return 0; + } else if (firstSample == null) { + return -1; + } else if (o.firstSample == null) { + return 1; + } else { + return firstSample.compareTo(o.firstSample); + } } } From eab24bfc4acfe02deedfcca12a5cd4bc396c001f Mon Sep 17 00:00:00 2001 From: weizijun Date: Thu, 12 May 2022 16:09:06 +0800 Subject: [PATCH 15/53] revert test --- .../search/aggregations/TimeSeriesAggregationsIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java index aac6f700a43e2..a093c8af7af6d 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java @@ -580,7 +580,7 @@ public void testTimeSeriesAggregationsDownsample() { .addAggregation( timeSeriesAggregation("by_ts").field("metric_0") .interval(fixedInterval) - .downsample(fixedInterval, Function.irate) + .downsample(fixedInterval, Function.sum) .size(data.size()) ) .get(); From 10ceecd2637493bddb1386b295773d6dbf870982 Mon Sep 17 00:00:00 2001 From: weizijun Date: Thu, 12 May 2022 16:46:45 +0800 Subject: [PATCH 16/53] add rate support --- .../timeseries/aggregation/function/RateFunction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/RateFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/RateFunction.java index de5c89fe2bd4c..ecad0591d2626 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/RateFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/RateFunction.java @@ -130,7 +130,7 @@ public static double extrapolatedRate( } resultValue = resultValue * (extrapolateToInterval / sampledInterval); if (isRate) { - resultValue = resultValue / range; + resultValue = resultValue / (range / 1000); } return resultValue; From dbfe860cca985521f53f3f5d0b31305cda7309d5 Mon Sep 17 00:00:00 2001 From: weizijun Date: Fri, 13 May 2022 17:28:06 +0800 Subject: [PATCH 17/53] fixed no_op error --- .../TimeSeriesAggregationAggregator.java | 18 +++++++++++++++++- ...eMetricTimeSeriesAggregationAggregator.java | 18 +++++++++++++++++- 2 files changed, 34 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index 0e668b049a9fd..3d03e611d6d7b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -11,6 +11,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.Scorable; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.PriorityQueue; import org.elasticsearch.common.Rounding; @@ -306,7 +307,22 @@ protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBu protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBucketCollector sub, AggregationExecutionContext aggCtx) throws IOException { if (valuesSource == null) { - return LeafBucketCollector.NO_OP_COLLECTOR; + return new LeafBucketCollector() { + @Override + public void setScorer(Scorable arg0) throws IOException { + // no-op + } + + @Override + public void collect(int doc, long bucket) { + // no-op + } + + @Override + public boolean isNoop() { + return false; + } + }; } final SortedNumericDoubleValues values = valuesSource.doubleValues(context); return new Collector(sub, values, aggCtx, (doc) -> { diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java index b7741c01bd25e..31d1ba26689b6 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java @@ -8,6 +8,7 @@ package org.elasticsearch.xpack.aggregatemetric.aggregations.metrics; import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.Scorable; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; import org.elasticsearch.search.aggregations.AggregationExecutionContext; import org.elasticsearch.search.aggregations.Aggregator; @@ -86,7 +87,22 @@ public AggregateMetricTimeSeriesAggregationAggregator( protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBucketCollector sub, AggregationExecutionContext aggCtx) throws IOException { if (valuesSource == null) { - return LeafBucketCollector.NO_OP_COLLECTOR; + return new LeafBucketCollector() { + @Override + public void setScorer(Scorable arg0) throws IOException { + // no-op + } + + @Override + public void collect(int doc, long bucket) { + // no-op + } + + @Override + public boolean isNoop() { + return false; + } + }; } Metric metricType = getAggregateMetric(); if (metricType != null) { From 96f453dbed9ec2e106c1736142089efda0031107 Mon Sep 17 00:00:00 2001 From: weizijun Date: Mon, 16 May 2022 10:33:02 +0800 Subject: [PATCH 18/53] code improve --- .../TimeSeriesAggregationsIT.java | 4 +- .../timeseries/aggregation/Aggregator.java | 99 +++++++++++++++++++ .../timeseries/aggregation/Function.java | 87 ++-------------- ...meSeriesAggregationAggregationBuilder.java | 12 +-- ...meSeriesAggregationAggregationFactory.java | 4 +- .../TimeSeriesAggregationAggregator.java | 4 +- ...meSeriesAggregationAggregatorSupplier.java | 2 +- .../aggregation/function/IRateFunction.java | 4 - .../internal/TSIDInternalAggregation.java | 18 ++-- .../aggregation/internal/TimeSeriesIRate.java | 4 - ...MetricTimeSeriesAggregationAggregator.java | 12 +-- 11 files changed, 137 insertions(+), 113 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java index a093c8af7af6d..6b8891b62921c 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java @@ -580,7 +580,7 @@ public void testTimeSeriesAggregationsDownsample() { .addAggregation( timeSeriesAggregation("by_ts").field("metric_0") .interval(fixedInterval) - .downsample(fixedInterval, Function.sum) + .downsample(fixedInterval, Function.sum_over_time) .size(data.size()) ) .get(); @@ -679,7 +679,7 @@ public void testTimeSeriesAggregationsGroupBy() { timeSeriesAggregation("by_ts").field("metric_0") .group(List.of("dim_0")) .interval(fixedInterval) - .downsample(fixedInterval, Function.max) + .downsample(fixedInterval, Function.max_over_time) .aggregator("sum") .size(data.size()) ) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java new file mode 100644 index 0000000000000..64406127c77e9 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java @@ -0,0 +1,99 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.AggregatorBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.AvgBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.MaxBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.MinBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.SumBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.ValueCountBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AvgFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MaxFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MinFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.SumFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.ValueCountFunction; + +public enum Aggregator { + count { + @Override + public AggregatorFunction getAggregatorFunction() { + return new ValueCountFunction(); + } + + @Override + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { + return new ValueCountBucketFunction(bigArrays); + } + }, + sum { + @Override + public AggregatorFunction getAggregatorFunction() { + return new SumFunction(); + } + + @Override + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { + return new SumBucketFunction(bigArrays); + } + }, + min { + @Override + public AggregatorFunction getAggregatorFunction() { + return new MinFunction(); + } + + @Override + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { + return new MinBucketFunction(bigArrays); + } + }, + max { + @Override + public AggregatorFunction getAggregatorFunction() { + return new MaxFunction(); + } + + @Override + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { + return new MaxBucketFunction(bigArrays); + } + }, + avg { + @Override + public AggregatorFunction getAggregatorFunction() { + return new AvgFunction(); + } + + @Override + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { + return new AvgBucketFunction(bigArrays); + } + }; + + public static Aggregator resolve(String name) { + try { + return Aggregator.valueOf(name); + } catch (Exception e){ + throw new IllegalArgumentException("aggregator [" + name + "] not support"); + } + } + + /** + * get the aggregator function + */ + public abstract AggregatorFunction getAggregatorFunction(); + + /** + * get the aggregator bucket function + */ + public abstract AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays); +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java index e5c499f807627..3106fb36deed4 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java @@ -8,13 +8,6 @@ package org.elasticsearch.search.aggregations.timeseries.aggregation; -import org.elasticsearch.common.util.BigArrays; -import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.AggregatorBucketFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.AvgBucketFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.MaxBucketFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.MinBucketFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.SumBucketFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.ValueCountBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AvgFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.IRateFunction; @@ -26,85 +19,35 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.function.ValueCountFunction; public enum Function { - count { + count_over_time { @Override public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { return new ValueCountFunction(); } - - @Override - public AggregatorFunction getAggregatorFunction() { - return new ValueCountFunction(); - } - - @Override - public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { - return new ValueCountBucketFunction(bigArrays); - } }, - sum { + sum_over_time { @Override public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { return new SumFunction(); } - - @Override - public AggregatorFunction getAggregatorFunction() { - return new SumFunction(); - } - - @Override - public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { - return new SumBucketFunction(bigArrays); - } }, - min { + min_over_time { @Override public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { return new MinFunction(); } - - @Override - public AggregatorFunction getAggregatorFunction() { - return new MinFunction(); - } - - @Override - public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { - return new MinBucketFunction(bigArrays); - } }, - max { + max_over_time { @Override public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { return new MaxFunction(); } - - @Override - public AggregatorFunction getAggregatorFunction() { - return new MaxFunction(); - } - - @Override - public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { - return new MaxBucketFunction(bigArrays); - } }, - avg { + avg_over_time { @Override public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { return new AvgFunction(); } - - @Override - public AggregatorFunction getAggregatorFunction() { - return new AvgFunction(); - } - - @Override - public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { - return new AvgBucketFunction(bigArrays); - } }, last { @Override @@ -144,25 +87,15 @@ public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArra }; public static Function resolve(String name) { - return Function.valueOf(name); + try { + return Function.valueOf(name); + } catch (Exception e) { + throw new IllegalArgumentException("function [" + name + "] not support"); + } } /** * get the function */ public abstract AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator); - - /** - * get the aggregator function - */ - public AggregatorFunction getAggregatorFunction() { - throw new UnsupportedOperationException(name() + " aggregator function not support"); - }; - - /** - * get the aggregator bucket function - */ - public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { - throw new UnsupportedOperationException(name() + " aggregator bucket function not support"); - }; } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java index 44f5e5777ba26..b8eb45454c268 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java @@ -72,7 +72,7 @@ public class TimeSeriesAggregationAggregationBuilder extends ValuesSourceAggrega private List without; private DateHistogramInterval interval; private DateHistogramInterval offset; - private String aggregator; + private Aggregator aggregator; private Downsample downsample; private TermsAggregator.BucketCountThresholds bucketCountThresholds = new TermsAggregator.BucketCountThresholds( DEFAULT_BUCKET_COUNT_THRESHOLDS @@ -140,7 +140,7 @@ public TimeSeriesAggregationAggregationBuilder(StreamInput in) throws IOExceptio without = in.readOptionalStringList(); interval = in.readOptionalWriteable(DateHistogramInterval::new); offset = in.readOptionalWriteable(DateHistogramInterval::new); - aggregator = in.readOptionalString(); + aggregator = in.readOptionalEnum(Aggregator.class); downsample = in.readOptionalWriteable(Downsample::new); order = InternalOrder.Streams.readOrder(in); bucketCountThresholds = new TermsAggregator.BucketCountThresholds(in); @@ -153,7 +153,7 @@ protected void innerWriteTo(StreamOutput out) throws IOException { out.writeOptionalStringCollection(without); out.writeOptionalWriteable(interval); out.writeOptionalWriteable(offset); - out.writeOptionalString(aggregator); + out.writeOptionalEnum(aggregator); out.writeOptionalWriteable(downsample); order.writeTo(out); bucketCountThresholds.writeTo(out); @@ -188,7 +188,7 @@ protected ValuesSourceAggregatorFactory innerBuild( without, interval, offset, - aggregator != null ? Function.resolve(aggregator) : null, + aggregator, downsample, bucketCountThresholds, order, @@ -325,7 +325,7 @@ public TimeSeriesAggregationAggregationBuilder offset(DateHistogramInterval offs /** * Returns the aggregator function */ - public String getAggregator() { + public Aggregator getAggregator() { return aggregator; } @@ -333,7 +333,7 @@ public String getAggregator() { * Sets the aggregator function, it used to aggregator time series lines to one time serie line */ public TimeSeriesAggregationAggregationBuilder aggregator(String aggregator) { - this.aggregator = aggregator; + this.aggregator = Aggregator.resolve(aggregator); return this; } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java index 8984b42f98b61..3040cd2e67fca 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java @@ -34,7 +34,7 @@ public class TimeSeriesAggregationAggregationFactory extends ValuesSourceAggrega private final List without; private final DateHistogramInterval interval; private final DateHistogramInterval offset; - private final Function aggregator; + private final org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator aggregator; private final Downsample downsample; private final TermsAggregator.BucketCountThresholds bucketCountThresholds; private final BucketOrder order; @@ -48,7 +48,7 @@ public TimeSeriesAggregationAggregationFactory( List without, DateHistogramInterval interval, DateHistogramInterval offset, - Function aggregator, + org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator aggregator, Downsample downsample, TermsAggregator.BucketCountThresholds bucketCountThresholds, BucketOrder order, diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index 3d03e611d6d7b..6b121dac67bca 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -74,7 +74,7 @@ public class TimeSeriesAggregationAggregator extends BucketsAggregator { private Set without; private long interval; private long offset; - private Function aggregator; + private Aggregator aggregator; protected long downsampleRange; protected Function downsampleFunction; private BucketOrder order; @@ -101,7 +101,7 @@ public TimeSeriesAggregationAggregator( List without, DateHistogramInterval interval, DateHistogramInterval offset, - Function aggregator, + Aggregator aggregator, Downsample downsample, TermsAggregator.BucketCountThresholds bucketCountThresholds, BucketOrder order, diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java index 98ce531f3ef7e..b89776b2680f1 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java @@ -31,7 +31,7 @@ Aggregator build( List without, DateHistogramInterval interval, DateHistogramInterval offset, - Function aggregator, + org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator aggregator, Downsample downsample, TermsAggregator.BucketCountThresholds bucketCountThresholds, BucketOrder order, diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/IRateFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/IRateFunction.java index e9dc8c16d644f..3d16a3a2a6411 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/IRateFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/IRateFunction.java @@ -15,10 +15,6 @@ import java.util.Map; -/** - * @author weizijun.wzj - * @date 2022/5/12 - */ public class IRateFunction implements AggregatorFunction { private final boolean isRate; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java index 2d39f6801f3e1..6dc29e47d350e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java @@ -16,7 +16,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation.SingleValue; -import org.elasticsearch.search.aggregations.timeseries.aggregation.Function; +import org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator; import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.TSIDBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; @@ -37,7 +37,7 @@ public class TSIDInternalAggregation extends InternalAggregation { public static final String NAME = "tsid"; private final Map values; - private final String aggreagator; + private final String aggregator; private final DocValueFormat formatter; public TSIDInternalAggregation( @@ -50,13 +50,13 @@ public TSIDInternalAggregation( super(name, metadata); this.values = values; this.formatter = formatter; - this.aggreagator = aggregator; + this.aggregator = aggregator; } public TSIDInternalAggregation(StreamInput in) throws IOException { super(in); formatter = in.readNamedWriteable(DocValueFormat.class); - aggreagator = in.readString(); + aggregator = in.readString(); values = in.readOrderedMap(StreamInput::readBytesRef, stream -> stream.readNamedWriteable(InternalAggregation.class)); } @@ -68,7 +68,7 @@ public String getWriteableName() { @Override protected void doWriteTo(StreamOutput out) throws IOException { out.writeNamedWriteable(formatter); - out.writeString(aggreagator); + out.writeString(aggregator); out.writeMap(values, StreamOutput::writeBytesRef, StreamOutput::writeNamedWriteable); } @@ -77,7 +77,7 @@ public InternalAggregation reduce(List aggregations, Aggreg if (aggregations.size() == 1) { TSIDInternalAggregation tsidAgg = (TSIDInternalAggregation) aggregations.get(0); if (reduceContext.isFinalReduce()) { - Function function = Function.valueOf(aggreagator); + Aggregator function = Aggregator.valueOf(aggregator); final AggregatorFunction aggregatorFunction = function.getAggregatorFunction(); tsidAgg.values.forEach( (tsid, agg) -> { @@ -104,7 +104,7 @@ public InternalAggregation reduce(List aggregations, Aggreg } if (reduceContext.isFinalReduce()) { - Function function = Function.valueOf(aggreagator); + Aggregator function = Aggregator.valueOf(aggregator); final AggregatorFunction aggregatorFunction = function.getAggregatorFunction(); reduced.forEach((tsid, aggs) -> { if (aggs.size() > 0) { @@ -122,7 +122,7 @@ public InternalAggregation reduce(List aggregations, Aggreg finalReduces.put(tsid, first.reduce(aggs, reduceContext)); } }); - return new TSIDInternalAggregation(name, finalReduces, aggreagator, formatter, getMetadata()); + return new TSIDInternalAggregation(name, finalReduces, aggregator, formatter, getMetadata()); } } @@ -138,7 +138,7 @@ public Object getProperty(List path) { @Override public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { - builder.field(CommonFields.VALUE.getPreferredName(), aggreagator); + builder.field(CommonFields.VALUE.getPreferredName(), aggregator); return builder; } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesIRate.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesIRate.java index dadf4a94657e7..700fd396f9ede 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesIRate.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesIRate.java @@ -25,10 +25,6 @@ import java.util.Objects; import java.util.stream.Collectors; -/** - * @author weizijun.wzj - * @date 2022/5/12 - */ public class TimeSeriesIRate extends InternalNumericMetricsAggregation.SingleValue implements Comparable { public static final String NAME = "time_series_irate"; diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java index 31d1ba26689b6..ee069b2743ef6 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java @@ -46,7 +46,7 @@ public AggregateMetricTimeSeriesAggregationAggregator( List without, DateHistogramInterval interval, DateHistogramInterval offset, - Function aggregator, + org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator aggregator, Downsample downsample, BucketCountThresholds bucketCountThresholds, BucketOrder order, @@ -79,7 +79,7 @@ public AggregateMetricTimeSeriesAggregationAggregator( : null; this.format = valuesSourceConfig.format(); if (this.downsampleFunction == null) { - this.downsampleFunction = Function.avg; + this.downsampleFunction = Function.avg_over_time; } } @@ -175,13 +175,13 @@ public boolean isNoop() { private Metric getAggregateMetric() { switch (downsampleFunction) { - case max: + case max_over_time: return Metric.max; - case min: + case min_over_time: return Metric.min; - case sum: + case sum_over_time: return Metric.sum; - case count: + case count_over_time: return Metric.value_count; } return null; From 4d1285abdd31510f1ab0b413efcd744a180320b2 Mon Sep 17 00:00:00 2001 From: weizijun Date: Thu, 19 May 2022 17:12:35 +0800 Subject: [PATCH 19/53] fix range boundary --- .../aggregation/TimeSeriesAggregationAggregator.java | 2 +- .../AggregateMetricTimeSeriesAggregationAggregator.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index 6b121dac67bca..bd215b5839ed6 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -340,7 +340,7 @@ public boolean isNoop() { for (Entry entry : timeBucketMetrics.entrySet()) { Long timestamp = entry.getKey(); AggregatorFunction function = entry.getValue(); - if (aggCtx.getTimestamp() + downsampleRange > timestamp) { + if (aggCtx.getTimestamp() + downsampleRange >= timestamp) { function.collect(new TimePoint(aggCtx.getTimestamp(), value)); } else { break; diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java index ee069b2743ef6..267e579eae4a7 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java @@ -122,7 +122,7 @@ public boolean isNoop() { for (Entry entry : timeBucketMetrics.entrySet()) { Long timestamp = entry.getKey(); AggregatorFunction function = entry.getValue(); - if (aggCtx.getTimestamp() + downsampleRange > timestamp) { + if (aggCtx.getTimestamp() + downsampleRange >= timestamp) { if (function instanceof ValueCountFunction) { ((ValueCountFunction) function).collectExact((long) value); } else { @@ -163,7 +163,7 @@ public boolean isNoop() { for (Entry entry : timeBucketMetrics.entrySet()) { Long timestamp = entry.getKey(); AggregatorFunction function = entry.getValue(); - if (aggCtx.getTimestamp() + downsampleRange > timestamp) { + if (aggCtx.getTimestamp() + downsampleRange >= timestamp) { ((AvgFunction) function).collectExact(sum, valueCount); } else { break; From c2737e393725e09cb1614380a23b6f4e52796438 Mon Sep 17 00:00:00 2001 From: weizijun Date: Fri, 20 May 2022 11:03:47 +0800 Subject: [PATCH 20/53] aggregator\function support params --- .../TimeSeriesAggregationsIT.java | 4 ++-- .../timeseries/aggregation/Downsample.java | 14 +++++++++-- ...meSeriesAggregationAggregationBuilder.java | 24 +++++++++++++++++-- ...meSeriesAggregationAggregationFactory.java | 5 ++++ .../TimeSeriesAggregationAggregator.java | 3 +++ ...meSeriesAggregationAggregatorSupplier.java | 1 + 6 files changed, 45 insertions(+), 6 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java index 6b8891b62921c..c88d0e46c6c76 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java @@ -580,7 +580,7 @@ public void testTimeSeriesAggregationsDownsample() { .addAggregation( timeSeriesAggregation("by_ts").field("metric_0") .interval(fixedInterval) - .downsample(fixedInterval, Function.sum_over_time) + .downsample(fixedInterval, Function.sum_over_time, null) .size(data.size()) ) .get(); @@ -679,7 +679,7 @@ public void testTimeSeriesAggregationsGroupBy() { timeSeriesAggregation("by_ts").field("metric_0") .group(List.of("dim_0")) .interval(fixedInterval) - .downsample(fixedInterval, Function.max_over_time) + .downsample(fixedInterval, Function.max_over_time, null) .aggregator("sum") .size(data.size()) ) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java index 561ed67b0b0d1..0062f22c5566a 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java @@ -20,17 +20,19 @@ import org.elasticsearch.xcontent.XContentParser; import java.io.IOException; +import java.util.Map; public class Downsample implements ToXContentObject, Writeable { public static final String NAME = "downsample"; public static final ParseField RANGE_FIELD = new ParseField("range"); public static final ParseField FUNCTION_FIELD = new ParseField("function"); + public static final ParseField PARAMS_FIELD = new ParseField("params"); public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( NAME, false, - (args, name) -> new Downsample((DateHistogramInterval) args[0], (Function) args[1]) + (args, name) -> new Downsample((DateHistogramInterval) args[0], (Function) args[1], (Map) args[2]) ); static { @@ -46,19 +48,23 @@ public class Downsample implements ToXContentObject, Writeable { FUNCTION_FIELD, ObjectParser.ValueType.STRING ); + PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(), (parser, c) -> parser.map(), PARAMS_FIELD); } private final DateHistogramInterval range; private final Function function; + private final Map params; - public Downsample(DateHistogramInterval range, Function function) { + public Downsample(DateHistogramInterval range, Function function, Map params) { this.range = range; this.function = function; + this.params = params; } public Downsample(StreamInput in) throws IOException { this.range = new DateHistogramInterval(in); this.function = Function.resolve(in.readString()); + this.params = in.readMap(); } public DateHistogramInterval getRange() { @@ -78,6 +84,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startObject(); builder.field(RANGE_FIELD.getPreferredName(), range); builder.field(FUNCTION_FIELD.getPreferredName(), function); + if (params != null) { + builder.field(PARAMS_FIELD.getPreferredName(), params); + } builder.endObject(); return builder; } @@ -86,5 +95,6 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws public void writeTo(StreamOutput out) throws IOException { range.writeTo(out); out.writeString(function.name()); + out.writeGenericMap(params); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java index b8eb45454c268..bf4bcf684b212 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java @@ -26,6 +26,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry.RegistryKey; import org.elasticsearch.search.aggregations.support.ValuesSourceType; +import org.elasticsearch.xcontent.ConstructingObjectParser; import org.elasticsearch.xcontent.ObjectParser; import org.elasticsearch.xcontent.ParseField; import org.elasticsearch.xcontent.XContentBuilder; @@ -43,6 +44,7 @@ public class TimeSeriesAggregationAggregationBuilder extends ValuesSourceAggrega public static final ParseField INTERVAL_FIELD = new ParseField("interval"); public static final ParseField OFFSET_FIELD = new ParseField("offset"); public static final ParseField AGGREGATOR_FIELD = new ParseField("aggregator"); + public static final ParseField AGGREGATOR_PARAMS_FIELD = new ParseField("aggregator_params"); public static final ParseField DOWNSAMPLE_FIELD = new ParseField("downsample"); public static final ParseField ORDER_FIELD = new ParseField("order"); public static final ParseField SIZE_FIELD = new ParseField("size"); @@ -73,6 +75,7 @@ public class TimeSeriesAggregationAggregationBuilder extends ValuesSourceAggrega private DateHistogramInterval interval; private DateHistogramInterval offset; private Aggregator aggregator; + private Map aggregatorParams; private Downsample downsample; private TermsAggregator.BucketCountThresholds bucketCountThresholds = new TermsAggregator.BucketCountThresholds( DEFAULT_BUCKET_COUNT_THRESHOLDS @@ -100,6 +103,7 @@ public class TimeSeriesAggregationAggregationBuilder extends ValuesSourceAggrega PARSER.declareStringArray(TimeSeriesAggregationAggregationBuilder::group, GROUP_FIELD); PARSER.declareStringArray(TimeSeriesAggregationAggregationBuilder::without, WITHOUT_FIELD); PARSER.declareString(TimeSeriesAggregationAggregationBuilder::aggregator, AGGREGATOR_FIELD); + PARSER.declareObject(TimeSeriesAggregationAggregationBuilder::aggregatorParams, (parser, c) -> parser.map(), AGGREGATOR_PARAMS_FIELD); PARSER.declareObject(TimeSeriesAggregationAggregationBuilder::downsample, (p, c) -> Downsample.fromXContent(p), DOWNSAMPLE_FIELD); PARSER.declareObjectArray( TimeSeriesAggregationAggregationBuilder::order, @@ -128,6 +132,7 @@ protected TimeSeriesAggregationAggregationBuilder( this.interval = clone.interval; this.offset = clone.offset; this.aggregator = clone.aggregator; + this.aggregatorParams = clone.aggregatorParams; this.downsample = clone.downsample; this.order = clone.order; this.bucketCountThresholds = clone.bucketCountThresholds; @@ -141,6 +146,7 @@ public TimeSeriesAggregationAggregationBuilder(StreamInput in) throws IOExceptio interval = in.readOptionalWriteable(DateHistogramInterval::new); offset = in.readOptionalWriteable(DateHistogramInterval::new); aggregator = in.readOptionalEnum(Aggregator.class); + aggregatorParams = in.readMap(); downsample = in.readOptionalWriteable(Downsample::new); order = InternalOrder.Streams.readOrder(in); bucketCountThresholds = new TermsAggregator.BucketCountThresholds(in); @@ -154,6 +160,7 @@ protected void innerWriteTo(StreamOutput out) throws IOException { out.writeOptionalWriteable(interval); out.writeOptionalWriteable(offset); out.writeOptionalEnum(aggregator); + out.writeGenericMap(aggregatorParams); out.writeOptionalWriteable(downsample); order.writeTo(out); bucketCountThresholds.writeTo(out); @@ -189,6 +196,7 @@ protected ValuesSourceAggregatorFactory innerBuild( interval, offset, aggregator, + aggregatorParams, downsample, bucketCountThresholds, order, @@ -219,6 +227,9 @@ protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) if (aggregator != null) { builder.field(AGGREGATOR_FIELD.getPreferredName(), aggregator); } + if (aggregatorParams != null) { + builder.field(AGGREGATOR_PARAMS_FIELD.getPreferredName(), aggregatorParams); + } if (downsample != null) { builder.field(DOWNSAMPLE_FIELD.getPreferredName(), downsample); } @@ -337,6 +348,15 @@ public TimeSeriesAggregationAggregationBuilder aggregator(String aggregator) { return this; } + public Map getAggregatorParams() { + return aggregatorParams; + } + + public TimeSeriesAggregationAggregationBuilder aggregatorParams(Map aggregatorParams) { + this.aggregatorParams = aggregatorParams; + return this; + } + /** * Sets the size - indicating how many term buckets should be returned * (defaults to 10) @@ -473,8 +493,8 @@ public TimeSeriesAggregationAggregationBuilder downsample(Downsample downsample) /** * Sets the downsample value */ - public TimeSeriesAggregationAggregationBuilder downsample(DateHistogramInterval range, Function function) { - this.downsample = new Downsample(range, function); + public TimeSeriesAggregationAggregationBuilder downsample(DateHistogramInterval range, Function function, Map params) { + this.downsample = new Downsample(range, function, params); return this; } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java index 3040cd2e67fca..8f95e91206787 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java @@ -35,6 +35,7 @@ public class TimeSeriesAggregationAggregationFactory extends ValuesSourceAggrega private final DateHistogramInterval interval; private final DateHistogramInterval offset; private final org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator aggregator; + private final Map aggregatorParams; private final Downsample downsample; private final TermsAggregator.BucketCountThresholds bucketCountThresholds; private final BucketOrder order; @@ -49,6 +50,7 @@ public TimeSeriesAggregationAggregationFactory( DateHistogramInterval interval, DateHistogramInterval offset, org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator aggregator, + Map aggregatorParams, Downsample downsample, TermsAggregator.BucketCountThresholds bucketCountThresholds, BucketOrder order, @@ -66,6 +68,7 @@ public TimeSeriesAggregationAggregationFactory( this.interval = interval; this.offset = offset; this.aggregator = aggregator; + this.aggregatorParams = aggregatorParams; this.downsample = downsample; this.bucketCountThresholds = bucketCountThresholds; this.order = order; @@ -102,6 +105,7 @@ protected Aggregator createUnmapped(Aggregator parent, Map metad interval, offset, aggregator, + aggregatorParams, downsample, thresholds, order, @@ -134,6 +138,7 @@ protected Aggregator doCreateInternal(Aggregator parent, CardinalityUpperBound c interval, offset, aggregator, + aggregatorParams, downsample, thresholds, order, diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index bd215b5839ed6..9959287bec1f7 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -75,6 +75,7 @@ public class TimeSeriesAggregationAggregator extends BucketsAggregator { private long interval; private long offset; private Aggregator aggregator; + private Map aggregatorParams; protected long downsampleRange; protected Function downsampleFunction; private BucketOrder order; @@ -102,6 +103,7 @@ public TimeSeriesAggregationAggregator( DateHistogramInterval interval, DateHistogramInterval offset, Aggregator aggregator, + Map aggregatorParams, Downsample downsample, TermsAggregator.BucketCountThresholds bucketCountThresholds, BucketOrder order, @@ -122,6 +124,7 @@ public TimeSeriesAggregationAggregator( this.rounding = Rounding.builder(new TimeValue(this.interval)).build().prepareForUnknown(); this.offset = offset != null ? offset.estimateMillis() : 0; this.aggregator = aggregator; + this.aggregatorParams = aggregatorParams; this.needAggregator = this.aggregator != null; this.downsampleRange = downsample != null ? downsample.getRange().estimateMillis() : -1; this.downsampleFunction = downsample != null ? downsample.getFunction() : Function.last; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java index b89776b2680f1..4f455b214f66b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java @@ -32,6 +32,7 @@ Aggregator build( DateHistogramInterval interval, DateHistogramInterval offset, org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator aggregator, + Map aggregatorParams, Downsample downsample, TermsAggregator.BucketCountThresholds bucketCountThresholds, BucketOrder order, From 7c73c9a499642b2c4a7fbc0d94256a5a3dd296eb Mon Sep 17 00:00:00 2001 From: weizijun Date: Mon, 23 May 2022 10:46:39 +0800 Subject: [PATCH 21/53] support topk --- .../timeseries/aggregation/Downsample.java | 1 + .../timeseries/aggregation/TSIDValue.java | 23 +++++++ .../TimeSeriesAggregationAggregator.java | 5 +- .../bucketfunction/TSIDBucketFunction.java | 13 +--- .../bucketfunction/TopkBucketFunction.java | 60 +++++++++++++++++++ .../aggregation/function/TopkFunction.java | 51 ++++++++++++++++ .../aggregation/internal/TimeSeriesTopk.java | 12 ++++ 7 files changed, 150 insertions(+), 15 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TSIDValue.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TopkBucketFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/TopkFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesTopk.java diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java index 0062f22c5566a..41cdab35c09ec 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java @@ -29,6 +29,7 @@ public class Downsample implements ToXContentObject, Writeable { public static final ParseField FUNCTION_FIELD = new ParseField("function"); public static final ParseField PARAMS_FIELD = new ParseField("params"); + @SuppressWarnings("unchecked") public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( NAME, false, diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TSIDValue.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TSIDValue.java new file mode 100644 index 0000000000000..de5073a47e96e --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TSIDValue.java @@ -0,0 +1,23 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation; + +import org.apache.lucene.util.BytesRef; + +public class TSIDValue { + public TSIDValue(BytesRef tsid, Value value, boolean detailed) { + this.tsid = tsid; + this.value = value; + this.detailed = detailed; + } + + public BytesRef tsid; + public Value value; + public boolean detailed; +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index 9959287bec1f7..296c179017507 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -43,7 +43,6 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.AggregatorBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.TSIDBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.function.LastFunction; import java.io.IOException; import java.util.ArrayList; @@ -443,11 +442,11 @@ public void collectTimeSeriesValues(long bucketOrd) throws IOException { } if (timestamp - interval <= timestampBounds.startTime() || timestamp > timestampBounds.endTime()) { aggregatorBucketFunction.collect( - new TSIDBucketFunction.TSIDValue(preTsid, value.getAggregation(format, metadata()), true), + new TSIDValue(preTsid, value.getAggregation(format, metadata()), true), ord ); } else { - aggregatorBucketFunction.collect(new TSIDBucketFunction.TSIDValue(preTsid, value.get(), false), ord); + aggregatorBucketFunction.collect(new TSIDValue(preTsid, value.get(), false), ord); } } } else { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java index 08ce41150cff5..f6df6cdd653c3 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java @@ -11,7 +11,7 @@ import org.apache.lucene.util.BytesRef; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; -import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.TSIDBucketFunction.TSIDValue; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TSIDValue; import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TSIDInternalAggregation; import java.util.HashMap; @@ -73,15 +73,4 @@ public InternalAggregation getAggregation(long bucket, DocValueFormat formatter, } } - public static class TSIDValue { - public TSIDValue(BytesRef tsid, Object value, boolean detailed) { - this.tsid = tsid; - this.value = value; - this.detailed = detailed; - } - - BytesRef tsid; - Object value; - boolean detailed; - } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TopkBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TopkBucketFunction.java new file mode 100644 index 0000000000000..e1581d16b892b --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TopkBucketFunction.java @@ -0,0 +1,60 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.util.PriorityQueue; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TSIDValue; + +public class TopkBucketFunction implements AggregatorBucketFunction>{ + + Map>> values; + private int topkSize; + + public TopkBucketFunction(int size) { + values = new HashMap<>(); + this.topkSize = size; + } + + @Override + public String name() { + return "topk"; + } + + @Override + public void collect(TSIDValue number, long bucket) { + PriorityQueue> queue = values.get(bucket); + if (queue == null) { + queue = new PriorityQueue<>(topkSize) { + @Override + protected boolean lessThan(TSIDValue a, TSIDValue b) { + return a.value < b.value; + } + }; + values.put(bucket, queue); + } + + queue.add(number); + } + + @Override + public InternalAggregation getAggregation(long bucket, + DocValueFormat formatter, Map metadata) { + return null; + } + + @Override + public void close() { + values = null; + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/TopkFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/TopkFunction.java new file mode 100644 index 0000000000000..8d9b048e95395 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/TopkFunction.java @@ -0,0 +1,51 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.function; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.lucene.util.PriorityQueue; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TSIDValue; + +public class TopkFunction implements AggregatorFunction, List>> { + private final PriorityQueue> queue; + + public TopkFunction(int size) { + queue = new PriorityQueue<>(size) { + @Override + protected boolean lessThan(TSIDValue a, TSIDValue b) { + return a.value < b.value; + } + }; + } + + @Override + public void collect(TSIDValue value) { + queue.add(value); + } + + @Override + public List> get() { + List> values = new ArrayList<>(queue.size()); + for (int b = queue.size() - 1; b >= 0; --b) { + values.add(queue.pop()); + } + return values; + } + + @Override + public InternalAggregation getAggregation( + DocValueFormat formatter, Map metadata) { + return null; + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesTopk.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesTopk.java new file mode 100644 index 0000000000000..166f096f672db --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesTopk.java @@ -0,0 +1,12 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.internal; + +public class TimeSeriesTopk { +} From c1b97b57c9d79f52f13d444cb8e43764f26c6013 Mon Sep 17 00:00:00 2001 From: weizijun Date: Mon, 23 May 2022 17:04:53 +0800 Subject: [PATCH 22/53] support topk --- .../TimeSeriesAggregationsIT.java | 34 +++--- .../elasticsearch/search/SearchModule.java | 8 +- .../timeseries/aggregation/Aggregator.java | 69 +++++++++-- .../InternalTimeSeriesAggregation.java | 55 +++------ .../timeseries/aggregation/TSIDValue.java | 4 +- .../TimeSeriesAggregationAggregator.java | 16 ++- .../AggregatorBucketFunction.java | 2 +- .../bucketfunction/AvgBucketFunction.java | 2 +- .../bucketfunction/MaxBucketFunction.java | 2 +- .../bucketfunction/MinBucketFunction.java | 2 +- .../NoAggregatorBucketFunction.java | 6 +- .../bucketfunction/SumBucketFunction.java | 2 +- .../bucketfunction/TSIDBucketFunction.java | 12 +- .../bucketfunction/TopkBucketFunction.java | 42 +++++-- .../ValueCountBucketFunction.java | 2 +- .../aggregation/function/LastFunction.java | 4 +- .../aggregation/function/TopkFunction.java | 28 +++-- .../internal/TSIDInternalAggregation.java | 28 ++++- .../{Last.java => TimeSeriesLast.java} | 20 +-- .../internal/TimeSeriesLineAggreagation.java | 114 +++++++++++++++++ .../aggregation/internal/TimeSeriesTopk.java | 115 +++++++++++++++++- .../InternalTimeSeriesAggregationTests.java | 3 +- 22 files changed, 430 insertions(+), 140 deletions(-) rename server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/{Last.java => TimeSeriesLast.java} (79%) create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesLineAggreagation.java diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java index c88d0e46c6c76..1e6ee4b3565a7 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java @@ -35,6 +35,7 @@ import org.elasticsearch.search.aggregations.timeseries.TimeSeries; import org.elasticsearch.search.aggregations.timeseries.aggregation.Function; import org.elasticsearch.search.aggregations.timeseries.aggregation.InternalTimeSeriesAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesLineAggreagation; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.xcontent.XContentBuilder; @@ -534,11 +535,7 @@ public void testBasicTimeSeriesAggregations() { Rounding.Prepared rounding = Rounding.builder(new TimeValue(fixedInterval.estimateMillis())).build().prepareForUnknown(); SearchResponse response = client().prepareSearch("index") .setSize(0) - .addAggregation( - timeSeriesAggregation("by_ts").field("metric_0") - .interval(fixedInterval) - .size(data.size()) - ) + .addAggregation(timeSeriesAggregation("by_ts").field("metric_0").interval(fixedInterval).size(data.size())) .get(); Aggregations aggregations = response.getAggregations(); assertNotNull(aggregations); @@ -564,8 +561,9 @@ public void testBasicTimeSeriesAggregations() { }); dataBucketValues.forEach((timestamp, value) -> { - assertTrue(bucket.getTimeBucketValues().containsKey(timestamp)); - InternalAggregation aggregation = bucket.getTimeBucketValues().get(timestamp); + assertTrue(((TimeSeriesLineAggreagation) bucket.getMetricAggregation()).getTimeBucketValues().containsKey(timestamp)); + InternalAggregation aggregation = ((TimeSeriesLineAggreagation) bucket.getMetricAggregation()).getTimeBucketValues() + .get(timestamp); assertTrue(aggregation instanceof InternalNumericMetricsAggregation.SingleValue); assertThat(((InternalNumericMetricsAggregation.SingleValue) aggregation).value(), closeTo(value.v1(), 0.0001d)); }); @@ -606,8 +604,9 @@ public void testTimeSeriesAggregationsDownsample() { }); dataBucketValues.forEach((timestamp, value) -> { - assertTrue(bucket.getTimeBucketValues().containsKey(timestamp)); - InternalAggregation aggregation = bucket.getTimeBucketValues().get(timestamp); + assertTrue(((TimeSeriesLineAggreagation) bucket.getMetricAggregation()).getTimeBucketValues().containsKey(timestamp)); + InternalAggregation aggregation = ((TimeSeriesLineAggreagation) bucket.getMetricAggregation()).getTimeBucketValues() + .get(timestamp); assertTrue(aggregation instanceof InternalNumericMetricsAggregation.SingleValue); assertThat(((InternalNumericMetricsAggregation.SingleValue) aggregation).value(), closeTo(value, 0.0001d)); }); @@ -619,12 +618,7 @@ public void testTimeSeriesAggregationsAggregator() { Rounding.Prepared rounding = Rounding.builder(new TimeValue(fixedInterval.estimateMillis())).build().prepareForUnknown(); SearchResponse response = client().prepareSearch("index") .setSize(0) - .addAggregation( - timeSeriesAggregation("by_ts").field("metric_0") - .interval(fixedInterval) - .aggregator("sum") - .size(data.size()) - ) + .addAggregation(timeSeriesAggregation("by_ts").field("metric_0").interval(fixedInterval).aggregator("sum").size(data.size())) .get(); Aggregations aggregations = response.getAggregations(); assertNotNull(aggregations); @@ -662,8 +656,9 @@ public void testTimeSeriesAggregationsAggregator() { for (InternalTimeSeriesAggregation.InternalBucket bucket : timeSeries.getBuckets()) { aggResults.forEach((timestamp, metric) -> { - assertTrue(bucket.getTimeBucketValues().containsKey(timestamp)); - InternalAggregation aggregation = bucket.getTimeBucketValues().get(timestamp); + assertTrue(((TimeSeriesLineAggreagation) bucket.getMetricAggregation()).getTimeBucketValues().containsKey(timestamp)); + InternalAggregation aggregation = ((TimeSeriesLineAggreagation) bucket.getMetricAggregation()).getTimeBucketValues() + .get(timestamp); assertTrue(aggregation instanceof InternalNumericMetricsAggregation.SingleValue); assertThat(((InternalNumericMetricsAggregation.SingleValue) aggregation).value(), closeTo(metric, 0.0001d)); }); @@ -735,8 +730,9 @@ public void testTimeSeriesAggregationsGroupBy() { Map key = bucket.getKey(); Map dataValues = aggResults.get(key); dataValues.forEach((timestamp, metric) -> { - assertTrue(bucket.getTimeBucketValues().containsKey(timestamp)); - InternalAggregation aggregation = bucket.getTimeBucketValues().get(timestamp); + assertTrue(((TimeSeriesLineAggreagation) bucket.getMetricAggregation()).getTimeBucketValues().containsKey(timestamp)); + InternalAggregation aggregation = ((TimeSeriesLineAggreagation) bucket.getMetricAggregation()).getTimeBucketValues() + .get(timestamp); assertTrue(aggregation instanceof InternalNumericMetricsAggregation.SingleValue); assertThat(((InternalNumericMetricsAggregation.SingleValue) aggregation).value(), closeTo(metric, 0.0001d)); }); diff --git a/server/src/main/java/org/elasticsearch/search/SearchModule.java b/server/src/main/java/org/elasticsearch/search/SearchModule.java index fb25cc05ef632..3264545b462ca 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchModule.java +++ b/server/src/main/java/org/elasticsearch/search/SearchModule.java @@ -217,10 +217,12 @@ import org.elasticsearch.search.aggregations.timeseries.TimeSeriesAggregationBuilder; import org.elasticsearch.search.aggregations.timeseries.aggregation.InternalTimeSeriesAggregation; import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregationAggregationBuilder; -import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.Last; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesLast; import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TSIDInternalAggregation; import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesIRate; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesLineAggreagation; import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesRate; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesTopk; import org.elasticsearch.search.fetch.FetchPhase; import org.elasticsearch.search.fetch.FetchSubPhase; import org.elasticsearch.search.fetch.subphase.ExplainPhase; @@ -677,10 +679,12 @@ private ValuesSourceRegistry registerAggregations(List plugins) { TimeSeriesAggregationAggregationBuilder::new, TimeSeriesAggregationAggregationBuilder.PARSER ).addResultReader(InternalTimeSeriesAggregation::new) - .addResultReader(Last.NAME, Last::new) + .addResultReader(TimeSeriesLast.NAME, TimeSeriesLast::new) .addResultReader(TSIDInternalAggregation.NAME, TSIDInternalAggregation::new) .addResultReader(TimeSeriesRate.NAME, TimeSeriesRate::new) .addResultReader(TimeSeriesIRate.NAME, TimeSeriesIRate::new) + .addResultReader(TimeSeriesLineAggreagation.NAME, TimeSeriesLineAggreagation::new) + .addResultReader(TimeSeriesTopk.NAME, TimeSeriesTopk::new) .setAggregatorRegistrar(TimeSeriesAggregationAggregationBuilder::registerAggregators), builder ); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java index 64406127c77e9..4c7c76a599d52 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java @@ -14,75 +14,118 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.MaxBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.MinBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.SumBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.TopkBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.ValueCountBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AvgFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MaxFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MinFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.SumFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.TopkFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.ValueCountFunction; +import java.util.Map; + public enum Aggregator { count { @Override - public AggregatorFunction getAggregatorFunction() { + public AggregatorFunction getAggregatorFunction(Map aggregatorParams) { return new ValueCountFunction(); } @Override - public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, Map aggregatorParams) { return new ValueCountBucketFunction(bigArrays); } }, sum { @Override - public AggregatorFunction getAggregatorFunction() { + public AggregatorFunction getAggregatorFunction(Map aggregatorParams) { return new SumFunction(); } @Override - public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, Map aggregatorParams) { return new SumBucketFunction(bigArrays); } }, min { @Override - public AggregatorFunction getAggregatorFunction() { + public AggregatorFunction getAggregatorFunction(Map aggregatorParams) { return new MinFunction(); } @Override - public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, Map aggregatorParams) { return new MinBucketFunction(bigArrays); } }, max { @Override - public AggregatorFunction getAggregatorFunction() { + public AggregatorFunction getAggregatorFunction(Map aggregatorParams) { return new MaxFunction(); } @Override - public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, Map aggregatorParams) { return new MaxBucketFunction(bigArrays); } }, avg { @Override - public AggregatorFunction getAggregatorFunction() { + public AggregatorFunction getAggregatorFunction(Map aggregatorParams) { return new AvgFunction(); } @Override - public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays) { + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, Map aggregatorParams) { return new AvgBucketFunction(bigArrays); } + }, + topk { + @Override + public AggregatorFunction getAggregatorFunction(Map aggregatorParams) { + int size = 0; + if (aggregatorParams != null && aggregatorParams.containsKey("size")) { + size = (int) aggregatorParams.get("size"); + } + return new TopkFunction(size, true); + } + + @Override + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, Map aggregatorParams) { + int size = 0; + if (aggregatorParams != null && aggregatorParams.containsKey("size")) { + size = (int) aggregatorParams.get("size"); + } + return new TopkBucketFunction(size, true); + } + }, + bottomk { + @Override + public AggregatorFunction getAggregatorFunction(Map aggregatorParams) { + int size = 0; + if (aggregatorParams != null && aggregatorParams.containsKey("size")) { + size = (int) aggregatorParams.get("size"); + } + return new TopkFunction(size, false); + } + + @Override + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, + Map aggregatorParams) { + int size = 0; + if (aggregatorParams != null && aggregatorParams.containsKey("size")) { + size = (int) aggregatorParams.get("size"); + } + return new TopkBucketFunction(size, false); + } }; public static Aggregator resolve(String name) { try { return Aggregator.valueOf(name); - } catch (Exception e){ + } catch (Exception e) { throw new IllegalArgumentException("aggregator [" + name + "] not support"); } } @@ -90,10 +133,10 @@ public static Aggregator resolve(String name) { /** * get the aggregator function */ - public abstract AggregatorFunction getAggregatorFunction(); + public abstract AggregatorFunction getAggregatorFunction(Map aggregatorParams); /** * get the aggregator bucket function */ - public abstract AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays); + public abstract AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, Map aggregatorParams); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregation.java index d5a1a352b0fac..3e84007a581b7 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregation.java @@ -28,9 +28,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Objects; -import java.util.TreeMap; import static org.elasticsearch.search.aggregations.ParsedMultiBucketAggregation.declareMultiBucketAggregationFields; import static org.elasticsearch.search.aggregations.bucket.terms.InternalTerms.DOC_COUNT_ERROR_UPPER_BOUND_FIELD_NAME; @@ -60,7 +58,7 @@ public static class InternalBucket extends AbstractTermsBucket implements TimeSe protected final boolean keyed; protected final Map key; protected long docCount; - protected Map timeBucketValues; + protected InternalAggregation metricAggregation; protected boolean showDocCountError; protected long docCountError; protected InternalAggregations aggregations; @@ -68,7 +66,7 @@ public static class InternalBucket extends AbstractTermsBucket implements TimeSe public InternalBucket( Map key, long docCount, - Map timeBucketValues, + InternalAggregation metricAggregation, InternalAggregations aggregations, boolean keyed, boolean showDocCountError, @@ -78,7 +76,7 @@ public InternalBucket( this.docCount = docCount; this.aggregations = aggregations; this.keyed = keyed; - this.timeBucketValues = timeBucketValues; + this.metricAggregation = metricAggregation; this.showDocCountError = showDocCountError; this.docCountError = docCountError; } @@ -90,7 +88,7 @@ public InternalBucket(StreamInput in, boolean keyed, boolean showDocCountError) this.keyed = keyed; key = in.readOrderedMap(StreamInput::readString, StreamInput::readGenericValue); docCount = in.readVLong(); - timeBucketValues = in.readOrderedMap(StreamInput::readLong, stream -> stream.readNamedWriteable(InternalAggregation.class)); + metricAggregation = in.readNamedWriteable(InternalAggregation.class); this.showDocCountError = showDocCountError; docCountError = -1; if (showDocCountError) { @@ -103,7 +101,7 @@ public InternalBucket(StreamInput in, boolean keyed, boolean showDocCountError) public void writeTo(StreamOutput out) throws IOException { out.writeMap(key, StreamOutput::writeString, StreamOutput::writeGenericValue); out.writeVLong(docCount); - out.writeMap(timeBucketValues, StreamOutput::writeLong, StreamOutput::writeNamedWriteable); + out.writeNamedWriteable(metricAggregation); if (showDocCountError) { // TODO recover -Dtests.seed=142C4BE4C242FF8B // out.writeLong(docCountError); @@ -126,8 +124,8 @@ public long getDocCount() { return docCount; } - public Map getTimeBucketValues() { - return timeBucketValues; + public InternalAggregation getMetricAggregation() { + return metricAggregation; } @Override @@ -151,11 +149,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(DOC_COUNT_ERROR_UPPER_BOUND_FIELD_NAME.getPreferredName(), docCountError); } builder.startObject(CommonFields.VALUES.getPreferredName()); - for (Entry entry : timeBucketValues.entrySet()) { - builder.startObject(String.valueOf(entry.getKey())); - entry.getValue().doXContentBody(builder, params); - builder.endObject(); - } + metricAggregation.doXContentBody(builder, params); builder.endObject(); aggregations.toXContentInternal(builder, params); builder.endObject(); @@ -174,12 +168,12 @@ public boolean equals(Object other) { return Objects.equals(key, that.key) && Objects.equals(keyed, that.keyed) && Objects.equals(docCount, that.docCount) - && Objects.equals(timeBucketValues, that.timeBucketValues); + && Objects.equals(metricAggregation, that.metricAggregation); } @Override public int hashCode() { - return Objects.hash(getClass(), key, keyed, docCount, timeBucketValues); + return Objects.hash(getClass(), key, keyed, docCount, metricAggregation); } @Override @@ -363,7 +357,7 @@ public InternalBucket createBucket(InternalAggregations aggregations, InternalBu return new InternalBucket( prototype.key, prototype.docCount, - prototype.timeBucketValues, + prototype.metricAggregation, prototype.aggregations, prototype.keyed, prototype.showDocCountError, @@ -376,7 +370,7 @@ protected InternalBucket createBucket(long docCount, InternalAggregations aggs, return new InternalBucket( prototype.key, prototype.docCount, - prototype.timeBucketValues, + prototype.metricAggregation, prototype.aggregations, prototype.keyed, prototype.showDocCountError, @@ -442,8 +436,7 @@ public InternalAggregation reduce(List aggregations, Aggreg public InternalBucket reduceBucket(List buckets, AggregationReduceContext context) { InternalBucket reduced = null; List aggregationsList = new ArrayList<>(buckets.size()); - Map timeBucketResults = new TreeMap<>(); - Map> timeBucketAggregationsList = new TreeMap<>(); + List metricAggregationsList = new ArrayList<>(buckets.size()); long docCountError = 0; for (InternalBucket bucket : buckets) { if (docCountError != -1) { @@ -457,7 +450,7 @@ public InternalBucket reduceBucket(List buckets, AggregationRedu reduced = new InternalBucket( bucket.key, bucket.docCount, - bucket.timeBucketValues, + bucket.metricAggregation, bucket.aggregations, bucket.keyed, bucket.showDocCountError, @@ -466,26 +459,12 @@ public InternalBucket reduceBucket(List buckets, AggregationRedu } else { reduced.docCount += bucket.docCount; } - for (Entry entry : bucket.timeBucketValues.entrySet()) { - Long timestamp = entry.getKey(); - InternalAggregation value = entry.getValue(); - List values = timeBucketAggregationsList.get(timestamp); - if (values == null) { - values = new ArrayList<>(); - timeBucketAggregationsList.put(timestamp, values); - } - values.add(value); - } + + metricAggregationsList.add(bucket.metricAggregation); aggregationsList.add(bucket.aggregations); } - timeBucketAggregationsList.forEach((timestamp, aggs) -> { - if (aggs.size() > 0) { - InternalAggregation first = aggs.get(0); - timeBucketResults.put(timestamp, first.reduce(aggs, context)); - } - }); - reduced.timeBucketValues = timeBucketResults; + reduced.metricAggregation = reduced.metricAggregation.reduce(metricAggregationsList, context); reduced.docCountError = docCountError; if (reduced.docCountError == -1) { reduced.showDocCountError = false; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TSIDValue.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TSIDValue.java index de5073a47e96e..ab79f1260c632 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TSIDValue.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TSIDValue.java @@ -11,13 +11,11 @@ import org.apache.lucene.util.BytesRef; public class TSIDValue { - public TSIDValue(BytesRef tsid, Value value, boolean detailed) { + public TSIDValue(BytesRef tsid, Value value) { this.tsid = tsid; this.value = value; - this.detailed = detailed; } public BytesRef tsid; public Value value; - public boolean detailed; } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index 296c179017507..4b1c942f82698 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -43,6 +43,7 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.AggregatorBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.TSIDBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesLineAggreagation; import java.io.IOException; import java.util.ArrayList; @@ -177,7 +178,7 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I InternalTimeSeriesAggregation.InternalBucket bucket = new InternalTimeSeriesAggregation.InternalBucket( TimeSeriesIdFieldMapper.decodeTsid(spareKey), docCount, - new HashMap<>(), + null, null, keyed, false, @@ -200,13 +201,13 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I while (timeOrdsEnum.next()) { values.put( timeOrdsEnum.value() + offset, - aggregatorBucketFunction.getAggregation(timeOrdsEnum.ord(), format, metadata()) + aggregatorBucketFunction.getAggregation(timeOrdsEnum.ord(), aggregatorParams, format, metadata()) ); } } else { values = groupBucketValues.get(ord); } - bucket.timeBucketValues = values; + bucket.metricAggregation = new TimeSeriesLineAggreagation(TimeSeriesLineAggreagation.NAME, values, format, metadata()); allBucketsPerOrd[ordIdx][b] = bucket; otherDocCounts[ordIdx] -= allBucketsPerOrd[ordIdx][b].getDocCount(); } @@ -418,7 +419,7 @@ public void collectTimeSeriesValues(long bucketOrd) throws IOException { if (needAggregator) { AggregatorBucketFunction aggregatorBucketFunction = aggregatorCollectors.get(bucketOrd); if (aggregatorBucketFunction == null) { - AggregatorBucketFunction internal = aggregator.getAggregatorBucketFunction(bigArrays()); + AggregatorBucketFunction internal = aggregator.getAggregatorBucketFunction(bigArrays(), aggregatorParams); aggregatorBucketFunction = new TSIDBucketFunction(internal); aggregatorCollectors.put(bucketOrd, aggregatorBucketFunction); } @@ -441,12 +442,9 @@ public void collectTimeSeriesValues(long bucketOrd) throws IOException { ord = -1 - ord; } if (timestamp - interval <= timestampBounds.startTime() || timestamp > timestampBounds.endTime()) { - aggregatorBucketFunction.collect( - new TSIDValue(preTsid, value.getAggregation(format, metadata()), true), - ord - ); + aggregatorBucketFunction.collect(new TSIDValue(preTsid, value.getAggregation(format, metadata())), ord); } else { - aggregatorBucketFunction.collect(new TSIDValue(preTsid, value.get(), false), ord); + aggregatorBucketFunction.collect(new TSIDValue(preTsid, value.get()), ord); } } } else { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AggregatorBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AggregatorBucketFunction.java index a8875a4eb2138..a807029857334 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AggregatorBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AggregatorBucketFunction.java @@ -35,7 +35,7 @@ public interface AggregatorBucketFunction { * get the {@link InternalAggregation}, it used to transport cross nodes and reduce result in the coordinate node * @return the {@link InternalAggregation} of the input bucket */ - InternalAggregation getAggregation(long bucket, DocValueFormat formatter, Map metadata); + InternalAggregation getAggregation(long bucket, Map aggregatorParams, DocValueFormat formatter, Map metadata); /** * close the inner big array diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AvgBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AvgBucketFunction.java index 596939f2f5b64..50221d6163891 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AvgBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AvgBucketFunction.java @@ -52,7 +52,7 @@ public void collect(Double number, long bucket) { } @Override - public InternalAggregation getAggregation(long bucket, DocValueFormat formatter, Map metadata) { + public InternalAggregation getAggregation(long bucket, Map aggregatorParams, DocValueFormat formatter, Map metadata) { return new org.elasticsearch.search.aggregations.metrics.InternalAvg( name(), sums.get(bucket), diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MaxBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MaxBucketFunction.java index ca30d765eddfd..caa03a0d4bf19 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MaxBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MaxBucketFunction.java @@ -45,7 +45,7 @@ public void collect(Double number, long bucket) { } @Override - public InternalAggregation getAggregation(long bucket, DocValueFormat formatter, Map metadata) { + public InternalAggregation getAggregation(long bucket, Map aggregatorParams, DocValueFormat formatter, Map metadata) { return new org.elasticsearch.search.aggregations.metrics.Max(name(), value.get(bucket), formatter, metadata); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MinBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MinBucketFunction.java index e6288619fb36a..32464dc3e5ed7 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MinBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MinBucketFunction.java @@ -45,7 +45,7 @@ public void collect(Double number, long bucket) { } @Override - public InternalAggregation getAggregation(long bucket, DocValueFormat formatter, Map metadata) { + public InternalAggregation getAggregation(long bucket, Map aggregatorParams, DocValueFormat formatter, Map metadata) { return new org.elasticsearch.search.aggregations.metrics.Min(name(), value.get(bucket), formatter, metadata); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/NoAggregatorBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/NoAggregatorBucketFunction.java index dece2d0afe8b6..58b6152599055 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/NoAggregatorBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/NoAggregatorBucketFunction.java @@ -10,7 +10,7 @@ import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; -import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.Last; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesLast; import java.util.Map; @@ -28,8 +28,8 @@ public void collect(Double number, long bucket) { } @Override - public InternalAggregation getAggregation(long bucket, DocValueFormat formatter, Map metadata) { - return new Last(name(), value, 0, formatter, metadata); + public InternalAggregation getAggregation(long bucket, Map aggregatorParams, DocValueFormat formatter, Map metadata) { + return new TimeSeriesLast(name(), value, 0, formatter, metadata); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/SumBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/SumBucketFunction.java index 49b62201cf9d3..3512294cc921d 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/SumBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/SumBucketFunction.java @@ -45,7 +45,7 @@ public void collect(Double number, long bucket) { } @Override - public InternalAggregation getAggregation(long bucket, DocValueFormat formatter, Map metadata) { + public InternalAggregation getAggregation(long bucket, Map aggregatorParams, DocValueFormat formatter, Map metadata) { return new org.elasticsearch.search.aggregations.metrics.Sum(name(), value.get(bucket), formatter, metadata); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java index f6df6cdd653c3..e5fa3cd8bbbf1 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java @@ -42,18 +42,20 @@ public TSIDBucketFunction(AggregatorBucketFunction aggregatorBucketFunction) { @Override public String name() { - return "tsid"; + return TSIDInternalAggregation.NAME; } @Override public void collect(TSIDValue tsidValue, long bucket) { - if (tsidValue.detailed) { + if (tsidValue.value instanceof InternalAggregation) { Map tsidValues = values.get(bucket); if (tsidValues == null) { tsidValues = new HashMap<>(); values.put(bucket, tsidValues); } tsidValues.put(tsidValue.tsid, (InternalAggregation) tsidValue.value); + } else if (aggregatorBucketFunction instanceof TopkBucketFunction) { + aggregatorBucketFunction.collect(tsidValue, bucket); } else { aggregatorBucketFunction.collect(tsidValue.value, bucket); } @@ -65,11 +67,11 @@ public void close() { } @Override - public InternalAggregation getAggregation(long bucket, DocValueFormat formatter, Map metadata) { + public InternalAggregation getAggregation(long bucket, Map aggregatorParams, DocValueFormat formatter, Map metadata) { if (values.containsKey(bucket)) { - return new TSIDInternalAggregation(name(), values.get(bucket), aggregatorBucketFunction.name(), formatter, metadata); + return new TSIDInternalAggregation(name(), values.get(bucket), aggregatorBucketFunction.name(), aggregatorParams, formatter, metadata); } else { - return aggregatorBucketFunction.getAggregation(bucket, formatter, metadata); + return aggregatorBucketFunction.getAggregation(bucket, aggregatorParams, formatter, metadata); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TopkBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TopkBucketFunction.java index e1581d16b892b..53272f77ed947 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TopkBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TopkBucketFunction.java @@ -8,22 +8,27 @@ package org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction; -import java.util.HashMap; -import java.util.Map; - import org.apache.lucene.util.PriorityQueue; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.timeseries.aggregation.TSIDValue; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesTopk; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; -public class TopkBucketFunction implements AggregatorBucketFunction>{ +public class TopkBucketFunction implements AggregatorBucketFunction> { - Map>> values; - private int topkSize; + private Map>> values; + private final int topkSize; + private final boolean isTop; - public TopkBucketFunction(int size) { + public TopkBucketFunction(int size, boolean isTop) { values = new HashMap<>(); this.topkSize = size; + this.isTop = isTop; } @Override @@ -38,19 +43,32 @@ public void collect(TSIDValue number, long bucket) { queue = new PriorityQueue<>(topkSize) { @Override protected boolean lessThan(TSIDValue a, TSIDValue b) { - return a.value < b.value; + if (isTop) { + return a.value > b.value; + } else { + return a.value < b.value; + } } }; values.put(bucket, queue); } - queue.add(number); + queue.insertWithOverflow(number); } @Override - public InternalAggregation getAggregation(long bucket, - DocValueFormat formatter, Map metadata) { - return null; + public InternalAggregation getAggregation( + long bucket, + Map aggregatorParams, + DocValueFormat formatter, + Map metadata + ) { + PriorityQueue> queue = values.get(bucket); + List> values = new ArrayList<>(queue.size()); + for (int b = queue.size() - 1; b >= 0; --b) { + values.add(queue.pop()); + } + return new TimeSeriesTopk(name(), values, topkSize, isTop, formatter, metadata); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/ValueCountBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/ValueCountBucketFunction.java index 6fbbb2bb8aa69..6016efda38d39 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/ValueCountBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/ValueCountBucketFunction.java @@ -45,7 +45,7 @@ public void collect(Double number, long bucket) { } @Override - public InternalAggregation getAggregation(long bucket, DocValueFormat formatter, Map metadata) { + public InternalAggregation getAggregation(long bucket, Map aggregatorParams, DocValueFormat formatter, Map metadata) { return new InternalValueCount(name(), counts.get(bucket), metadata); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java index 415f84b6302cd..ceee5402112d3 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java @@ -11,7 +11,7 @@ import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; -import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.Last; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesLast; import java.util.Map; @@ -32,6 +32,6 @@ public Double get() { @Override public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { - return new Last(Last.NAME, point.getValue(), point.getTimestamp(), formatter, metadata); + return new TimeSeriesLast(TimeSeriesLast.NAME, point.getValue(), point.getTimestamp(), formatter, metadata); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/TopkFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/TopkFunction.java index 8d9b048e95395..1e7e83bc4e5d7 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/TopkFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/TopkFunction.java @@ -8,30 +8,39 @@ package org.elasticsearch.search.aggregations.timeseries.aggregation.function; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - import org.apache.lucene.util.PriorityQueue; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.timeseries.aggregation.TSIDValue; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesTopk; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; public class TopkFunction implements AggregatorFunction, List>> { private final PriorityQueue> queue; + private final int topkSize; + private final boolean isTop; - public TopkFunction(int size) { + public TopkFunction(int size, boolean isTop) { queue = new PriorityQueue<>(size) { @Override protected boolean lessThan(TSIDValue a, TSIDValue b) { - return a.value < b.value; + if (isTop) { + return a.value > b.value; + } else { + return a.value < b.value; + } } }; + this.isTop = isTop; + this.topkSize = size; } @Override public void collect(TSIDValue value) { - queue.add(value); + queue.insertWithOverflow(value); } @Override @@ -44,8 +53,7 @@ public List> get() { } @Override - public InternalAggregation getAggregation( - DocValueFormat formatter, Map metadata) { - return null; + public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { + return new TimeSeriesTopk(TimeSeriesTopk.NAME, get(), topkSize, isTop, formatter, metadata); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java index 6dc29e47d350e..474a47366a3c9 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java @@ -17,9 +17,11 @@ import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation.SingleValue; import org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TSIDValue; import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.TSIDBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.TopkFunction; import org.elasticsearch.xcontent.XContentBuilder; import java.io.IOException; @@ -34,16 +36,18 @@ */ @SuppressWarnings({ "unchecked", "rawtypes" }) public class TSIDInternalAggregation extends InternalAggregation { - public static final String NAME = "tsid"; + public static final String NAME = "time_series_tsid"; private final Map values; private final String aggregator; + private final Map aggregatorParams; private final DocValueFormat formatter; public TSIDInternalAggregation( String name, Map values, String aggregator, + Map aggregatorParams, DocValueFormat formatter, Map metadata ) { @@ -51,12 +55,14 @@ public TSIDInternalAggregation( this.values = values; this.formatter = formatter; this.aggregator = aggregator; + this.aggregatorParams = aggregatorParams; } public TSIDInternalAggregation(StreamInput in) throws IOException { super(in); formatter = in.readNamedWriteable(DocValueFormat.class); aggregator = in.readString(); + aggregatorParams = in.readMap(); values = in.readOrderedMap(StreamInput::readBytesRef, stream -> stream.readNamedWriteable(InternalAggregation.class)); } @@ -69,6 +75,7 @@ public String getWriteableName() { protected void doWriteTo(StreamOutput out) throws IOException { out.writeNamedWriteable(formatter); out.writeString(aggregator); + out.writeGenericMap(aggregatorParams); out.writeMap(values, StreamOutput::writeBytesRef, StreamOutput::writeNamedWriteable); } @@ -78,10 +85,15 @@ public InternalAggregation reduce(List aggregations, Aggreg TSIDInternalAggregation tsidAgg = (TSIDInternalAggregation) aggregations.get(0); if (reduceContext.isFinalReduce()) { Aggregator function = Aggregator.valueOf(aggregator); - final AggregatorFunction aggregatorFunction = function.getAggregatorFunction(); + final AggregatorFunction aggregatorFunction = function.getAggregatorFunction(aggregatorParams); tsidAgg.values.forEach( (tsid, agg) -> { - aggregatorFunction.collect(new TimePoint(0, ((InternalNumericMetricsAggregation.SingleValue) agg).value())); + if (aggregatorFunction instanceof TopkFunction) { + aggregatorFunction.collect(new TSIDValue<>(tsid, ((InternalNumericMetricsAggregation.SingleValue) agg).value())); + } else { + aggregatorFunction.collect(new TimePoint(0, ((InternalNumericMetricsAggregation.SingleValue) agg).value())); + } + } ); return aggregatorFunction.getAggregation(formatter, getMetadata()); @@ -105,12 +117,16 @@ public InternalAggregation reduce(List aggregations, Aggreg if (reduceContext.isFinalReduce()) { Aggregator function = Aggregator.valueOf(aggregator); - final AggregatorFunction aggregatorFunction = function.getAggregatorFunction(); + final AggregatorFunction aggregatorFunction = function.getAggregatorFunction(aggregatorParams); reduced.forEach((tsid, aggs) -> { if (aggs.size() > 0) { InternalAggregation first = aggs.get(0); InternalNumericMetricsAggregation.SingleValue internalAggregation = (SingleValue) first.reduce(aggs, reduceContext); - aggregatorFunction.collect(new TimePoint(0, internalAggregation.value())); + if (aggregatorFunction instanceof TopkFunction) { + aggregatorFunction.collect(new TSIDValue<>(tsid, internalAggregation.value())); + } else { + aggregatorFunction.collect(new TimePoint(0, internalAggregation.value())); + } } }); return aggregatorFunction.getAggregation(formatter, getMetadata()); @@ -122,7 +138,7 @@ public InternalAggregation reduce(List aggregations, Aggreg finalReduces.put(tsid, first.reduce(aggs, reduceContext)); } }); - return new TSIDInternalAggregation(name, finalReduces, aggregator, formatter, getMetadata()); + return new TSIDInternalAggregation(name, finalReduces, aggregator, aggregatorParams, formatter, getMetadata()); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesLast.java similarity index 79% rename from server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java rename to server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesLast.java index 7c082ffa98553..2872784be9d9a 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/Last.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesLast.java @@ -22,13 +22,13 @@ import java.util.Map; import java.util.Objects; -public class Last extends InternalNumericMetricsAggregation.SingleValue { - public static final String NAME = "last"; +public class TimeSeriesLast extends InternalNumericMetricsAggregation.SingleValue { + public static final String NAME = "time_series_last"; private final double last; private final long timestamp; - public Last(String name, double last, long timestamp, DocValueFormat formatter, Map metadata) { + public TimeSeriesLast(String name, double last, long timestamp, DocValueFormat formatter, Map metadata) { super(name, formatter, metadata); this.last = last; this.timestamp = timestamp; @@ -37,7 +37,7 @@ public Last(String name, double last, long timestamp, DocValueFormat formatter, /** * Read from a stream. */ - public Last(StreamInput in) throws IOException { + public TimeSeriesLast(StreamInput in) throws IOException { super(in); last = in.readDouble(); timestamp = in.readLong(); @@ -69,16 +69,16 @@ public long getTimestamp() { } @Override - public Last reduce(List aggregations, AggregationReduceContext reduceContext) { + public TimeSeriesLast reduce(List aggregations, AggregationReduceContext reduceContext) { double last = Double.NEGATIVE_INFINITY; long timestamp = Long.MIN_VALUE; for (InternalAggregation aggregation : aggregations) { - if (((Last) aggregation).timestamp > timestamp) { - last = ((Last) aggregation).last; - timestamp = ((Last) aggregation).timestamp; + if (((TimeSeriesLast) aggregation).timestamp > timestamp) { + last = ((TimeSeriesLast) aggregation).last; + timestamp = ((TimeSeriesLast) aggregation).timestamp; } } - return new Last(name, last, timestamp, format, getMetadata()); + return new TimeSeriesLast(name, last, timestamp, format, getMetadata()); } @Override @@ -102,7 +102,7 @@ public boolean equals(Object o) { if (false == super.equals(o)) { return false; } - Last last1 = (Last) o; + TimeSeriesLast last1 = (TimeSeriesLast) o; return Double.compare(last1.last, last) == 0 && timestamp == last1.timestamp; } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesLineAggreagation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesLineAggreagation.java new file mode 100644 index 0000000000000..2524f3c69b1bf --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesLineAggreagation.java @@ -0,0 +1,114 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.internal; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.AggregationReduceContext; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.TreeMap; + +/** + * @author weizijun.wzj + * @date 2022/5/23 + */ +public class TimeSeriesLineAggreagation extends InternalAggregation { + public static final String NAME = "time_series_line"; + + private final Map timeBucketValues; + private final DocValueFormat formatter; + + public TimeSeriesLineAggreagation( + String name, + Map timeBucketValues, + DocValueFormat formatter, + Map metadata + ) { + super(name, metadata); + this.timeBucketValues = timeBucketValues; + this.formatter = formatter; + } + + public TimeSeriesLineAggreagation(StreamInput in) throws IOException { + super(in); + formatter = in.readNamedWriteable(DocValueFormat.class); + timeBucketValues = in.readOrderedMap(StreamInput::readLong, stream -> stream.readNamedWriteable(InternalAggregation.class)); + } + + public Map getTimeBucketValues() { + return timeBucketValues; + } + + @Override + public String getWriteableName() { + return NAME; + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + out.writeNamedWriteable(formatter); + out.writeMap(timeBucketValues, StreamOutput::writeLong, StreamOutput::writeNamedWriteable); + } + + @Override + public InternalAggregation reduce(List aggregations, AggregationReduceContext reduceContext) { + Map> timeBucketAggregationsList = new TreeMap<>(); + Map timeBucketResults = new TreeMap<>(); + for (InternalAggregation internalAggregation : aggregations) { + TimeSeriesLineAggreagation timeSeriesLineAggreagation = (TimeSeriesLineAggreagation) internalAggregation; + for (Entry entry : timeSeriesLineAggreagation.timeBucketValues.entrySet()) { + Long timestamp = entry.getKey(); + InternalAggregation value = entry.getValue(); + List values = timeBucketAggregationsList.get(timestamp); + if (values == null) { + values = new ArrayList<>(); + timeBucketAggregationsList.put(timestamp, values); + } + values.add(value); + } + } + + timeBucketAggregationsList.forEach((timestamp, aggs) -> { + if (aggs.size() > 0) { + InternalAggregation first = aggs.get(0); + timeBucketResults.put(timestamp, first.reduce(aggs, reduceContext)); + } + }); + + return new TimeSeriesLineAggreagation(name, timeBucketResults, formatter, getMetadata()); + } + + @Override + protected boolean mustReduceOnSingleInternalAgg() { + return false; + } + + @Override + public Object getProperty(List path) { + return null; + } + + @Override + public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + for (Entry entry : timeBucketValues.entrySet()) { + builder.startObject(String.valueOf(entry.getKey())); + entry.getValue().doXContentBody(builder, params); + builder.endObject(); + } + return builder; + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesTopk.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesTopk.java index 166f096f672db..c95ec07827b04 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesTopk.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesTopk.java @@ -8,5 +8,118 @@ package org.elasticsearch.search.aggregations.timeseries.aggregation.internal; -public class TimeSeriesTopk { +import org.apache.lucene.util.PriorityQueue; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.index.mapper.TimeSeriesIdFieldMapper; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.AggregationReduceContext; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TSIDValue; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class TimeSeriesTopk extends InternalAggregation { + public static final String NAME = "time_series_topk"; + + private final List> tsidValues; + private final int topkSize; + private final boolean isTop; + private final DocValueFormat formatter; + + public TimeSeriesTopk( + String name, + List> tsidValues, + int topkSize, + boolean isTop, + DocValueFormat formatter, + Map metadata + ) { + super(name, metadata); + this.tsidValues = tsidValues; + this.topkSize = topkSize; + this.isTop = isTop; + this.formatter = formatter; + } + + public TimeSeriesTopk(StreamInput in) throws IOException { + super(in); + formatter = in.readNamedWriteable(DocValueFormat.class); + tsidValues = in.readList((input -> new TSIDValue<>(in.readBytesRef(), in.readDouble()))); + topkSize = in.readInt(); + isTop = in.readBoolean(); + } + + public List> getTsidValues() { + return tsidValues; + } + + @Override + public String getWriteableName() { + return NAME; + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + out.writeNamedWriteable(formatter); + out.writeCollection(tsidValues, (output, value) -> { + out.writeBytesRef(value.tsid); + out.writeDouble(value.value); + }); + out.writeInt(topkSize); + out.writeBoolean(isTop); + } + + @Override + public InternalAggregation reduce(List aggregations, AggregationReduceContext reduceContext) { + PriorityQueue> queue = new PriorityQueue<>(topkSize) { + @Override + protected boolean lessThan(TSIDValue a, TSIDValue b) { + if (isTop) { + return a.value > b.value; + } else { + return a.value < b.value; + } + } + }; + + for (InternalAggregation internalAggregation : aggregations) { + TimeSeriesTopk timeSeriesTopk = (TimeSeriesTopk) internalAggregation; + timeSeriesTopk.tsidValues.forEach((value -> queue.insertWithOverflow(value))); + } + + List> values = new ArrayList<>(queue.size()); + for (int b = queue.size() - 1; b >= 0; --b) { + values.add(queue.pop()); + } + return new TimeSeriesTopk(name, values, topkSize, isTop, formatter, getMetadata()); + } + + @Override + protected boolean mustReduceOnSingleInternalAgg() { + return false; + } + + @Override + public Object getProperty(List path) { + return null; + } + + @Override + public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + builder.field(CommonFields.VALUE.getPreferredName()); + builder.startArray(); + for (TSIDValue value : tsidValues) { + builder.startObject(); + builder.field("_tsid", TimeSeriesIdFieldMapper.decodeTsid(value.tsid)); + builder.field(CommonFields.VALUE.getPreferredName(), value.value); + builder.endObject(); + } + builder.endArray(); + return builder; + } } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregationTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregationTests.java index f86a9c87bd0c5..196e06cc856bb 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregationTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregationTests.java @@ -11,6 +11,7 @@ import org.elasticsearch.search.aggregations.BucketOrder; import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.ParsedMultiBucketAggregation; +import org.elasticsearch.search.aggregations.metrics.Max; import org.elasticsearch.search.aggregations.timeseries.aggregation.InternalTimeSeriesAggregation.InternalBucket; import org.elasticsearch.test.InternalMultiBucketAggregationTestCase; @@ -84,7 +85,7 @@ private List randomBuckets( new InternalTimeSeriesAggregation.InternalBucket( keys.get(j), docCount, - new HashMap<>(), + null, InternalAggregations.EMPTY, keyed, showTermDocCountError, From c8ec76900dc58200c928e839d7cdc9ac0167c8ff Mon Sep 17 00:00:00 2001 From: weizijun Date: Mon, 23 May 2022 17:40:40 +0800 Subject: [PATCH 23/53] support count_values --- .../metrics/InternalCardinality.java | 2 +- .../timeseries/aggregation/Aggregator.java | 14 +++++ .../CountValuesBucketFunction.java | 57 +++++++++++++++++++ .../function/CountValuesFunction.java | 49 ++++++++++++++++ 4 files changed, 121 insertions(+), 1 deletion(-) create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/CountValuesBucketFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/CountValuesFunction.java diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalCardinality.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalCardinality.java index 50cd873dd8947..078204d242922 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalCardinality.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalCardinality.java @@ -23,7 +23,7 @@ public final class InternalCardinality extends InternalNumericMetricsAggregation.SingleValue implements Cardinality { private final AbstractHyperLogLogPlusPlus counts; - InternalCardinality(String name, AbstractHyperLogLogPlusPlus counts, Map metadata) { + public InternalCardinality(String name, AbstractHyperLogLogPlusPlus counts, Map metadata) { super(name, null, metadata); this.counts = counts; } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java index 4c7c76a599d52..33ae4d102972b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java @@ -11,6 +11,7 @@ import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.AggregatorBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.AvgBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.CountValuesBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.MaxBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.MinBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.SumBucketFunction; @@ -18,6 +19,7 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.ValueCountBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AvgFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.CountValuesFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MaxFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MinFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.SumFunction; @@ -120,6 +122,18 @@ public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArra } return new TopkBucketFunction(size, false); } + }, + count_values { + @Override + public AggregatorFunction getAggregatorFunction(Map aggregatorParams) { + return new CountValuesFunction(); + } + + @Override + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, + Map aggregatorParams) { + return new CountValuesBucketFunction(bigArrays); + } }; public static Aggregator resolve(String name) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/CountValuesBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/CountValuesBucketFunction.java new file mode 100644 index 0000000000000..b8f9c86ec473f --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/CountValuesBucketFunction.java @@ -0,0 +1,57 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction; + +import org.apache.lucene.util.hppc.BitMixer; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.AbstractHyperLogLogPlusPlus; +import org.elasticsearch.search.aggregations.metrics.CardinalityAggregationBuilder; +import org.elasticsearch.search.aggregations.metrics.HyperLogLogPlusPlus; +import org.elasticsearch.search.aggregations.metrics.InternalCardinality; + +import java.util.Map; + +public class CountValuesBucketFunction implements AggregatorBucketFunction { + + private HyperLogLogPlusPlus counts; + + public CountValuesBucketFunction(BigArrays bigArrays) { + counts = new HyperLogLogPlusPlus(HyperLogLogPlusPlus.DEFAULT_PRECISION, bigArrays, 1); + } + + @Override + public String name() { + return "count_values"; + } + + @Override + public void collect(Double number, long bucket) { + long value = BitMixer.mix64(java.lang.Double.doubleToLongBits(number)); + counts.collect(bucket, value); + } + + @Override + public InternalAggregation getAggregation( + long bucket, + Map aggregatorParams, + DocValueFormat formatter, + Map metadata + ) { + AbstractHyperLogLogPlusPlus copy = counts.clone(bucket, BigArrays.NON_RECYCLING_INSTANCE); + return new InternalCardinality(CardinalityAggregationBuilder.NAME, copy, metadata); + } + + @Override + public void close() { + Releasables.close(counts); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/CountValuesFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/CountValuesFunction.java new file mode 100644 index 0000000000000..85b63221f634b --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/CountValuesFunction.java @@ -0,0 +1,49 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.function; + +import java.util.Map; + +import org.apache.lucene.util.hppc.BitMixer; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.AbstractHyperLogLogPlusPlus; +import org.elasticsearch.search.aggregations.metrics.CardinalityAggregationBuilder; +import org.elasticsearch.search.aggregations.metrics.HyperLogLogPlusPlus; +import org.elasticsearch.search.aggregations.metrics.InternalCardinality; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; + +public class CountValuesFunction implements AggregatorFunction { + + private HyperLogLogPlusPlus counts; + + public CountValuesFunction() { + counts = new HyperLogLogPlusPlus(HyperLogLogPlusPlus.DEFAULT_PRECISION, BigArrays.NON_RECYCLING_INSTANCE, 1); + } + + @Override + public void collect(TimePoint value) { + long hash = BitMixer.mix64(java.lang.Double.doubleToLongBits(value.getValue())); + counts.collect(0, hash); + } + + @Override + public Long get() { + AbstractHyperLogLogPlusPlus copy = counts.clone(0, BigArrays.NON_RECYCLING_INSTANCE); + return copy.cardinality(0); + } + + @Override + public InternalAggregation getAggregation( + DocValueFormat formatter, Map metadata) { + AbstractHyperLogLogPlusPlus copy = counts.clone(0, BigArrays.NON_RECYCLING_INSTANCE); + return new InternalCardinality(CardinalityAggregationBuilder.NAME, copy, metadata); + } +} From d7449babc3dff82310c20574444d9648ad987e83 Mon Sep 17 00:00:00 2001 From: weizijun Date: Mon, 23 May 2022 18:05:41 +0800 Subject: [PATCH 24/53] support quantile --- .../TimeSeriesAggregationsIT.java | 8 ++- .../timeseries/aggregation/Aggregator.java | 27 ++++++-- .../QuantileBucketFunction.java | 64 +++++++++++++++++++ .../function/QuantileFunction.java | 44 +++++++++++++ 4 files changed, 138 insertions(+), 5 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/QuantileBucketFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/QuantileFunction.java diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java index 1e6ee4b3565a7..e68bbe883a5d9 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java @@ -618,7 +618,13 @@ public void testTimeSeriesAggregationsAggregator() { Rounding.Prepared rounding = Rounding.builder(new TimeValue(fixedInterval.estimateMillis())).build().prepareForUnknown(); SearchResponse response = client().prepareSearch("index") .setSize(0) - .addAggregation(timeSeriesAggregation("by_ts").field("metric_0").interval(fixedInterval).aggregator("sum").size(data.size())) + .addAggregation( + timeSeriesAggregation("by_ts").field("metric_0") + .interval(fixedInterval) + .aggregator("quantile") + .aggregatorParams(Map.of("quantile", 0.5)) + .size(data.size()) + ) .get(); Aggregations aggregations = response.getAggregations(); assertNotNull(aggregations); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java index 33ae4d102972b..b9b31f3a8527e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java @@ -14,6 +14,7 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.CountValuesBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.MaxBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.MinBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.QuantileBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.SumBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.TopkBucketFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.ValueCountBucketFunction; @@ -22,6 +23,7 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.function.CountValuesFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MaxFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MinFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.QuantileFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.SumFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.TopkFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.ValueCountFunction; @@ -114,8 +116,7 @@ public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArra } @Override - public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, - Map aggregatorParams) { + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, Map aggregatorParams) { int size = 0; if (aggregatorParams != null && aggregatorParams.containsKey("size")) { size = (int) aggregatorParams.get("size"); @@ -130,10 +131,28 @@ public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArra } @Override - public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, - Map aggregatorParams) { + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, Map aggregatorParams) { return new CountValuesBucketFunction(bigArrays); } + }, + quantile { + @Override + public AggregatorFunction getAggregatorFunction(Map aggregatorParams) { + double quantile = 1f; + if (aggregatorParams != null && aggregatorParams.containsKey("quantile")) { + quantile = (double) aggregatorParams.get("quantile"); + } + return new QuantileFunction(quantile); + } + + @Override + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, Map aggregatorParams) { + double quantile = 1f; + if (aggregatorParams != null && aggregatorParams.containsKey("quantile")) { + quantile = (double) aggregatorParams.get("quantile"); + } + return new QuantileBucketFunction(bigArrays, quantile); + } }; public static Aggregator resolve(String name) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/QuantileBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/QuantileBucketFunction.java new file mode 100644 index 0000000000000..80e3b307b97ca --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/QuantileBucketFunction.java @@ -0,0 +1,64 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.ObjectArray; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.InternalTDigestPercentiles; +import org.elasticsearch.search.aggregations.metrics.TDigestState; + +import java.util.Map; + +public class QuantileBucketFunction implements AggregatorBucketFunction { + + private final BigArrays bigArrays; + private ObjectArray states; + private double[] keys = new double[1]; + private final double compression = 100.0; + + public QuantileBucketFunction(BigArrays bigArrays, double quantile) { + this.bigArrays = bigArrays; + states = bigArrays.newObjectArray(1); + keys[0] = quantile; + } + + @Override + public String name() { + return "quantile"; + } + + @Override + public void collect(Double number, long bucket) { + states = bigArrays.grow(states, bucket + 1); + TDigestState state = states.get(bucket); + if (state == null) { + state = new TDigestState(compression); + states.set(bucket, state); + } + state.add(number); + } + + @Override + public InternalAggregation getAggregation( + long bucket, + Map aggregatorParams, + DocValueFormat formatter, + Map metadata + ) { + return new InternalTDigestPercentiles(InternalTDigestPercentiles.NAME, keys, states.get(bucket), false, formatter, metadata); + } + + @Override + public void close() { + Releasables.close(states); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/QuantileFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/QuantileFunction.java new file mode 100644 index 0000000000000..4e0f19bd90ba1 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/QuantileFunction.java @@ -0,0 +1,44 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.function; + +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.InternalTDigestPercentiles; +import org.elasticsearch.search.aggregations.metrics.TDigestState; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; + +import java.util.Map; + +public class QuantileFunction implements AggregatorFunction { + + private TDigestState state; + private double[] keys = new double[1]; + private final double compression = 100.0; + + public QuantileFunction(double quantile) { + state = new TDigestState(compression); + keys[0] = quantile; + } + + @Override + public void collect(TimePoint value) { + state.add(value.getValue()); + } + + @Override + public Double get() { + return state.quantile(keys[0]); + } + + @Override + public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { + return new InternalTDigestPercentiles(InternalTDigestPercentiles.NAME, keys, state, false, formatter, metadata); + } +} From ca23fa4190a63e67a323ce03e6b7da4d85b30328 Mon Sep 17 00:00:00 2001 From: weizijun Date: Mon, 23 May 2022 19:28:17 +0800 Subject: [PATCH 25/53] support count_values --- .../TimeSeriesAggregationsIT.java | 8 +- .../elasticsearch/search/SearchModule.java | 2 + .../CountValuesBucketFunction.java | 36 ++++--- .../function/CountValuesFunction.java | 39 ++++---- .../internal/TimeSeriesCountValues.java | 96 +++++++++++++++++++ 5 files changed, 142 insertions(+), 39 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesCountValues.java diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java index e68bbe883a5d9..1e6ee4b3565a7 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java @@ -618,13 +618,7 @@ public void testTimeSeriesAggregationsAggregator() { Rounding.Prepared rounding = Rounding.builder(new TimeValue(fixedInterval.estimateMillis())).build().prepareForUnknown(); SearchResponse response = client().prepareSearch("index") .setSize(0) - .addAggregation( - timeSeriesAggregation("by_ts").field("metric_0") - .interval(fixedInterval) - .aggregator("quantile") - .aggregatorParams(Map.of("quantile", 0.5)) - .size(data.size()) - ) + .addAggregation(timeSeriesAggregation("by_ts").field("metric_0").interval(fixedInterval).aggregator("sum").size(data.size())) .get(); Aggregations aggregations = response.getAggregations(); assertNotNull(aggregations); diff --git a/server/src/main/java/org/elasticsearch/search/SearchModule.java b/server/src/main/java/org/elasticsearch/search/SearchModule.java index 3264545b462ca..237cd10c5310d 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchModule.java +++ b/server/src/main/java/org/elasticsearch/search/SearchModule.java @@ -217,6 +217,7 @@ import org.elasticsearch.search.aggregations.timeseries.TimeSeriesAggregationBuilder; import org.elasticsearch.search.aggregations.timeseries.aggregation.InternalTimeSeriesAggregation; import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregationAggregationBuilder; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesCountValues; import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesLast; import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TSIDInternalAggregation; import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesIRate; @@ -685,6 +686,7 @@ private ValuesSourceRegistry registerAggregations(List plugins) { .addResultReader(TimeSeriesIRate.NAME, TimeSeriesIRate::new) .addResultReader(TimeSeriesLineAggreagation.NAME, TimeSeriesLineAggreagation::new) .addResultReader(TimeSeriesTopk.NAME, TimeSeriesTopk::new) + .addResultReader(TimeSeriesCountValues.NAME, TimeSeriesCountValues::new) .setAggregatorRegistrar(TimeSeriesAggregationAggregationBuilder::registerAggregators), builder ); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/CountValuesBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/CountValuesBucketFunction.java index b8f9c86ec473f..36893ab839507 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/CountValuesBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/CountValuesBucketFunction.java @@ -8,24 +8,25 @@ package org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction; -import org.apache.lucene.util.hppc.BitMixer; import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.ObjectArray; import org.elasticsearch.core.Releasables; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; -import org.elasticsearch.search.aggregations.metrics.AbstractHyperLogLogPlusPlus; -import org.elasticsearch.search.aggregations.metrics.CardinalityAggregationBuilder; -import org.elasticsearch.search.aggregations.metrics.HyperLogLogPlusPlus; -import org.elasticsearch.search.aggregations.metrics.InternalCardinality; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesCountValues; +import java.util.HashMap; import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; public class CountValuesBucketFunction implements AggregatorBucketFunction { - private HyperLogLogPlusPlus counts; + private ObjectArray> values; // TODO change hashmap + private BigArrays bigArrays; public CountValuesBucketFunction(BigArrays bigArrays) { - counts = new HyperLogLogPlusPlus(HyperLogLogPlusPlus.DEFAULT_PRECISION, bigArrays, 1); + this.bigArrays = bigArrays; + this.values = bigArrays.newObjectArray(1); } @Override @@ -35,8 +36,20 @@ public String name() { @Override public void collect(Double number, long bucket) { - long value = BitMixer.mix64(java.lang.Double.doubleToLongBits(number)); - counts.collect(bucket, value); + values = bigArrays.grow(values, bucket + 1); + long value = java.lang.Double.doubleToLongBits(number); + Map valueCount = values.get(bucket); + if (valueCount == null) { + valueCount = new HashMap<>(); + values.set(bucket, valueCount); + } + + AtomicInteger count = valueCount.get(value); + if (count == null) { + count = new AtomicInteger(0); + valueCount.put(value, count); + } + count.incrementAndGet(); } @Override @@ -46,12 +59,11 @@ public InternalAggregation getAggregation( DocValueFormat formatter, Map metadata ) { - AbstractHyperLogLogPlusPlus copy = counts.clone(bucket, BigArrays.NON_RECYCLING_INSTANCE); - return new InternalCardinality(CardinalityAggregationBuilder.NAME, copy, metadata); + return new TimeSeriesCountValues(TimeSeriesCountValues.NAME, values.get(bucket), formatter, metadata); } @Override public void close() { - Releasables.close(counts); + Releasables.close(values); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/CountValuesFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/CountValuesFunction.java index 85b63221f634b..9e83c6e904146 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/CountValuesFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/CountValuesFunction.java @@ -8,42 +8,41 @@ package org.elasticsearch.search.aggregations.timeseries.aggregation.function; -import java.util.Map; - -import org.apache.lucene.util.hppc.BitMixer; -import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; -import org.elasticsearch.search.aggregations.metrics.AbstractHyperLogLogPlusPlus; -import org.elasticsearch.search.aggregations.metrics.CardinalityAggregationBuilder; -import org.elasticsearch.search.aggregations.metrics.HyperLogLogPlusPlus; -import org.elasticsearch.search.aggregations.metrics.InternalCardinality; import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesCountValues; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; -public class CountValuesFunction implements AggregatorFunction { +public class CountValuesFunction implements AggregatorFunction> { - private HyperLogLogPlusPlus counts; + private Map valueCount; public CountValuesFunction() { - counts = new HyperLogLogPlusPlus(HyperLogLogPlusPlus.DEFAULT_PRECISION, BigArrays.NON_RECYCLING_INSTANCE, 1); + valueCount = new HashMap<>(); } @Override public void collect(TimePoint value) { - long hash = BitMixer.mix64(java.lang.Double.doubleToLongBits(value.getValue())); - counts.collect(0, hash); + long val = java.lang.Double.doubleToLongBits(value.getValue()); + AtomicInteger count = valueCount.get(value); + if (count == null) { + count = new AtomicInteger(0); + valueCount.put(val, count); + } + count.incrementAndGet(); } @Override - public Long get() { - AbstractHyperLogLogPlusPlus copy = counts.clone(0, BigArrays.NON_RECYCLING_INSTANCE); - return copy.cardinality(0); + public Map get() { + return valueCount; } @Override - public InternalAggregation getAggregation( - DocValueFormat formatter, Map metadata) { - AbstractHyperLogLogPlusPlus copy = counts.clone(0, BigArrays.NON_RECYCLING_INSTANCE); - return new InternalCardinality(CardinalityAggregationBuilder.NAME, copy, metadata); + public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { + return new TimeSeriesCountValues(TimeSeriesCountValues.NAME, valueCount, formatter, metadata); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesCountValues.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesCountValues.java new file mode 100644 index 0000000000000..b2d9fa39adfa3 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesCountValues.java @@ -0,0 +1,96 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.internal; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.AggregationReduceContext; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * @author weizijun.wzj + * @date 2022/5/23 + */ +public class TimeSeriesCountValues extends InternalAggregation { + public static final String NAME = "time_series_count_values"; + + private final Map valueCount; + private final DocValueFormat formatter; + + public TimeSeriesCountValues(String name, Map valueCount, DocValueFormat formatter, Map metadata) { + super(name, metadata); + this.valueCount = valueCount; + this.formatter = formatter; + } + + public TimeSeriesCountValues(StreamInput in) throws IOException { + super(in); + formatter = in.readNamedWriteable(DocValueFormat.class); + valueCount = in.readOrderedMap(input -> input.readLong(), input -> new AtomicInteger(input.readInt())); + } + + @Override + public String getWriteableName() { + return NAME; + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + out.writeNamedWriteable(formatter); + out.writeMap(valueCount, (out1, value) -> out1.writeLong(value), (out1, value) -> out1.writeInt(value.get())); + } + + @Override + public InternalAggregation reduce(List aggregations, AggregationReduceContext reduceContext) { + + Map reduced = new HashMap<>(); + for (InternalAggregation internalAggregation : aggregations) { + TimeSeriesCountValues timeSeriesCountValues = (TimeSeriesCountValues) internalAggregation; + if (reduced.isEmpty()) { + reduced.putAll(timeSeriesCountValues.valueCount); + } else { + timeSeriesCountValues.valueCount.forEach((value, count) -> { + if (reduced.containsKey(value)) { + AtomicInteger current = reduced.get(value); + current.addAndGet(count.get()); + } else { + reduced.put(value, count); + } + }); + } + } + return new TimeSeriesCountValues(name, reduced, formatter, getMetadata()); + } + + @Override + protected boolean mustReduceOnSingleInternalAgg() { + return false; + } + + @Override + public Object getProperty(List path) { + return null; + } + + @Override + public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + for (Map.Entry value : valueCount.entrySet()) { + builder.field(String.valueOf(Double.longBitsToDouble(value.getKey())), value.getValue().get()); + } + return builder; + } +} From 6374917f8648abb7a1bdcd282e4f1cc16dc9abd7 Mon Sep 17 00:00:00 2001 From: weizijun Date: Tue, 24 May 2022 15:18:07 +0800 Subject: [PATCH 26/53] support more function: abs,ceil,floor,exp,sqrt,ln,log10,sin,cos,tan,asin,acos,getFunction,atan,sinh,cosh,tanh,rad,deg,pi,sgn,timestamp,day_of_month,day_of_week,hour,minute,month,year,clamp,clamp_max,clamp_min,quantile_over_time,last_over_time --- .../timeseries/aggregation/Downsample.java | 4 + .../timeseries/aggregation/Function.java | 233 +++++++++++++++++- .../TimeSeriesAggregationAggregator.java | 2 + .../aggregation/function/AbsentFunction.java | 20 ++ .../function/AbstractLastFunction.java | 44 ++++ .../aggregation/function/ClampFunction.java | 27 ++ .../function/ClampMaxFunction.java | 23 ++ .../function/ClampMinFunction.java | 23 ++ .../aggregation/function/DateFunction.java | 37 +++ .../aggregation/function/LastFunction.java | 27 +- .../function/TimestampFunction.java | 17 ++ 11 files changed, 436 insertions(+), 21 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AbsentFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AbstractLastFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ClampFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ClampMaxFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ClampMinFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/DateFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/TimestampFunction.java diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java index 41cdab35c09ec..4be1034cfae53 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java @@ -76,6 +76,10 @@ public Function getFunction() { return function; } + public Map getParams() { + return params; + } + public static Downsample fromXContent(XContentParser parser) { return PARSER.apply(parser, null); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java index 3106fb36deed4..aa965c12c8ceb 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java @@ -10,14 +10,23 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AvgFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.ClampFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.ClampMaxFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.ClampMinFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.DateFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.IRateFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.LastFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MaxFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MinFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.QuantileFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.RateFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.SumFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.TimestampFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.ValueCountFunction; +import java.time.DayOfWeek; +import java.time.ZonedDateTime; + public enum Function { count_over_time { @Override @@ -52,7 +61,7 @@ public enum Function { last { @Override public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { - return new LastFunction(); + return new LastFunction(value -> value); } }, rate { @@ -84,6 +93,228 @@ public enum Function { public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { return new IRateFunction(true); } + }, + abs { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new LastFunction(Math::abs); + } + }, + ceil { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new LastFunction(Math::ceil); + } + }, + floor { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new LastFunction(Math::floor); + } + }, + exp { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new LastFunction(Math::exp); + } + }, + sqrt { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new LastFunction(Math::sqrt); + } + }, + ln { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new LastFunction(Math::log); + } + }, + log10 { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new LastFunction(Math::log10); + } + }, + sin { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new LastFunction(Math::sin); + } + }, + cos { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new LastFunction(Math::cos); + } + }, + tan { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new LastFunction(Math::tan); + } + }, + asin { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new LastFunction(Math::asin); + } + }, + acos { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new LastFunction(Math::acos); + } + }, + atan { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new LastFunction(Math::atan); + } + }, + sinh { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new LastFunction(Math::sinh); + } + }, + cosh { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new LastFunction(Math::cosh); + } + }, + tanh { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new LastFunction(Math::tanh); + } + }, + rad { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new LastFunction(value -> value * Math.PI / 180); + } + }, + deg { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new LastFunction(value -> value * 180 / Math.PI); + } + }, + pi { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new LastFunction(value -> Math.PI); + } + }, + sgn { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new LastFunction(v -> { + if (v < 0) { + return -1d; + } else if (v > 0) { + return 1d; + } + return v; + }); + } + }, + timestamp { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new TimestampFunction(); + } + }, + day_of_month { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new DateFunction(ZonedDateTime::getDayOfMonth); + } + }, + day_of_week { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new DateFunction(zonedDateTime -> { + DayOfWeek dayOfWeek = zonedDateTime.getDayOfWeek(); + return dayOfWeek.getValue() % 7; + }); + } + }, + hour { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new DateFunction(ZonedDateTime::getHour); + } + }, + minute { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new DateFunction(ZonedDateTime::getMinute); + } + }, + month { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new DateFunction(ZonedDateTime::getMonthValue); + } + }, + year { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new DateFunction(ZonedDateTime::getMonthValue); + } + }, + clamp { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + double max = Double.MAX_VALUE; + if (aggregator.downsampleParams != null && aggregator.downsampleParams.containsKey("max")) { + max = Double.valueOf(String.valueOf(aggregator.downsampleParams.get("max"))); + } + double min = Double.MIN_VALUE; + if (aggregator.downsampleParams != null && aggregator.downsampleParams.containsKey("min")) { + min = Double.valueOf(String.valueOf(aggregator.downsampleParams.get("min"))); + } + return new ClampFunction(max, min); + } + }, + clamp_max { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + double max = Double.MAX_VALUE; + if (aggregator.downsampleParams != null && aggregator.downsampleParams.containsKey("max")) { + max = Double.valueOf(String.valueOf(aggregator.downsampleParams.get("max"))); + } + return new ClampMaxFunction(max); + } + }, + clamp_min { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + double min = Double.MIN_VALUE; + if (aggregator.downsampleParams != null && aggregator.downsampleParams.containsKey("min")) { + min = Double.valueOf(String.valueOf(aggregator.downsampleParams.get("min"))); + } + return new ClampMinFunction(min); + } + }, + quantile_over_time { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + double quantile = 1f; + if (aggregator.downsampleParams != null && aggregator.downsampleParams.containsKey("quantile")) { + quantile = (double) aggregator.downsampleParams.get("quantile"); + } + return new QuantileFunction(quantile); + } + }, + last_over_time { + @Override + public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + return new LastFunction(value -> value); + } }; public static Function resolve(String name) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index 4b1c942f82698..58b97d05fa0b3 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -78,6 +78,7 @@ public class TimeSeriesAggregationAggregator extends BucketsAggregator { private Map aggregatorParams; protected long downsampleRange; protected Function downsampleFunction; + protected Map downsampleParams; private BucketOrder order; private TermsAggregator.BucketCountThresholds bucketCountThresholds; protected Comparator partiallyBuiltBucketComparator; @@ -131,6 +132,7 @@ public TimeSeriesAggregationAggregator( if (this.downsampleRange <= 0) { this.downsampleRange = this.interval; } + this.downsampleParams = downsample != null ? downsample.getParams() : null; this.bucketCountThresholds = bucketCountThresholds; this.order = order == null ? BucketOrder.key(true) : order; this.partiallyBuiltBucketComparator = order.partiallyBuiltBucketComparator(b -> b.bucketOrd, this); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AbsentFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AbsentFunction.java new file mode 100644 index 0000000000000..6cedbc25f4c7f --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AbsentFunction.java @@ -0,0 +1,20 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.function; + +public class AbsentFunction extends AbstractLastFunction { + @Override + protected Double interGet() { + if (getPoint() == null) { + return 1d; + } else { + return Double.NaN; + } + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AbstractLastFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AbstractLastFunction.java new file mode 100644 index 0000000000000..c2198dad91ee0 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AbstractLastFunction.java @@ -0,0 +1,44 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.function; + +import java.util.Map; + +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesLast; + +public abstract class AbstractLastFunction implements AggregatorFunction { + protected TimePoint point; + + @Override + public void collect(TimePoint value) { + if (this.point == null || value.getTimestamp() > this.point.getTimestamp()) { + this.point = value; + } + } + + @Override + public Double get() { + return interGet(); + } + + @Override + public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { + return new TimeSeriesLast(TimeSeriesLast.NAME, interGet(), point.getTimestamp(), formatter, metadata); + } + + protected abstract Double interGet(); + + protected TimePoint getPoint() { + return point; + } +} + diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ClampFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ClampFunction.java new file mode 100644 index 0000000000000..66ca3e471266d --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ClampFunction.java @@ -0,0 +1,27 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.function; + +public class ClampFunction extends AbstractLastFunction { + private final Double max; + private final Double min; + + public ClampFunction(Double max, Double min) { + this.max = max; + this.min = min; + } + + @Override + protected Double interGet() { + if (min > max) { + return Double.NaN; + } + return Math.max(min, Math.min(max, getPoint().getValue())); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ClampMaxFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ClampMaxFunction.java new file mode 100644 index 0000000000000..c2ead17743b7b --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ClampMaxFunction.java @@ -0,0 +1,23 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.function; + +public class ClampMaxFunction extends AbstractLastFunction { + + private final Double max; + + public ClampMaxFunction(double max) { + this.max = max; + } + + @Override + protected Double interGet() { + return Math.min(max, getPoint().getValue()); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ClampMinFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ClampMinFunction.java new file mode 100644 index 0000000000000..a78733aec0bf0 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ClampMinFunction.java @@ -0,0 +1,23 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.function; + +public class ClampMinFunction extends AbstractLastFunction{ + + private final Double min; + + public ClampMinFunction(double min) { + this.min = min; + } + + @Override + protected Double interGet() { + return Math.max(min, getPoint().getValue()); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/DateFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/DateFunction.java new file mode 100644 index 0000000000000..879480c3f5734 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/DateFunction.java @@ -0,0 +1,37 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.function; + +import java.time.Instant; +import java.time.ZoneId; +import java.time.ZonedDateTime; +import java.util.function.Function; + +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; + +/** + * @author weizijun.wzj + * @date 2022/5/24 + */ +public class DateFunction extends AbstractLastFunction { + + private Function function; + + public DateFunction(Function function) { + this.function = function; + } + + @Override + protected Double interGet() { + TimePoint point = getPoint(); + Instant instant = Instant.ofEpochMilli(point.getTimestamp()); + ZonedDateTime zonedDateTime = ZonedDateTime.ofInstant(instant, ZoneId.of("UTC")); + return function.apply(zonedDateTime).doubleValue(); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java index ceee5402112d3..0368c6f7022a4 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.java @@ -8,30 +8,17 @@ package org.elasticsearch.search.aggregations.timeseries.aggregation.function; -import org.elasticsearch.search.DocValueFormat; -import org.elasticsearch.search.aggregations.InternalAggregation; -import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; -import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesLast; +import java.util.function.Function; -import java.util.Map; +public class LastFunction extends AbstractLastFunction { + private Function function; -public class LastFunction implements AggregatorFunction { - private TimePoint point; - - @Override - public void collect(TimePoint value) { - if (this.point == null || value.getTimestamp() > this.point.getTimestamp()) { - this.point = value; - } - } - - @Override - public Double get() { - return point.getValue(); + public LastFunction(Function function) { + this.function = function; } @Override - public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { - return new TimeSeriesLast(TimeSeriesLast.NAME, point.getValue(), point.getTimestamp(), formatter, metadata); + public Double interGet() { + return function.apply(point.getValue()); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/TimestampFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/TimestampFunction.java new file mode 100644 index 0000000000000..6d69ffe537d58 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/TimestampFunction.java @@ -0,0 +1,17 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.function; + +public class TimestampFunction extends AbstractLastFunction { + + @Override + protected Double interGet() { + return (double) getPoint().getTimestamp() / 1000; + } +} From 7891163dc892fe414f51341435890e4a8c8be389 Mon Sep 17 00:00:00 2001 From: weizijun Date: Tue, 24 May 2022 15:59:33 +0800 Subject: [PATCH 27/53] fix failed check --- .../timeseries/aggregation/function/DateFunction.java | 4 ---- .../aggregation/internal/TimeSeriesCountValues.java | 4 ---- .../internal/TimeSeriesLineAggreagation.java | 4 ---- .../InternalTimeSeriesAggregationTests.java | 11 ++++++++--- ...ggregateMetricTimeSeriesAggregationAggregator.java | 2 ++ 5 files changed, 10 insertions(+), 15 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/DateFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/DateFunction.java index 879480c3f5734..95c74844391ce 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/DateFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/DateFunction.java @@ -15,10 +15,6 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; -/** - * @author weizijun.wzj - * @date 2022/5/24 - */ public class DateFunction extends AbstractLastFunction { private Function function; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesCountValues.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesCountValues.java index b2d9fa39adfa3..c01579b004d5a 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesCountValues.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesCountValues.java @@ -21,10 +21,6 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; -/** - * @author weizijun.wzj - * @date 2022/5/23 - */ public class TimeSeriesCountValues extends InternalAggregation { public static final String NAME = "time_series_count_values"; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesLineAggreagation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesLineAggreagation.java index 2524f3c69b1bf..fddf8c811f70e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesLineAggreagation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesLineAggreagation.java @@ -22,10 +22,6 @@ import java.util.Map.Entry; import java.util.TreeMap; -/** - * @author weizijun.wzj - * @date 2022/5/23 - */ public class TimeSeriesLineAggreagation extends InternalAggregation { public static final String NAME = "time_series_line"; diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregationTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregationTests.java index 196e06cc856bb..d3d82c460844e 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregationTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregationTests.java @@ -8,16 +8,16 @@ package org.elasticsearch.search.aggregations.timeseries.aggregation; +import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.BucketOrder; import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.ParsedMultiBucketAggregation; -import org.elasticsearch.search.aggregations.metrics.Max; import org.elasticsearch.search.aggregations.timeseries.aggregation.InternalTimeSeriesAggregation.InternalBucket; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesLineAggreagation; import org.elasticsearch.test.InternalMultiBucketAggregationTestCase; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.TreeMap; @@ -85,7 +85,12 @@ private List randomBuckets( new InternalTimeSeriesAggregation.InternalBucket( keys.get(j), docCount, - null, + new TimeSeriesLineAggreagation( + TimeSeriesLineAggreagation.NAME, + Collections.emptyMap(), + DocValueFormat.RAW, + Collections.emptyMap() + ), InternalAggregations.EMPTY, keyed, showTermDocCountError, diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java index 267e579eae4a7..54c2725817ca8 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java @@ -47,6 +47,7 @@ public AggregateMetricTimeSeriesAggregationAggregator( DateHistogramInterval interval, DateHistogramInterval offset, org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator aggregator, + Map aggregatorParams, Downsample downsample, BucketCountThresholds bucketCountThresholds, BucketOrder order, @@ -65,6 +66,7 @@ public AggregateMetricTimeSeriesAggregationAggregator( interval, offset, aggregator, + aggregatorParams, downsample, bucketCountThresholds, order, From 159f32a574c8f00402310220ed17771eb4f419d5 Mon Sep 17 00:00:00 2001 From: weizijun Date: Tue, 24 May 2022 20:09:47 +0800 Subject: [PATCH 28/53] fixup --- .../timeseries/aggregation/Downsample.java | 18 +++++++++--------- .../TimeSeriesAggregationAggregator.java | 2 +- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java index 4be1034cfae53..861843d868b56 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java @@ -54,18 +54,18 @@ public class Downsample implements ToXContentObject, Writeable { private final DateHistogramInterval range; private final Function function; - private final Map params; + private final Map parameters; - public Downsample(DateHistogramInterval range, Function function, Map params) { + public Downsample(DateHistogramInterval range, Function function, Map parameters) { this.range = range; this.function = function; - this.params = params; + this.parameters = parameters; } public Downsample(StreamInput in) throws IOException { this.range = new DateHistogramInterval(in); this.function = Function.resolve(in.readString()); - this.params = in.readMap(); + this.parameters = in.readMap(); } public DateHistogramInterval getRange() { @@ -76,8 +76,8 @@ public Function getFunction() { return function; } - public Map getParams() { - return params; + public Map getParameters() { + return parameters; } public static Downsample fromXContent(XContentParser parser) { @@ -89,8 +89,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startObject(); builder.field(RANGE_FIELD.getPreferredName(), range); builder.field(FUNCTION_FIELD.getPreferredName(), function); - if (params != null) { - builder.field(PARAMS_FIELD.getPreferredName(), params); + if (parameters != null) { + builder.field(PARAMS_FIELD.getPreferredName(), parameters); } builder.endObject(); return builder; @@ -100,6 +100,6 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws public void writeTo(StreamOutput out) throws IOException { range.writeTo(out); out.writeString(function.name()); - out.writeGenericMap(params); + out.writeGenericMap(parameters); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index 58b97d05fa0b3..6dc95e7edd099 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -132,7 +132,7 @@ public TimeSeriesAggregationAggregator( if (this.downsampleRange <= 0) { this.downsampleRange = this.interval; } - this.downsampleParams = downsample != null ? downsample.getParams() : null; + this.downsampleParams = downsample != null ? downsample.getParameters() : null; this.bucketCountThresholds = bucketCountThresholds; this.order = order == null ? BucketOrder.key(true) : order; this.partiallyBuiltBucketComparator = order.partiallyBuiltBucketComparator(b -> b.bucketOrd, this); From d29c0d5d4d6ae52fba387b073c0552deb5a3bd4a Mon Sep 17 00:00:00 2001 From: weizijun Date: Wed, 25 May 2022 15:07:23 +0800 Subject: [PATCH 29/53] fixup --- .../timeseries/aggregation/Aggregator.java | 8 +++--- .../timeseries/aggregation/Downsample.java | 10 +++++--- .../TimeSeriesAggregationAggregator.java | 2 +- .../bucketfunction/TopkBucketFunction.java | 12 ++------- .../aggregation/function/TopkFunction.java | 25 +++++++++++-------- .../internal/TimeSeriesCountValues.java | 4 +++ .../aggregation/internal/TimeSeriesTopk.java | 12 ++------- 7 files changed, 34 insertions(+), 39 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java index b9b31f3a8527e..1b7dd400f2b64 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java @@ -91,7 +91,7 @@ public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArra public AggregatorFunction getAggregatorFunction(Map aggregatorParams) { int size = 0; if (aggregatorParams != null && aggregatorParams.containsKey("size")) { - size = (int) aggregatorParams.get("size"); + size = ((Double) aggregatorParams.get("size")).intValue(); } return new TopkFunction(size, true); } @@ -100,7 +100,7 @@ public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArra public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, Map aggregatorParams) { int size = 0; if (aggregatorParams != null && aggregatorParams.containsKey("size")) { - size = (int) aggregatorParams.get("size"); + size = ((Double) aggregatorParams.get("size")).intValue(); } return new TopkBucketFunction(size, true); } @@ -110,7 +110,7 @@ public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArra public AggregatorFunction getAggregatorFunction(Map aggregatorParams) { int size = 0; if (aggregatorParams != null && aggregatorParams.containsKey("size")) { - size = (int) aggregatorParams.get("size"); + size = ((Double) aggregatorParams.get("size")).intValue(); } return new TopkFunction(size, false); } @@ -119,7 +119,7 @@ public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArra public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, Map aggregatorParams) { int size = 0; if (aggregatorParams != null && aggregatorParams.containsKey("size")) { - size = (int) aggregatorParams.get("size"); + size = ((Double) aggregatorParams.get("size")).intValue(); } return new TopkBucketFunction(size, false); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java index 861843d868b56..20cdb09e0c264 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java @@ -38,7 +38,7 @@ public class Downsample implements ToXContentObject, Writeable { static { PARSER.declareField( - ConstructingObjectParser.constructorArg(), + ConstructingObjectParser.optionalConstructorArg(), p -> new DateHistogramInterval(p.text()), RANGE_FIELD, ObjectParser.ValueType.STRING @@ -63,7 +63,7 @@ public Downsample(DateHistogramInterval range, Function function, Map number, long bucket) { PriorityQueue> queue = values.get(bucket); if (queue == null) { - queue = new PriorityQueue<>(topkSize) { - @Override - protected boolean lessThan(TSIDValue a, TSIDValue b) { - if (isTop) { - return a.value > b.value; - } else { - return a.value < b.value; - } - } - }; + queue = TopkFunction.getTopkQueue(topkSize, isTop); values.put(bucket, queue); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/TopkFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/TopkFunction.java index 1e7e83bc4e5d7..10398f631392b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/TopkFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/TopkFunction.java @@ -24,16 +24,7 @@ public class TopkFunction implements AggregatorFunction, List< private final boolean isTop; public TopkFunction(int size, boolean isTop) { - queue = new PriorityQueue<>(size) { - @Override - protected boolean lessThan(TSIDValue a, TSIDValue b) { - if (isTop) { - return a.value > b.value; - } else { - return a.value < b.value; - } - } - }; + queue = getTopkQueue(size, isTop); this.isTop = isTop; this.topkSize = size; } @@ -56,4 +47,18 @@ public List> get() { public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { return new TimeSeriesTopk(TimeSeriesTopk.NAME, get(), topkSize, isTop, formatter, metadata); } + + public static PriorityQueue> getTopkQueue(int size, boolean isTop) { + return new PriorityQueue>(size) { + @Override + protected boolean lessThan(TSIDValue a, TSIDValue b) { + if (isTop) { + return a.value < b.value; + } else { + return a.value > b.value; + } + } + }; + } + } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesCountValues.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesCountValues.java index c01579b004d5a..bd9f4e112bfdf 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesCountValues.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesCountValues.java @@ -44,6 +44,10 @@ public String getWriteableName() { return NAME; } + public Map getValueCount() { + return valueCount; + } + @Override protected void doWriteTo(StreamOutput out) throws IOException { out.writeNamedWriteable(formatter); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesTopk.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesTopk.java index c95ec07827b04..11bbe406b9456 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesTopk.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesTopk.java @@ -16,6 +16,7 @@ import org.elasticsearch.search.aggregations.AggregationReduceContext; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.timeseries.aggregation.TSIDValue; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.TopkFunction; import org.elasticsearch.xcontent.XContentBuilder; import java.io.IOException; @@ -76,16 +77,7 @@ protected void doWriteTo(StreamOutput out) throws IOException { @Override public InternalAggregation reduce(List aggregations, AggregationReduceContext reduceContext) { - PriorityQueue> queue = new PriorityQueue<>(topkSize) { - @Override - protected boolean lessThan(TSIDValue a, TSIDValue b) { - if (isTop) { - return a.value > b.value; - } else { - return a.value < b.value; - } - } - }; + PriorityQueue> queue = TopkFunction.getTopkQueue(topkSize, isTop); for (InternalAggregation internalAggregation : aggregations) { TimeSeriesTopk timeSeriesTopk = (TimeSeriesTopk) internalAggregation; From 0a8113fd1ad63e74363d4158491f299bb77bc69e Mon Sep 17 00:00:00 2001 From: weizijun Date: Wed, 25 May 2022 15:23:18 +0800 Subject: [PATCH 30/53] function param improve --- .../timeseries/aggregation/Function.java | 121 +++++++++--------- 1 file changed, 64 insertions(+), 57 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java index aa965c12c8ceb..aed165910713a 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java @@ -26,191 +26,192 @@ import java.time.DayOfWeek; import java.time.ZonedDateTime; +import java.util.Map; public enum Function { count_over_time { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new ValueCountFunction(); } }, sum_over_time { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new SumFunction(); } }, min_over_time { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new MinFunction(); } }, max_over_time { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new MaxFunction(); } }, avg_over_time { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new AvgFunction(); } }, last { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new LastFunction(value -> value); } }, rate { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { - return new RateFunction(aggregator.downsampleRange, aggregator.preRounding, true, true); + public AggregatorFunction getFunction(Map params) { + return new RateFunction((long)params.get(RANGE_FIELD), (long)params.get(ROUNDING_FIELD), true, true); } }, delta { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { - return new RateFunction(aggregator.downsampleRange, aggregator.preRounding, false, false); + public AggregatorFunction getFunction(Map params) { + return new RateFunction((long)params.get(RANGE_FIELD), (long)params.get(ROUNDING_FIELD), false, false); } }, increase { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { - return new RateFunction(aggregator.downsampleRange, aggregator.preRounding, true, false); + public AggregatorFunction getFunction(Map params) { + return new RateFunction((long)params.get(RANGE_FIELD), (long)params.get(ROUNDING_FIELD), true, false); } }, irate { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new IRateFunction(true); } }, idelta { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new IRateFunction(true); } }, abs { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::abs); } }, ceil { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::ceil); } }, floor { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::floor); } }, exp { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::exp); } }, sqrt { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::sqrt); } }, ln { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::log); } }, log10 { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::log10); } }, sin { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::sin); } }, cos { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::cos); } }, tan { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::tan); } }, asin { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::asin); } }, acos { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::acos); } }, atan { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::atan); } }, sinh { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::sinh); } }, cosh { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::cosh); } }, tanh { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::tanh); } }, rad { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new LastFunction(value -> value * Math.PI / 180); } }, deg { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new LastFunction(value -> value * 180 / Math.PI); } }, pi { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new LastFunction(value -> Math.PI); } }, sgn { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new LastFunction(v -> { if (v < 0) { return -1d; @@ -223,19 +224,19 @@ public enum Function { }, timestamp { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new TimestampFunction(); } }, day_of_month { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new DateFunction(ZonedDateTime::getDayOfMonth); } }, day_of_week { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new DateFunction(zonedDateTime -> { DayOfWeek dayOfWeek = zonedDateTime.getDayOfWeek(); return dayOfWeek.getValue() % 7; @@ -244,79 +245,85 @@ public enum Function { }, hour { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new DateFunction(ZonedDateTime::getHour); } }, minute { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new DateFunction(ZonedDateTime::getMinute); } }, month { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new DateFunction(ZonedDateTime::getMonthValue); } }, year { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new DateFunction(ZonedDateTime::getMonthValue); } }, clamp { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { double max = Double.MAX_VALUE; - if (aggregator.downsampleParams != null && aggregator.downsampleParams.containsKey("max")) { - max = Double.valueOf(String.valueOf(aggregator.downsampleParams.get("max"))); + if (params != null && params.containsKey(MAX_FIELD)) { + max = Double.valueOf(String.valueOf(params.get(MAX_FIELD))); } double min = Double.MIN_VALUE; - if (aggregator.downsampleParams != null && aggregator.downsampleParams.containsKey("min")) { - min = Double.valueOf(String.valueOf(aggregator.downsampleParams.get("min"))); + if (params != null && params.containsKey(MIN_FIELD)) { + min = Double.valueOf(String.valueOf(params.get(MIN_FIELD))); } return new ClampFunction(max, min); } }, clamp_max { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { double max = Double.MAX_VALUE; - if (aggregator.downsampleParams != null && aggregator.downsampleParams.containsKey("max")) { - max = Double.valueOf(String.valueOf(aggregator.downsampleParams.get("max"))); + if (params != null && params.containsKey(MAX_FIELD)) { + max = Double.valueOf(String.valueOf(params.get(MAX_FIELD))); } return new ClampMaxFunction(max); } }, clamp_min { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { double min = Double.MIN_VALUE; - if (aggregator.downsampleParams != null && aggregator.downsampleParams.containsKey("min")) { - min = Double.valueOf(String.valueOf(aggregator.downsampleParams.get("min"))); + if (params != null && params.containsKey(MIN_FIELD)) { + min = Double.valueOf(String.valueOf(params.get(MIN_FIELD))); } return new ClampMinFunction(min); } }, quantile_over_time { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { double quantile = 1f; - if (aggregator.downsampleParams != null && aggregator.downsampleParams.containsKey("quantile")) { - quantile = (double) aggregator.downsampleParams.get("quantile"); + if (params != null && params.containsKey(QUANTILE_FIELD)) { + quantile = (double) params.get(QUANTILE_FIELD); } return new QuantileFunction(quantile); } }, last_over_time { @Override - public AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator) { + public AggregatorFunction getFunction(Map params) { return new LastFunction(value -> value); } }; + public static final String RANGE_FIELD = "range"; + public static final String ROUNDING_FIELD = "rounding"; + public static final String MAX_FIELD = "max"; + public static final String MIN_FIELD = "min"; + public static final String QUANTILE_FIELD = "quantile"; + public static Function resolve(String name) { try { return Function.valueOf(name); @@ -328,5 +335,5 @@ public static Function resolve(String name) { /** * get the function */ - public abstract AggregatorFunction getFunction(TimeSeriesAggregationAggregator aggregator); + public abstract AggregatorFunction getFunction(Map params); } From 9eaa3305826cc36bfeced2d3c4a83127be827baf Mon Sep 17 00:00:00 2001 From: weizijun Date: Wed, 25 May 2022 15:23:40 +0800 Subject: [PATCH 31/53] function param improve --- .../aggregation/TimeSeriesAggregationAggregator.java | 6 ++++-- .../AggregateMetricTimeSeriesAggregationAggregator.java | 6 ++++-- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index 795ef6a88cd66..029d660f47428 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -132,7 +132,8 @@ public TimeSeriesAggregationAggregator( if (this.downsampleRange <= 0) { this.downsampleRange = this.interval; } - this.downsampleParams = downsample != null ? downsample.getParameters() : null; + this.downsampleParams = downsample != null && downsample.getParameters() != null ? new HashMap<>(downsample.getParameters()) : new HashMap<>(); + this.downsampleParams.put(Function.RANGE_FIELD, downsampleRange); this.bucketCountThresholds = bucketCountThresholds; this.order = order == null ? BucketOrder.key(true) : order; this.partiallyBuiltBucketComparator = order.partiallyBuiltBucketComparator(b -> b.bucketOrd, this); @@ -340,7 +341,8 @@ public boolean isNoop() { for (int i = 0; i < valuesCount; i++) { double value = values.nextValue(); if (false == timeBucketMetrics.containsKey(preRounding)) { - timeBucketMetrics.put(preRounding, downsampleFunction.getFunction(this)); + downsampleParams.put(Function.ROUNDING_FIELD, preRounding); + timeBucketMetrics.put(preRounding, downsampleFunction.getFunction(downsampleParams)); } for (Entry entry : timeBucketMetrics.entrySet()) { Long timestamp = entry.getKey(); diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java index 54c2725817ca8..9e02194e15367 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java @@ -119,7 +119,8 @@ public boolean isNoop() { for (int i = 0; i < valuesCount; i++) { double value = values.nextValue(); if (false == timeBucketMetrics.containsKey(preRounding)) { - timeBucketMetrics.put(preRounding, downsampleFunction.getFunction(this)); + downsampleParams.put(Function.ROUNDING_FIELD, preRounding); + timeBucketMetrics.put(preRounding, downsampleFunction.getFunction(downsampleParams)); } for (Entry entry : timeBucketMetrics.entrySet()) { Long timestamp = entry.getKey(); @@ -160,7 +161,8 @@ public boolean isNoop() { } if (false == timeBucketMetrics.containsKey(preRounding)) { - timeBucketMetrics.put(preRounding, downsampleFunction.getFunction(this)); + downsampleParams.put(Function.ROUNDING_FIELD, preRounding); + timeBucketMetrics.put(preRounding, downsampleFunction.getFunction(downsampleParams)); } for (Entry entry : timeBucketMetrics.entrySet()) { Long timestamp = entry.getKey(); From a479a506dc463d41aecada39054e30550d783d1f Mon Sep 17 00:00:00 2001 From: weizijun Date: Thu, 26 May 2022 15:43:07 +0800 Subject: [PATCH 32/53] return NaN when value is invaild --- .../timeseries/aggregation/function/IRateFunction.java | 4 ++-- .../timeseries/aggregation/function/RateFunction.java | 2 +- .../timeseries/aggregation/function/TopkFunction.java | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/IRateFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/IRateFunction.java index 3d16a3a2a6411..8c35786392b10 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/IRateFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/IRateFunction.java @@ -52,7 +52,7 @@ public InternalAggregation getAggregation(DocValueFormat formatter, Map> getTopkQueue(int size, boolean isTop) { - return new PriorityQueue>(size) { + return new PriorityQueue<>(size) { @Override protected boolean lessThan(TSIDValue a, TSIDValue b) { if (isTop) { From 112c7abf7650f46a8d8fa28f26614ad995bdeefc Mon Sep 17 00:00:00 2001 From: weizijun Date: Fri, 27 May 2022 13:55:50 +0800 Subject: [PATCH 33/53] improve aggregate_metric_double --- .../timeseries/aggregation/Function.java | 20 +++++- .../TimeSeriesAggregationAggregator.java | 62 +++++++++++-------- .../function/AvgExactFunction.java | 37 +++++++++++ .../aggregation/function/AvgFunction.java | 5 -- .../function/ValueCountExactFunction.java | 35 +++++++++++ .../function/ValueCountFunction.java | 4 -- ...MetricTimeSeriesAggregationAggregator.java | 46 ++++---------- 7 files changed, 138 insertions(+), 71 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AvgExactFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountExactFunction.java diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java index aed165910713a..0c9eadf9bd6f6 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java @@ -9,6 +9,7 @@ package org.elasticsearch.search.aggregations.timeseries.aggregation; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AvgExactFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AvgFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.ClampFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.ClampMaxFunction; @@ -22,6 +23,7 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.function.RateFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.SumFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.TimestampFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.ValueCountExactFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.ValueCountFunction; import java.time.DayOfWeek; @@ -35,6 +37,12 @@ public enum Function { return new ValueCountFunction(); } }, + count_exact_over_time { + @Override + public AggregatorFunction getFunction(Map params) { + return new ValueCountExactFunction(); + } + }, sum_over_time { @Override public AggregatorFunction getFunction(Map params) { @@ -59,6 +67,12 @@ public enum Function { return new AvgFunction(); } }, + avg_exact_over_time { + @Override + public AggregatorFunction getFunction(Map params) { + return new AvgExactFunction(); + } + }, last { @Override public AggregatorFunction getFunction(Map params) { @@ -68,19 +82,19 @@ public enum Function { rate { @Override public AggregatorFunction getFunction(Map params) { - return new RateFunction((long)params.get(RANGE_FIELD), (long)params.get(ROUNDING_FIELD), true, true); + return new RateFunction((long) params.get(RANGE_FIELD), (long) params.get(ROUNDING_FIELD), true, true); } }, delta { @Override public AggregatorFunction getFunction(Map params) { - return new RateFunction((long)params.get(RANGE_FIELD), (long)params.get(ROUNDING_FIELD), false, false); + return new RateFunction((long) params.get(RANGE_FIELD), (long) params.get(ROUNDING_FIELD), false, false); } }, increase { @Override public AggregatorFunction getFunction(Map params) { - return new RateFunction((long)params.get(RANGE_FIELD), (long)params.get(ROUNDING_FIELD), true, false); + return new RateFunction((long) params.get(RANGE_FIELD), (long) params.get(ROUNDING_FIELD), true, false); } }, irate { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index 029d660f47428..c323f92a05be2 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -132,7 +132,9 @@ public TimeSeriesAggregationAggregator( if (this.downsampleRange <= 0) { this.downsampleRange = this.interval; } - this.downsampleParams = downsample != null && downsample.getParameters() != null ? new HashMap<>(downsample.getParameters()) : new HashMap<>(); + this.downsampleParams = downsample != null && downsample.getParameters() != null + ? new HashMap<>(downsample.getParameters()) + : new HashMap<>(); this.downsampleParams.put(Function.RANGE_FIELD, downsampleRange); this.bucketCountThresholds = bucketCountThresholds; this.order = order == null ? BucketOrder.key(true) : order; @@ -304,6 +306,38 @@ private void reset(BytesRef tsid, long bucket) { } } + protected LeafBucketCollector getCollector( + LeafBucketCollector sub, + AggregationExecutionContext aggCtx, + SortedNumericDoubleValues values + ) { + return new Collector(sub, values, aggCtx, (doc) -> { + if (aggCtx.getTimestamp() + downsampleRange < preRounding) { + return; + } + + if (values.advanceExact(doc)) { + final int valuesCount = values.docValueCount(); + for (int i = 0; i < valuesCount; i++) { + double value = values.nextValue(); + if (false == timeBucketMetrics.containsKey(preRounding)) { + downsampleParams.put(Function.ROUNDING_FIELD, preRounding); + timeBucketMetrics.put(preRounding, downsampleFunction.getFunction(downsampleParams)); + } + for (Entry entry : timeBucketMetrics.entrySet()) { + Long timestamp = entry.getKey(); + AggregatorFunction function = entry.getValue(); + if (aggCtx.getTimestamp() + downsampleRange >= timestamp) { + function.collect(new TimePoint(aggCtx.getTimestamp(), value)); + } else { + break; + } + } + } + } + }); + } + @Override protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBucketCollector sub) throws IOException { // TODO: remove this method in a follow up PR @@ -331,31 +365,7 @@ public boolean isNoop() { }; } final SortedNumericDoubleValues values = valuesSource.doubleValues(context); - return new Collector(sub, values, aggCtx, (doc) -> { - if (aggCtx.getTimestamp() + downsampleRange < preRounding) { - return; - } - - if (values.advanceExact(doc)) { - final int valuesCount = values.docValueCount(); - for (int i = 0; i < valuesCount; i++) { - double value = values.nextValue(); - if (false == timeBucketMetrics.containsKey(preRounding)) { - downsampleParams.put(Function.ROUNDING_FIELD, preRounding); - timeBucketMetrics.put(preRounding, downsampleFunction.getFunction(downsampleParams)); - } - for (Entry entry : timeBucketMetrics.entrySet()) { - Long timestamp = entry.getKey(); - AggregatorFunction function = entry.getValue(); - if (aggCtx.getTimestamp() + downsampleRange >= timestamp) { - function.collect(new TimePoint(aggCtx.getTimestamp(), value)); - } else { - break; - } - } - } - } - }); + return getCollector(sub, aggCtx, values); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AvgExactFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AvgExactFunction.java new file mode 100644 index 0000000000000..72ef657bec65b --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AvgExactFunction.java @@ -0,0 +1,37 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.function; + +import org.elasticsearch.core.Tuple; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.AvgAggregationBuilder; + +import java.util.Map; + +public class AvgExactFunction implements AggregatorFunction, Double> { + private double sum = 0; + private long count = 0; + + @Override + public void collect(Tuple value) { + this.sum += value.v1(); + this.count += value.v2(); + } + + @Override + public Double get() { + return sum / count; + } + + @Override + public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { + return new org.elasticsearch.search.aggregations.metrics.InternalAvg(AvgAggregationBuilder.NAME, sum, count, formatter, metadata); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AvgFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AvgFunction.java index 4bd22d3e11b2d..eea43c6645294 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AvgFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AvgFunction.java @@ -25,11 +25,6 @@ public void collect(TimePoint value) { count += 1; } - public void collectExact(double sum, long count) { - this.sum += sum; - this.count += count; - } - @Override public Double get() { return sum / count; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountExactFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountExactFunction.java new file mode 100644 index 0000000000000..4f9f9703325d0 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountExactFunction.java @@ -0,0 +1,35 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.function; + +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; + +import java.util.Map; + +public class ValueCountExactFunction implements AggregatorFunction { + private long count = 0; + + @Override + public void collect(TimePoint value) { + count += count; + } + + @Override + public Long get() { + return count; + } + + @Override + public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { + return new org.elasticsearch.search.aggregations.metrics.InternalValueCount(ValueCountAggregationBuilder.NAME, count, metadata); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountFunction.java index dee2d2da40ce1..dbe3aa8a99e0d 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountFunction.java @@ -23,10 +23,6 @@ public void collect(TimePoint value) { count += 1; } - public void collectExact(long count) { - this.count += count; - } - @Override public Long get() { return count; diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java index 9e02194e15367..33320ca354821 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java @@ -9,6 +9,7 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.Scorable; +import org.elasticsearch.core.Tuple; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; import org.elasticsearch.search.aggregations.AggregationExecutionContext; import org.elasticsearch.search.aggregations.Aggregator; @@ -24,8 +25,6 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.Function; import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregationAggregator; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AvgFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.function.ValueCountFunction; import org.elasticsearch.xpack.aggregatemetric.aggregations.support.AggregateMetricsValuesSource; import org.elasticsearch.xpack.aggregatemetric.mapper.AggregateDoubleMetricFieldMapper.Metric; @@ -81,8 +80,9 @@ public AggregateMetricTimeSeriesAggregationAggregator( : null; this.format = valuesSourceConfig.format(); if (this.downsampleFunction == null) { - this.downsampleFunction = Function.avg_over_time; + this.downsampleFunction = Function.avg_exact_over_time; } + rewriteFunction(); } @Override @@ -109,35 +109,7 @@ public boolean isNoop() { Metric metricType = getAggregateMetric(); if (metricType != null) { final SortedNumericDoubleValues values = valuesSource.getAggregateMetricValues(context, metricType); - return new Collector(sub, values, aggCtx, (doc) -> { - if (aggCtx.getTimestamp() + downsampleRange < preRounding) { - return; - } - - if (values.advanceExact(doc)) { - final int valuesCount = values.docValueCount(); - for (int i = 0; i < valuesCount; i++) { - double value = values.nextValue(); - if (false == timeBucketMetrics.containsKey(preRounding)) { - downsampleParams.put(Function.ROUNDING_FIELD, preRounding); - timeBucketMetrics.put(preRounding, downsampleFunction.getFunction(downsampleParams)); - } - for (Entry entry : timeBucketMetrics.entrySet()) { - Long timestamp = entry.getKey(); - AggregatorFunction function = entry.getValue(); - if (aggCtx.getTimestamp() + downsampleRange >= timestamp) { - if (function instanceof ValueCountFunction) { - ((ValueCountFunction) function).collectExact((long) value); - } else { - function.collect(value); - } - } else { - break; - } - } - } - } - }); + return getCollector(sub, aggCtx, values); } else { final SortedNumericDoubleValues aggregateSums = valuesSource.getAggregateMetricValues(context, Metric.sum); final SortedNumericDoubleValues aggregateValueCounts = valuesSource.getAggregateMetricValues(context, Metric.value_count); @@ -168,7 +140,7 @@ public boolean isNoop() { Long timestamp = entry.getKey(); AggregatorFunction function = entry.getValue(); if (aggCtx.getTimestamp() + downsampleRange >= timestamp) { - ((AvgFunction) function).collectExact(sum, valueCount); + function.collect(new Tuple<>(sum, valueCount)); } else { break; } @@ -177,6 +149,14 @@ public boolean isNoop() { } } + private void rewriteFunction() { + if (downsampleFunction == Function.count_over_time) { + downsampleFunction = Function.count_exact_over_time; + } else if (downsampleFunction == Function.avg_over_time) { + downsampleFunction = Function.avg_exact_over_time; + } + } + private Metric getAggregateMetric() { switch (downsampleFunction) { case max_over_time: From 309ea72d6f2d78f316d813d8f00d2847bdc346bf Mon Sep 17 00:00:00 2001 From: weizijun Date: Fri, 27 May 2022 17:07:27 +0800 Subject: [PATCH 34/53] support get origin value --- .../elasticsearch/search/SearchModule.java | 6 +- .../timeseries/aggregation/Aggregator.java | 4 +- .../timeseries/aggregation/Function.java | 7 ++ .../TimeSeriesAggregationAggregator.java | 6 +- .../bucketfunction/TSIDBucketFunction.java | 33 ++++-- .../bucketfunction/TopkBucketFunction.java | 17 +-- .../function/OriginValuesFunction.java | 41 +++++++ .../internal/TimeSeriesOriginValues.java | 100 ++++++++++++++++++ 8 files changed, 194 insertions(+), 20 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/OriginValuesFunction.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesOriginValues.java diff --git a/server/src/main/java/org/elasticsearch/search/SearchModule.java b/server/src/main/java/org/elasticsearch/search/SearchModule.java index 237cd10c5310d..4434403fd119b 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchModule.java +++ b/server/src/main/java/org/elasticsearch/search/SearchModule.java @@ -217,11 +217,12 @@ import org.elasticsearch.search.aggregations.timeseries.TimeSeriesAggregationBuilder; import org.elasticsearch.search.aggregations.timeseries.aggregation.InternalTimeSeriesAggregation; import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregationAggregationBuilder; -import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesCountValues; -import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesLast; import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TSIDInternalAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesCountValues; import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesIRate; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesLast; import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesLineAggreagation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesOriginValues; import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesRate; import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesTopk; import org.elasticsearch.search.fetch.FetchPhase; @@ -687,6 +688,7 @@ private ValuesSourceRegistry registerAggregations(List plugins) { .addResultReader(TimeSeriesLineAggreagation.NAME, TimeSeriesLineAggreagation::new) .addResultReader(TimeSeriesTopk.NAME, TimeSeriesTopk::new) .addResultReader(TimeSeriesCountValues.NAME, TimeSeriesCountValues::new) + .addResultReader(TimeSeriesOriginValues.NAME, TimeSeriesOriginValues::new) .setAggregatorRegistrar(TimeSeriesAggregationAggregationBuilder::registerAggregators), builder ); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java index 1b7dd400f2b64..cd5b93859a413 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java @@ -102,7 +102,7 @@ public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArra if (aggregatorParams != null && aggregatorParams.containsKey("size")) { size = ((Double) aggregatorParams.get("size")).intValue(); } - return new TopkBucketFunction(size, true); + return new TopkBucketFunction(bigArrays, size, true); } }, bottomk { @@ -121,7 +121,7 @@ public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArra if (aggregatorParams != null && aggregatorParams.containsKey("size")) { size = ((Double) aggregatorParams.get("size")).intValue(); } - return new TopkBucketFunction(size, false); + return new TopkBucketFunction(bigArrays, size, false); } }, count_values { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java index 0c9eadf9bd6f6..90b1ebdea97b4 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java @@ -19,6 +19,7 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.function.LastFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MaxFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.MinFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.OriginValuesFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.QuantileFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.RateFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.SumFunction; @@ -79,6 +80,12 @@ public enum Function { return new LastFunction(value -> value); } }, + origin_value { + @Override + public AggregatorFunction getFunction(Map params) { + return new OriginValuesFunction(); + } + }, rate { @Override public AggregatorFunction getFunction(Map params) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index c323f92a05be2..14e8500499a5b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -128,7 +128,9 @@ public TimeSeriesAggregationAggregator( this.aggregatorParams = aggregatorParams; this.needAggregator = this.aggregator != null; this.downsampleRange = downsample != null && downsample.getRange() != null ? downsample.getRange().estimateMillis() : -1; - this.downsampleFunction = downsample != null ? downsample.getFunction() : Function.last; + this.downsampleFunction = downsample != null + ? downsample.getFunction() + : (downsampleRange > 0 ? Function.origin_value : Function.last); if (this.downsampleRange <= 0) { this.downsampleRange = this.interval; } @@ -434,7 +436,7 @@ public void collectTimeSeriesValues(long bucketOrd) throws IOException { AggregatorBucketFunction aggregatorBucketFunction = aggregatorCollectors.get(bucketOrd); if (aggregatorBucketFunction == null) { AggregatorBucketFunction internal = aggregator.getAggregatorBucketFunction(bigArrays(), aggregatorParams); - aggregatorBucketFunction = new TSIDBucketFunction(internal); + aggregatorBucketFunction = new TSIDBucketFunction(bigArrays(), internal); aggregatorCollectors.put(bucketOrd, aggregatorBucketFunction); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java index e5fa3cd8bbbf1..43fd0aa322517 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java @@ -9,6 +9,9 @@ package org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.ObjectArray; +import org.elasticsearch.core.Releasables; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.timeseries.aggregation.TSIDValue; @@ -33,11 +36,14 @@ */ @SuppressWarnings({ "unchecked", "rawtypes" }) public class TSIDBucketFunction implements AggregatorBucketFunction { - private Map> values = new HashMap<>(); + private final BigArrays bigArrays; + private ObjectArray> values; private final AggregatorBucketFunction aggregatorBucketFunction; - public TSIDBucketFunction(AggregatorBucketFunction aggregatorBucketFunction) { + public TSIDBucketFunction(BigArrays bigArrays, AggregatorBucketFunction aggregatorBucketFunction) { this.aggregatorBucketFunction = aggregatorBucketFunction; + this.bigArrays = bigArrays; + values = bigArrays.newObjectArray(1); } @Override @@ -47,29 +53,40 @@ public String name() { @Override public void collect(TSIDValue tsidValue, long bucket) { + values = bigArrays.grow(values, bucket + 1); if (tsidValue.value instanceof InternalAggregation) { Map tsidValues = values.get(bucket); if (tsidValues == null) { tsidValues = new HashMap<>(); - values.put(bucket, tsidValues); + values.set(bucket, tsidValues); } tsidValues.put(tsidValue.tsid, (InternalAggregation) tsidValue.value); } else if (aggregatorBucketFunction instanceof TopkBucketFunction) { aggregatorBucketFunction.collect(tsidValue, bucket); - } else { + } else if (tsidValue.value instanceof Double) { aggregatorBucketFunction.collect(tsidValue.value, bucket); + } else { + throw new UnsupportedOperationException( + "aggregator [" + aggregatorBucketFunction.name() + "] unsupported collect non-double value" + ); } } @Override public void close() { - aggregatorBucketFunction.close(); + Releasables.close(values); } @Override - public InternalAggregation getAggregation(long bucket, Map aggregatorParams, DocValueFormat formatter, Map metadata) { - if (values.containsKey(bucket)) { - return new TSIDInternalAggregation(name(), values.get(bucket), aggregatorBucketFunction.name(), aggregatorParams, formatter, metadata); + public InternalAggregation getAggregation( + long bucket, + Map aggregatorParams, + DocValueFormat formatter, + Map metadata + ) { + Map value = values.get(bucket); + if (value != null) { + return new TSIDInternalAggregation(name(), value, aggregatorBucketFunction.name(), aggregatorParams, formatter, metadata); } else { return aggregatorBucketFunction.getAggregation(bucket, aggregatorParams, formatter, metadata); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TopkBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TopkBucketFunction.java index ba9d4bc4c2a60..a5f2444827555 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TopkBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TopkBucketFunction.java @@ -9,6 +9,9 @@ package org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction; import org.apache.lucene.util.PriorityQueue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.ObjectArray; +import org.elasticsearch.core.Releasables; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.timeseries.aggregation.TSIDValue; @@ -16,18 +19,19 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesTopk; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; public class TopkBucketFunction implements AggregatorBucketFunction> { - private Map>> values; + private final BigArrays bigArrays; + private ObjectArray>> values; private final int topkSize; private final boolean isTop; - public TopkBucketFunction(int size, boolean isTop) { - values = new HashMap<>(); + public TopkBucketFunction(BigArrays bigArrays, int size, boolean isTop) { + this.bigArrays = bigArrays; + values = bigArrays.newObjectArray(1); this.topkSize = size; this.isTop = isTop; } @@ -39,10 +43,11 @@ public String name() { @Override public void collect(TSIDValue number, long bucket) { + values = bigArrays.grow(values, bucket + 1); PriorityQueue> queue = values.get(bucket); if (queue == null) { queue = TopkFunction.getTopkQueue(topkSize, isTop); - values.put(bucket, queue); + values.set(bucket, queue); } queue.insertWithOverflow(number); @@ -65,6 +70,6 @@ public InternalAggregation getAggregation( @Override public void close() { - values = null; + Releasables.close(values); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/OriginValuesFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/OriginValuesFunction.java new file mode 100644 index 0000000000000..acf2c64e761f9 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/OriginValuesFunction.java @@ -0,0 +1,41 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.function; + +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesOriginValues; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class OriginValuesFunction implements AggregatorFunction> { + private List timePoints; + + public OriginValuesFunction() { + this.timePoints = new ArrayList<>(); + } + + @Override + public void collect(TimePoint value) { + this.timePoints.add(value); + } + + @Override + public List get() { + return timePoints; + } + + @Override + public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { + return new TimeSeriesOriginValues(TimeSeriesOriginValues.NAME, timePoints, formatter, metadata); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesOriginValues.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesOriginValues.java new file mode 100644 index 0000000000000..ffd38d1dd25df --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesOriginValues.java @@ -0,0 +1,100 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation.internal; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.AggregationReduceContext; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class TimeSeriesOriginValues extends InternalAggregation implements Comparable { + public static final String NAME = "time_series_origin_values"; + + private final List timePoints; + private final DocValueFormat formatter; + + public TimeSeriesOriginValues(String name, List timePoints, DocValueFormat formatter, Map metadata) { + super(name, metadata); + this.timePoints = timePoints; + this.formatter = formatter; + } + + public TimeSeriesOriginValues(StreamInput in) throws IOException { + super(in); + formatter = in.readNamedWriteable(DocValueFormat.class); + timePoints = in.readList(TimePoint::new); + } + + public List getTimePoints() { + return timePoints; + } + + @Override + public String getWriteableName() { + return NAME; + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + out.writeNamedWriteable(formatter); + out.writeList(timePoints); + } + + @Override + public InternalAggregation reduce(List aggregations, AggregationReduceContext reduceContext) { + List allTimePoints = new ArrayList<>(); + + List timeSeriesOriginValues = aggregations.stream() + .map(c -> (TimeSeriesOriginValues) c) + .sorted() + .collect(Collectors.toList()); + timeSeriesOriginValues.stream().forEach(v -> allTimePoints.addAll(v.timePoints)); + return new TimeSeriesOriginValues(name, allTimePoints, formatter, getMetadata()); + } + + @Override + protected boolean mustReduceOnSingleInternalAgg() { + return false; + } + + @Override + public Object getProperty(List path) { + return null; + } + + @Override + public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + builder.field(CommonFields.VALUE.getPreferredName()); + builder.startObject(); + for (TimePoint timePoint : timePoints) { + builder.field(String.valueOf(timePoint.getTimestamp()), timePoint.getValue()); + } + builder.endObject(); + return builder; + } + + @Override + public int compareTo(TimeSeriesOriginValues o) { + if (timePoints.size() == 0) { + return -1; + } else if (o.timePoints.size() == 0) { + return 1; + } + return (int) (timePoints.get(0).getTimestamp() - o.timePoints.get(0).getTimestamp()); + } +} From 15fc22ea30508eb4f02f1076caf4e02a676b4913 Mon Sep 17 00:00:00 2001 From: weizijun Date: Fri, 27 May 2022 17:30:16 +0800 Subject: [PATCH 35/53] revert --- .../search/aggregations/metrics/InternalCardinality.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalCardinality.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalCardinality.java index 078204d242922..50cd873dd8947 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalCardinality.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalCardinality.java @@ -23,7 +23,7 @@ public final class InternalCardinality extends InternalNumericMetricsAggregation.SingleValue implements Cardinality { private final AbstractHyperLogLogPlusPlus counts; - public InternalCardinality(String name, AbstractHyperLogLogPlusPlus counts, Map metadata) { + InternalCardinality(String name, AbstractHyperLogLogPlusPlus counts, Map metadata) { super(name, null, metadata); this.counts = counts; } From 4cd2be4816232a4f3b5f8302247de77e7fc67f0f Mon Sep 17 00:00:00 2001 From: weizijun Date: Mon, 30 May 2022 09:58:02 +0800 Subject: [PATCH 36/53] function add value type --- .../timeseries/aggregation/Function.java | 107 ++++++++++-------- 1 file changed, 61 insertions(+), 46 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java index 90b1ebdea97b4..309fa93ef9ef9 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java @@ -32,205 +32,205 @@ import java.util.Map; public enum Function { - count_over_time { + count_over_time(ValueType.matrix) { @Override public AggregatorFunction getFunction(Map params) { return new ValueCountFunction(); } }, - count_exact_over_time { + count_exact_over_time(ValueType.matrix) { @Override public AggregatorFunction getFunction(Map params) { return new ValueCountExactFunction(); } }, - sum_over_time { + sum_over_time(ValueType.matrix) { @Override public AggregatorFunction getFunction(Map params) { return new SumFunction(); } }, - min_over_time { + min_over_time(ValueType.matrix) { @Override public AggregatorFunction getFunction(Map params) { return new MinFunction(); } }, - max_over_time { + max_over_time(ValueType.matrix) { @Override public AggregatorFunction getFunction(Map params) { return new MaxFunction(); } }, - avg_over_time { + avg_over_time(ValueType.matrix) { @Override public AggregatorFunction getFunction(Map params) { return new AvgFunction(); } }, - avg_exact_over_time { + avg_exact_over_time(ValueType.matrix) { @Override public AggregatorFunction getFunction(Map params) { return new AvgExactFunction(); } }, - last { + last(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new LastFunction(value -> value); } }, - origin_value { + origin_value(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new OriginValuesFunction(); } }, - rate { + rate(ValueType.matrix) { @Override public AggregatorFunction getFunction(Map params) { return new RateFunction((long) params.get(RANGE_FIELD), (long) params.get(ROUNDING_FIELD), true, true); } }, - delta { + delta(ValueType.matrix) { @Override public AggregatorFunction getFunction(Map params) { return new RateFunction((long) params.get(RANGE_FIELD), (long) params.get(ROUNDING_FIELD), false, false); } }, - increase { + increase(ValueType.matrix) { @Override public AggregatorFunction getFunction(Map params) { return new RateFunction((long) params.get(RANGE_FIELD), (long) params.get(ROUNDING_FIELD), true, false); } }, - irate { + irate(ValueType.matrix) { @Override public AggregatorFunction getFunction(Map params) { return new IRateFunction(true); } }, - idelta { + idelta(ValueType.matrix) { @Override public AggregatorFunction getFunction(Map params) { return new IRateFunction(true); } }, - abs { + abs(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::abs); } }, - ceil { + ceil(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::ceil); } }, - floor { + floor(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::floor); } }, - exp { + exp(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::exp); } }, - sqrt { + sqrt(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::sqrt); } }, - ln { + ln(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::log); } }, - log10 { + log10(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::log10); } }, - sin { + sin(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::sin); } }, - cos { + cos(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::cos); } }, - tan { + tan(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::tan); } }, - asin { + asin(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::asin); } }, - acos { + acos(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::acos); } }, - atan { + atan(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::atan); } }, - sinh { + sinh(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::sinh); } }, - cosh { + cosh(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::cosh); } }, - tanh { + tanh(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new LastFunction(Math::tanh); } }, - rad { + rad(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new LastFunction(value -> value * Math.PI / 180); } }, - deg { + deg(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new LastFunction(value -> value * 180 / Math.PI); } }, - pi { + pi(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new LastFunction(value -> Math.PI); } }, - sgn { + sgn(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new LastFunction(v -> { @@ -243,19 +243,19 @@ public enum Function { }); } }, - timestamp { + timestamp(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new TimestampFunction(); } }, - day_of_month { + day_of_month(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new DateFunction(ZonedDateTime::getDayOfMonth); } }, - day_of_week { + day_of_week(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new DateFunction(zonedDateTime -> { @@ -264,31 +264,31 @@ public enum Function { }); } }, - hour { + hour(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new DateFunction(ZonedDateTime::getHour); } }, - minute { + minute(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new DateFunction(ZonedDateTime::getMinute); } }, - month { + month(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new DateFunction(ZonedDateTime::getMonthValue); } }, - year { + year(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { return new DateFunction(ZonedDateTime::getMonthValue); } }, - clamp { + clamp(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { double max = Double.MAX_VALUE; @@ -302,7 +302,7 @@ public enum Function { return new ClampFunction(max, min); } }, - clamp_max { + clamp_max(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { double max = Double.MAX_VALUE; @@ -312,7 +312,7 @@ public enum Function { return new ClampMaxFunction(max); } }, - clamp_min { + clamp_min(ValueType.vector) { @Override public AggregatorFunction getFunction(Map params) { double min = Double.MIN_VALUE; @@ -322,7 +322,7 @@ public enum Function { return new ClampMinFunction(min); } }, - quantile_over_time { + quantile_over_time(ValueType.matrix) { @Override public AggregatorFunction getFunction(Map params) { double quantile = 1f; @@ -332,7 +332,7 @@ public enum Function { return new QuantileFunction(quantile); } }, - last_over_time { + last_over_time(ValueType.matrix) { @Override public AggregatorFunction getFunction(Map params) { return new LastFunction(value -> value); @@ -345,6 +345,16 @@ public enum Function { public static final String MIN_FIELD = "min"; public static final String QUANTILE_FIELD = "quantile"; + private ValueType valueType; + + Function(ValueType valueType) { + this.valueType = valueType; + } + + public ValueType getValueType() { + return valueType; + } + public static Function resolve(String name) { try { return Function.valueOf(name); @@ -357,4 +367,9 @@ public static Function resolve(String name) { * get the function */ public abstract AggregatorFunction getFunction(Map params); + + public enum ValueType { + vector, + matrix + } } From aad754d102e8eaaaac27e1673b4739ad39f7ba87 Mon Sep 17 00:00:00 2001 From: weizijun Date: Tue, 31 May 2022 10:54:26 +0800 Subject: [PATCH 37/53] improve rounding --- .../timeseries/aggregation/Downsample.java | 10 ++-- .../aggregation/RoundingInterval.java | 37 +++++++++++++++ ...meSeriesAggregationAggregationBuilder.java | 47 +++++++++++++++++-- ...meSeriesAggregationAggregationFactory.java | 10 ++++ .../TimeSeriesAggregationAggregator.java | 14 ++++-- ...meSeriesAggregationAggregatorSupplier.java | 2 + .../aggregation/RoundingIntervalTests.java | 35 ++++++++++++++ ...MetricTimeSeriesAggregationAggregator.java | 8 ++++ 8 files changed, 150 insertions(+), 13 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingInterval.java create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingIntervalTests.java diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java index 20cdb09e0c264..0dfc367aeaf25 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java @@ -44,7 +44,7 @@ public class Downsample implements ToXContentObject, Writeable { ObjectParser.ValueType.STRING ); PARSER.declareField( - ConstructingObjectParser.constructorArg(), + ConstructingObjectParser.optionalConstructorArg(), p -> Function.resolve(p.text()), FUNCTION_FIELD, ObjectParser.ValueType.STRING @@ -64,7 +64,7 @@ public Downsample(DateHistogramInterval range, Function function, Map parser.map(), AGGREGATOR_PARAMS_FIELD); + PARSER.declareObject( + TimeSeriesAggregationAggregationBuilder::aggregatorParams, + (parser, c) -> parser.map(), + AGGREGATOR_PARAMS_FIELD + ); PARSER.declareObject(TimeSeriesAggregationAggregationBuilder::downsample, (p, c) -> Downsample.fromXContent(p), DOWNSAMPLE_FIELD); PARSER.declareObjectArray( TimeSeriesAggregationAggregationBuilder::order, @@ -114,6 +121,8 @@ public class TimeSeriesAggregationAggregationBuilder extends ValuesSourceAggrega PARSER.declareInt(TimeSeriesAggregationAggregationBuilder::shardSize, SHARD_SIZE_FIELD); PARSER.declareInt(TimeSeriesAggregationAggregationBuilder::minDocCount, MIN_DOC_COUNT_FIELD); PARSER.declareInt(TimeSeriesAggregationAggregationBuilder::shardMinDocCount, SHARD_MIN_DOC_COUNT_FIELD); + PARSER.declareLong(TimeSeriesAggregationAggregationBuilder::startTime, START_TIME_FIELD); + PARSER.declareLong(TimeSeriesAggregationAggregationBuilder::endTime, END_TIME_FIELD); } public TimeSeriesAggregationAggregationBuilder(String name) { @@ -136,6 +145,8 @@ protected TimeSeriesAggregationAggregationBuilder( this.downsample = clone.downsample; this.order = clone.order; this.bucketCountThresholds = clone.bucketCountThresholds; + this.startTime = clone.startTime; + this.endTime = clone.endTime; } public TimeSeriesAggregationAggregationBuilder(StreamInput in) throws IOException { @@ -150,6 +161,8 @@ public TimeSeriesAggregationAggregationBuilder(StreamInput in) throws IOExceptio downsample = in.readOptionalWriteable(Downsample::new); order = InternalOrder.Streams.readOrder(in); bucketCountThresholds = new TermsAggregator.BucketCountThresholds(in); + startTime = in.readOptionalLong(); + endTime = in.readOptionalLong(); } @Override @@ -164,6 +177,8 @@ protected void innerWriteTo(StreamOutput out) throws IOException { out.writeOptionalWriteable(downsample); order.writeTo(out); bucketCountThresholds.writeTo(out); + out.writeOptionalLong(startTime); + out.writeOptionalLong(endTime); } @Override @@ -200,6 +215,8 @@ protected ValuesSourceAggregatorFactory innerBuild( downsample, bucketCountThresholds, order, + startTime, + endTime, config, context, parent, @@ -498,6 +515,24 @@ public TimeSeriesAggregationAggregationBuilder downsample(DateHistogramInterval return this; } + public long startTime() { + return startTime; + } + + public TimeSeriesAggregationAggregationBuilder startTime(long startTime) { + this.startTime = startTime; + return this; + } + + public long endTime() { + return endTime; + } + + public TimeSeriesAggregationAggregationBuilder endTime(long endTime) { + this.endTime = endTime; + return this; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -518,7 +553,9 @@ public boolean equals(Object o) { && Objects.equals(aggregator, that.aggregator) && Objects.equals(downsample, that.downsample) && Objects.equals(bucketCountThresholds, that.bucketCountThresholds) - && Objects.equals(order, that.order); + && Objects.equals(order, that.order) + && Objects.equals(startTime, that.startTime) + && Objects.equals(endTime, that.endTime); } @Override @@ -533,7 +570,9 @@ public int hashCode() { aggregator, downsample, bucketCountThresholds, - order + order, + startTime, + endTime ); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java index 8f95e91206787..25bef1563e214 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java @@ -38,6 +38,8 @@ public class TimeSeriesAggregationAggregationFactory extends ValuesSourceAggrega private final Map aggregatorParams; private final Downsample downsample; private final TermsAggregator.BucketCountThresholds bucketCountThresholds; + private final long startTime; + private final long endTime; private final BucketOrder order; private final ValuesSourceConfig config; private final TimeSeriesAggregationAggregatorSupplier aggregatorSupplier; @@ -54,6 +56,8 @@ public TimeSeriesAggregationAggregationFactory( Downsample downsample, TermsAggregator.BucketCountThresholds bucketCountThresholds, BucketOrder order, + long startTime, + long endTime, ValuesSourceConfig config, AggregationContext context, AggregatorFactory parent, @@ -71,6 +75,8 @@ public TimeSeriesAggregationAggregationFactory( this.aggregatorParams = aggregatorParams; this.downsample = downsample; this.bucketCountThresholds = bucketCountThresholds; + this.startTime = startTime; + this.endTime = endTime; this.order = order; this.config = config; this.aggregatorSupplier = aggregatorSupplier; @@ -109,6 +115,8 @@ protected Aggregator createUnmapped(Aggregator parent, Map metad downsample, thresholds, order, + startTime, + endTime, config, context, parent, @@ -142,6 +150,8 @@ protected Aggregator doCreateInternal(Aggregator parent, CardinalityUpperBound c downsample, thresholds, order, + startTime, + endTime, config, context, parent, diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index 14e8500499a5b..c655326e8ac6c 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -14,11 +14,9 @@ import org.apache.lucene.search.Scorable; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.PriorityQueue; -import org.elasticsearch.common.Rounding; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.Releasables; -import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.TimestampBounds; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; import org.elasticsearch.index.mapper.TimeSeriesIdFieldMapper; @@ -84,11 +82,13 @@ public class TimeSeriesAggregationAggregator extends BucketsAggregator { protected Comparator partiallyBuiltBucketComparator; protected DocValueFormat format; private TimestampBounds timestampBounds; + private long startTime; + private long endTime; private BytesRef preTsid; private long preBucketOrdinal; protected long preRounding = -1; - private Rounding.Prepared rounding; + private RoundingInterval rounding; private boolean needAggregator; protected Map timeBucketMetrics; // TODO replace map private Map> groupBucketValues; // TODO replace map @@ -108,6 +108,8 @@ public TimeSeriesAggregationAggregator( Downsample downsample, TermsAggregator.BucketCountThresholds bucketCountThresholds, BucketOrder order, + long startTime, + long endTime, ValuesSourceConfig valuesSourceConfig, AggregationContext context, org.elasticsearch.search.aggregations.Aggregator parent, @@ -122,13 +124,15 @@ public TimeSeriesAggregationAggregator( if (this.interval <= 0) { throw new IllegalArgumentException("time_series_aggregation invalid interval [" + interval + "]"); } - this.rounding = Rounding.builder(new TimeValue(this.interval)).build().prepareForUnknown(); + this.startTime = startTime; + this.endTime = endTime; + this.rounding = new RoundingInterval(this.startTime, this.interval); this.offset = offset != null ? offset.estimateMillis() : 0; this.aggregator = aggregator; this.aggregatorParams = aggregatorParams; this.needAggregator = this.aggregator != null; this.downsampleRange = downsample != null && downsample.getRange() != null ? downsample.getRange().estimateMillis() : -1; - this.downsampleFunction = downsample != null + this.downsampleFunction = downsample != null && downsample.getFunction() != null ? downsample.getFunction() : (downsampleRange > 0 ? Function.origin_value : Function.last); if (this.downsampleRange <= 0) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java index 4f455b214f66b..fb1e28cc9a38f 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java @@ -36,6 +36,8 @@ Aggregator build( Downsample downsample, TermsAggregator.BucketCountThresholds bucketCountThresholds, BucketOrder order, + long startTime, + long endTime, ValuesSourceConfig valuesSourceConfig, AggregationContext context, Aggregator parent, diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingIntervalTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingIntervalTests.java new file mode 100644 index 0000000000000..c33903d11ef7d --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingIntervalTests.java @@ -0,0 +1,35 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation; + +import org.elasticsearch.test.ESTestCase; + +import static org.hamcrest.Matchers.equalTo; + +public class RoundingIntervalTests extends ESTestCase { + public void testRoundingByStartTime() { + long startTime = 111; + long interval = 10; + RoundingInterval rounding = new RoundingInterval(startTime, interval); + assertThat(rounding.nextRoundingValue(113), equalTo(121L)); + assertThat(rounding.nextRoundingValue(120), equalTo(121L)); + assertThat(rounding.nextRoundingValue(121), equalTo(121L)); + assertThat(rounding.nextRoundingValue(122), equalTo(131L)); + } + + public void testRoundingByRounding() { + long startTime = 0; + long interval = 10; + RoundingInterval rounding = new RoundingInterval(startTime, interval); + assertThat(rounding.nextRoundingValue(113), equalTo(120L)); + assertThat(rounding.nextRoundingValue(119), equalTo(120L)); + assertThat(rounding.nextRoundingValue(120), equalTo(120L)); + assertThat(rounding.nextRoundingValue(121), equalTo(130L)); + } +} diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java index 33320ca354821..8495189633781 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java @@ -51,6 +51,8 @@ public AggregateMetricTimeSeriesAggregationAggregator( BucketCountThresholds bucketCountThresholds, BucketOrder order, ValuesSourceConfig valuesSourceConfig, + long startTime, + long endTime, AggregationContext context, Aggregator parent, CardinalityUpperBound bucketCardinality, @@ -69,6 +71,8 @@ public AggregateMetricTimeSeriesAggregationAggregator( downsample, bucketCountThresholds, order, + startTime, + endTime, null, context, parent, @@ -114,6 +118,10 @@ public boolean isNoop() { final SortedNumericDoubleValues aggregateSums = valuesSource.getAggregateMetricValues(context, Metric.sum); final SortedNumericDoubleValues aggregateValueCounts = valuesSource.getAggregateMetricValues(context, Metric.value_count); return new Collector(sub, aggregateSums, aggCtx, (doc) -> { + if (aggCtx.getTimestamp() + downsampleRange < preRounding) { + return; + } + double sum = 0; long valueCount = 0; if (aggregateSums.advanceExact(doc)) { From 64319346168ef76777180487169862f366960178 Mon Sep 17 00:00:00 2001 From: weizijun Date: Tue, 31 May 2022 11:07:52 +0800 Subject: [PATCH 38/53] improve rounding --- .../metrics/AggregateMetricTimeSeriesAggregationAggregator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java index 8495189633781..7ded9cbf7c09b 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java @@ -50,9 +50,9 @@ public AggregateMetricTimeSeriesAggregationAggregator( Downsample downsample, BucketCountThresholds bucketCountThresholds, BucketOrder order, - ValuesSourceConfig valuesSourceConfig, long startTime, long endTime, + ValuesSourceConfig valuesSourceConfig, AggregationContext context, Aggregator parent, CardinalityUpperBound bucketCardinality, From 7a26c135ad9f666d1524f9d90b871c7d081c6c82 Mon Sep 17 00:00:00 2001 From: weizijun Date: Tue, 31 May 2022 11:18:11 +0800 Subject: [PATCH 39/53] improve rounding --- .../aggregations/timeseries/aggregation/RoundingInterval.java | 4 ++-- .../aggregation/TimeSeriesAggregationAggregator.java | 2 +- .../timeseries/aggregation/RoundingIntervalTests.java | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingInterval.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingInterval.java index b92f4ef4f7120..1f0db5e8073e9 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingInterval.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingInterval.java @@ -28,9 +28,9 @@ public RoundingInterval(long startTime, long interval) { public long nextRoundingValue(long utcMillis) { if (rounding != null) { - return rounding.nextRoundingValue(utcMillis-1); + return rounding.nextRoundingValue(utcMillis); } else { - long step = (utcMillis - 1 - startTime) / interval + 1; + long step = (utcMillis - startTime) / interval + 1; return startTime + interval * step; } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index c655326e8ac6c..8947928955239 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -288,7 +288,7 @@ public void collect(int doc, long bucket) throws IOException { reset(newTsid, bucket); } - if (preRounding < 0 || aggCtx.getTimestamp() <= preRounding - interval) { + if (preRounding < 0 || aggCtx.getTimestamp() < preRounding - interval) { preRounding = rounding.nextRoundingValue(aggCtx.getTimestamp()); } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingIntervalTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingIntervalTests.java index c33903d11ef7d..fded10a67b1f6 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingIntervalTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingIntervalTests.java @@ -19,7 +19,7 @@ public void testRoundingByStartTime() { RoundingInterval rounding = new RoundingInterval(startTime, interval); assertThat(rounding.nextRoundingValue(113), equalTo(121L)); assertThat(rounding.nextRoundingValue(120), equalTo(121L)); - assertThat(rounding.nextRoundingValue(121), equalTo(121L)); + assertThat(rounding.nextRoundingValue(121), equalTo(131L)); assertThat(rounding.nextRoundingValue(122), equalTo(131L)); } @@ -29,7 +29,7 @@ public void testRoundingByRounding() { RoundingInterval rounding = new RoundingInterval(startTime, interval); assertThat(rounding.nextRoundingValue(113), equalTo(120L)); assertThat(rounding.nextRoundingValue(119), equalTo(120L)); - assertThat(rounding.nextRoundingValue(120), equalTo(120L)); + assertThat(rounding.nextRoundingValue(120), equalTo(130L)); assertThat(rounding.nextRoundingValue(121), equalTo(130L)); } } From 15da69f2a676f87b1529a96a38d897a150e119f5 Mon Sep 17 00:00:00 2001 From: weizijun Date: Tue, 31 May 2022 11:36:26 +0800 Subject: [PATCH 40/53] improve rounding --- .../TimeSeriesAggregationAggregationBuilder.java | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java index b56a0cfccf92f..4fdc97ec5e28a 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java @@ -82,8 +82,8 @@ public class TimeSeriesAggregationAggregationBuilder extends ValuesSourceAggrega DEFAULT_BUCKET_COUNT_THRESHOLDS ); private BucketOrder order = BucketOrder.key(true); - private long startTime; - private long endTime; + private Long startTime; + private Long endTime; static { ValuesSourceAggregationBuilder.declareFields(PARSER, false, true, false); @@ -253,6 +253,12 @@ protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) bucketCountThresholds.toXContent(builder, params); builder.field(ORDER_FIELD.getPreferredName()); order.toXContent(builder, params); + if (startTime != null) { + builder.field(START_TIME_FIELD.getPreferredName(), startTime); + } + if (endTime != null) { + builder.field(END_TIME_FIELD.getPreferredName(), endTime); + } return builder; } @@ -515,7 +521,7 @@ public TimeSeriesAggregationAggregationBuilder downsample(DateHistogramInterval return this; } - public long startTime() { + public Long startTime() { return startTime; } @@ -524,7 +530,7 @@ public TimeSeriesAggregationAggregationBuilder startTime(long startTime) { return this; } - public long endTime() { + public Long endTime() { return endTime; } From 6b106db14b2697fa4bf492d1fb2c120571a49e70 Mon Sep 17 00:00:00 2001 From: weizijun Date: Tue, 31 May 2022 14:30:53 +0800 Subject: [PATCH 41/53] fix TimeSeriesLineAggreagation NPE --- .../aggregation/TimeSeriesAggregationAggregator.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index 8947928955239..a0578eaba802b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -217,6 +217,9 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I } } else { values = groupBucketValues.get(ord); + if (values == null) { + values = new LinkedHashMap<>(); + } } bucket.metricAggregation = new TimeSeriesLineAggreagation(TimeSeriesLineAggreagation.NAME, values, format, metadata()); allBucketsPerOrd[ordIdx][b] = bucket; From 5d0fbd1a290dc45ca52998b5a214e1d49570e037 Mon Sep 17 00:00:00 2001 From: weizijun Date: Wed, 1 Jun 2022 14:14:42 +0800 Subject: [PATCH 42/53] fix rate and count --- .../aggregation/bucketfunction/AvgBucketFunction.java | 4 ++-- .../bucketfunction/CountValuesBucketFunction.java | 5 +++-- .../aggregation/bucketfunction/MaxBucketFunction.java | 4 ++-- .../aggregation/bucketfunction/MinBucketFunction.java | 4 ++-- .../aggregation/bucketfunction/QuantileBucketFunction.java | 3 ++- .../aggregation/bucketfunction/SumBucketFunction.java | 4 ++-- .../aggregation/bucketfunction/TopkBucketFunction.java | 3 ++- .../bucketfunction/ValueCountBucketFunction.java | 4 ++-- .../timeseries/aggregation/function/RateFunction.java | 6 +++--- .../aggregation/function/ValueCountExactFunction.java | 3 ++- .../timeseries/aggregation/function/ValueCountFunction.java | 3 ++- .../aggregation/internal/TSIDInternalAggregation.java | 2 +- 12 files changed, 25 insertions(+), 20 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AvgBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AvgBucketFunction.java index 50221d6163891..0d5caeb82e1a5 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AvgBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AvgBucketFunction.java @@ -14,7 +14,7 @@ import org.elasticsearch.core.Releasables; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; -import org.elasticsearch.search.aggregations.metrics.AvgAggregationBuilder; +import org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator; import java.util.Map; @@ -33,7 +33,7 @@ public AvgBucketFunction(BigArrays bigArrays) { @Override public String name() { - return AvgAggregationBuilder.NAME; + return Aggregator.avg.name(); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/CountValuesBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/CountValuesBucketFunction.java index 36893ab839507..584a230f680d1 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/CountValuesBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/CountValuesBucketFunction.java @@ -13,6 +13,7 @@ import org.elasticsearch.core.Releasables; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator; import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesCountValues; import java.util.HashMap; @@ -31,7 +32,7 @@ public CountValuesBucketFunction(BigArrays bigArrays) { @Override public String name() { - return "count_values"; + return Aggregator.count_values.name(); } @Override @@ -59,7 +60,7 @@ public InternalAggregation getAggregation( DocValueFormat formatter, Map metadata ) { - return new TimeSeriesCountValues(TimeSeriesCountValues.NAME, values.get(bucket), formatter, metadata); + return new TimeSeriesCountValues(name(), values.get(bucket), formatter, metadata); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MaxBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MaxBucketFunction.java index caa03a0d4bf19..e3e34b564ad70 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MaxBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MaxBucketFunction.java @@ -13,7 +13,7 @@ import org.elasticsearch.core.Releasables; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; -import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder; +import org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator; import java.util.Map; @@ -29,7 +29,7 @@ public MaxBucketFunction(BigArrays bigArrays) { @Override public String name() { - return MaxAggregationBuilder.NAME; + return Aggregator.max.name(); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MinBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MinBucketFunction.java index 32464dc3e5ed7..ef3a7e00ad0e5 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MinBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/MinBucketFunction.java @@ -13,7 +13,7 @@ import org.elasticsearch.core.Releasables; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; -import org.elasticsearch.search.aggregations.metrics.MinAggregationBuilder; +import org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator; import java.util.Map; @@ -29,7 +29,7 @@ public MinBucketFunction(BigArrays bigArrays) { @Override public String name() { - return MinAggregationBuilder.NAME; + return Aggregator.min.name(); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/QuantileBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/QuantileBucketFunction.java index 80e3b307b97ca..0c7c873da773d 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/QuantileBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/QuantileBucketFunction.java @@ -15,6 +15,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.InternalTDigestPercentiles; import org.elasticsearch.search.aggregations.metrics.TDigestState; +import org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator; import java.util.Map; @@ -33,7 +34,7 @@ public QuantileBucketFunction(BigArrays bigArrays, double quantile) { @Override public String name() { - return "quantile"; + return Aggregator.quantile.name(); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/SumBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/SumBucketFunction.java index 3512294cc921d..ba95bc93cf7c1 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/SumBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/SumBucketFunction.java @@ -13,7 +13,7 @@ import org.elasticsearch.core.Releasables; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; -import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder; +import org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator; import java.util.Map; @@ -29,7 +29,7 @@ public SumBucketFunction(BigArrays bigArrays) { @Override public String name() { - return SumAggregationBuilder.NAME; + return Aggregator.sum.name(); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TopkBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TopkBucketFunction.java index a5f2444827555..518021758e32a 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TopkBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TopkBucketFunction.java @@ -14,6 +14,7 @@ import org.elasticsearch.core.Releasables; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator; import org.elasticsearch.search.aggregations.timeseries.aggregation.TSIDValue; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.TopkFunction; import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesTopk; @@ -38,7 +39,7 @@ public TopkBucketFunction(BigArrays bigArrays, int size, boolean isTop) { @Override public String name() { - return "topk"; + return Aggregator.topk.name(); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/ValueCountBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/ValueCountBucketFunction.java index 6016efda38d39..cfdc68ca90d29 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/ValueCountBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/ValueCountBucketFunction.java @@ -14,7 +14,7 @@ import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.InternalValueCount; -import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder; +import org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator; import java.util.Map; @@ -30,7 +30,7 @@ public ValueCountBucketFunction(BigArrays bigArrays) { @Override public String name() { - return ValueCountAggregationBuilder.NAME; + return Aggregator.count.name(); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/RateFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/RateFunction.java index cbb08a385f7a6..f4ddd64bae7d0 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/RateFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/RateFunction.java @@ -102,10 +102,10 @@ public static double extrapolatedRate( resultValue += totalRevertValue; } - double durationToStart = (firstSample.getTimestamp() - rangeStart) / 1000; - double durationToEnd = (rangeEnd - lastSample.getTimestamp()) / 1000; + double durationToStart = (double) (firstSample.getTimestamp() - rangeStart) / 1000; + double durationToEnd = (double) (rangeEnd - lastSample.getTimestamp()) / 1000; - double sampledInterval = (lastSample.getTimestamp() - firstSample.getTimestamp()) / 1000; + double sampledInterval = (double) (lastSample.getTimestamp() - firstSample.getTimestamp()) / 1000; double averageDurationBetweenSamples = sampledInterval / (count - 1); if (isCounter && resultValue > 0 && firstSample.getValue() >= 0) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountExactFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountExactFunction.java index 4f9f9703325d0..31b1f900a4549 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountExactFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountExactFunction.java @@ -11,6 +11,7 @@ import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder; +import org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator; import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; import java.util.Map; @@ -30,6 +31,6 @@ public Long get() { @Override public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { - return new org.elasticsearch.search.aggregations.metrics.InternalValueCount(ValueCountAggregationBuilder.NAME, count, metadata); + return new org.elasticsearch.search.aggregations.metrics.InternalValueCount(Aggregator.count.name(), count, metadata); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountFunction.java index dbe3aa8a99e0d..66048a4fcee5d 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountFunction.java @@ -11,6 +11,7 @@ import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder; +import org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator; import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; import java.util.Map; @@ -30,6 +31,6 @@ public Long get() { @Override public InternalAggregation getAggregation(DocValueFormat formatter, Map metadata) { - return new org.elasticsearch.search.aggregations.metrics.InternalValueCount(ValueCountAggregationBuilder.NAME, count, metadata); + return new org.elasticsearch.search.aggregations.metrics.InternalValueCount(Aggregator.count.name(), count, metadata); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java index 474a47366a3c9..b1d05e3b2555b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java @@ -84,7 +84,7 @@ public InternalAggregation reduce(List aggregations, Aggreg if (aggregations.size() == 1) { TSIDInternalAggregation tsidAgg = (TSIDInternalAggregation) aggregations.get(0); if (reduceContext.isFinalReduce()) { - Aggregator function = Aggregator.valueOf(aggregator); + Aggregator function = Aggregator.resolve(aggregator); final AggregatorFunction aggregatorFunction = function.getAggregatorFunction(aggregatorParams); tsidAgg.values.forEach( (tsid, agg) -> { From 8e3956a50ff0e83c25a0d5a312809b5d7da4ef69 Mon Sep 17 00:00:00 2001 From: weizijun Date: Wed, 1 Jun 2022 14:20:47 +0800 Subject: [PATCH 43/53] fix start_time and end_time --- .../TimeSeriesAggregationAggregationBuilder.java | 16 ++++++---------- 1 file changed, 6 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java index 4fdc97ec5e28a..8fc1fb9818ee2 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java @@ -82,8 +82,8 @@ public class TimeSeriesAggregationAggregationBuilder extends ValuesSourceAggrega DEFAULT_BUCKET_COUNT_THRESHOLDS ); private BucketOrder order = BucketOrder.key(true); - private Long startTime; - private Long endTime; + private long startTime; + private long endTime; static { ValuesSourceAggregationBuilder.declareFields(PARSER, false, true, false); @@ -253,12 +253,8 @@ protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) bucketCountThresholds.toXContent(builder, params); builder.field(ORDER_FIELD.getPreferredName()); order.toXContent(builder, params); - if (startTime != null) { - builder.field(START_TIME_FIELD.getPreferredName(), startTime); - } - if (endTime != null) { - builder.field(END_TIME_FIELD.getPreferredName(), endTime); - } + builder.field(START_TIME_FIELD.getPreferredName(), startTime); + builder.field(END_TIME_FIELD.getPreferredName(), endTime); return builder; } @@ -521,7 +517,7 @@ public TimeSeriesAggregationAggregationBuilder downsample(DateHistogramInterval return this; } - public Long startTime() { + public long startTime() { return startTime; } @@ -530,7 +526,7 @@ public TimeSeriesAggregationAggregationBuilder startTime(long startTime) { return this; } - public Long endTime() { + public long endTime() { return endTime; } From 0f79448c195755767d34dc84e2e772a51210744a Mon Sep 17 00:00:00 2001 From: weizijun Date: Thu, 2 Jun 2022 11:45:25 +0800 Subject: [PATCH 44/53] fix memory leak --- ...meSeriesAggregationAggregationBuilder.java | 24 ++++++++++++------- .../bucketfunction/TSIDBucketFunction.java | 1 + 2 files changed, 16 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java index 8fc1fb9818ee2..bc6d54a6d2a21 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java @@ -82,8 +82,8 @@ public class TimeSeriesAggregationAggregationBuilder extends ValuesSourceAggrega DEFAULT_BUCKET_COUNT_THRESHOLDS ); private BucketOrder order = BucketOrder.key(true); - private long startTime; - private long endTime; + private Long startTime; + private Long endTime; static { ValuesSourceAggregationBuilder.declareFields(PARSER, false, true, false); @@ -215,8 +215,8 @@ protected ValuesSourceAggregatorFactory innerBuild( downsample, bucketCountThresholds, order, - startTime, - endTime, + startTime != null ? startTime : -1, + endTime != null ? endTime : -1, config, context, parent, @@ -253,8 +253,12 @@ protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) bucketCountThresholds.toXContent(builder, params); builder.field(ORDER_FIELD.getPreferredName()); order.toXContent(builder, params); - builder.field(START_TIME_FIELD.getPreferredName(), startTime); - builder.field(END_TIME_FIELD.getPreferredName(), endTime); + if (startTime != null) { + builder.field(START_TIME_FIELD.getPreferredName(), startTime); + } + if (endTime != null) { + builder.field(END_TIME_FIELD.getPreferredName(), endTime); + } return builder; } @@ -517,7 +521,7 @@ public TimeSeriesAggregationAggregationBuilder downsample(DateHistogramInterval return this; } - public long startTime() { + public Long startTime() { return startTime; } @@ -526,7 +530,7 @@ public TimeSeriesAggregationAggregationBuilder startTime(long startTime) { return this; } - public long endTime() { + public Long endTime() { return endTime; } @@ -552,7 +556,8 @@ public boolean equals(Object o) { && Objects.equals(without, that.without) && Objects.equals(interval, that.interval) && Objects.equals(offset, that.offset) - && Objects.equals(aggregator, that.aggregator) + && aggregator == that.aggregator + && Objects.equals(aggregatorParams, that.aggregatorParams) && Objects.equals(downsample, that.downsample) && Objects.equals(bucketCountThresholds, that.bucketCountThresholds) && Objects.equals(order, that.order) @@ -570,6 +575,7 @@ public int hashCode() { interval, offset, aggregator, + aggregatorParams, downsample, bucketCountThresholds, order, diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java index 43fd0aa322517..ab14a71b7f075 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.java @@ -74,6 +74,7 @@ public void collect(TSIDValue tsidValue, long bucket) { @Override public void close() { + aggregatorBucketFunction.close(); Releasables.close(values); } From fdebd07e67a6ae4db35c0541407020f72fa5c994 Mon Sep 17 00:00:00 2001 From: weizijun Date: Thu, 9 Jun 2022 19:58:46 +0800 Subject: [PATCH 45/53] fixup --- .../aggregations/timeseries/aggregation/RoundingInterval.java | 4 ++-- .../timeseries/aggregation/function/CountValuesFunction.java | 2 +- .../timeseries/aggregation/RoundingIntervalTests.java | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingInterval.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingInterval.java index 1f0db5e8073e9..e2594dd8b8dba 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingInterval.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingInterval.java @@ -28,9 +28,9 @@ public RoundingInterval(long startTime, long interval) { public long nextRoundingValue(long utcMillis) { if (rounding != null) { - return rounding.nextRoundingValue(utcMillis); + return rounding.nextRoundingValue(utcMillis - 1); } else { - long step = (utcMillis - startTime) / interval + 1; + long step = (utcMillis - 1 - startTime) / interval + 1; return startTime + interval * step; } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/CountValuesFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/CountValuesFunction.java index 9e83c6e904146..bd2a806f4309b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/CountValuesFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/CountValuesFunction.java @@ -28,7 +28,7 @@ public CountValuesFunction() { @Override public void collect(TimePoint value) { long val = java.lang.Double.doubleToLongBits(value.getValue()); - AtomicInteger count = valueCount.get(value); + AtomicInteger count = valueCount.get(val); if (count == null) { count = new AtomicInteger(0); valueCount.put(val, count); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingIntervalTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingIntervalTests.java index fded10a67b1f6..c33903d11ef7d 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingIntervalTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingIntervalTests.java @@ -19,7 +19,7 @@ public void testRoundingByStartTime() { RoundingInterval rounding = new RoundingInterval(startTime, interval); assertThat(rounding.nextRoundingValue(113), equalTo(121L)); assertThat(rounding.nextRoundingValue(120), equalTo(121L)); - assertThat(rounding.nextRoundingValue(121), equalTo(131L)); + assertThat(rounding.nextRoundingValue(121), equalTo(121L)); assertThat(rounding.nextRoundingValue(122), equalTo(131L)); } @@ -29,7 +29,7 @@ public void testRoundingByRounding() { RoundingInterval rounding = new RoundingInterval(startTime, interval); assertThat(rounding.nextRoundingValue(113), equalTo(120L)); assertThat(rounding.nextRoundingValue(119), equalTo(120L)); - assertThat(rounding.nextRoundingValue(120), equalTo(130L)); + assertThat(rounding.nextRoundingValue(120), equalTo(120L)); assertThat(rounding.nextRoundingValue(121), equalTo(130L)); } } From 0113910f0084caa1782480aab7d6017eac017a30 Mon Sep 17 00:00:00 2001 From: weizijun Date: Mon, 13 Jun 2022 18:06:39 +0800 Subject: [PATCH 46/53] memory improve --- .../TimeSeriesAggregationAggregator.java | 24 +++++++++++++------ .../CountValuesBucketFunction.java | 2 +- 2 files changed, 18 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index a0578eaba802b..b5ed8eed8fb87 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -14,6 +14,7 @@ import org.apache.lucene.search.Scorable; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.PriorityQueue; +import org.elasticsearch.common.util.ObjectArray; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.Releasables; @@ -91,8 +92,8 @@ public class TimeSeriesAggregationAggregator extends BucketsAggregator { private RoundingInterval rounding; private boolean needAggregator; protected Map timeBucketMetrics; // TODO replace map - private Map> groupBucketValues; // TODO replace map - private Map aggregatorCollectors; // TODO replace map + private ObjectArray> groupBucketValues; // TODO replace map + private ObjectArray aggregatorCollectors; @SuppressWarnings("unchecked") public TimeSeriesAggregationAggregator( @@ -160,8 +161,8 @@ public TimeSeriesAggregationAggregator( throw new IllegalArgumentException("time_series_aggregation group by must have an aggregator"); } - groupBucketValues = new LinkedHashMap<>(); - aggregatorCollectors = new HashMap<>(); + groupBucketValues = bigArrays().newObjectArray(1); + aggregatorCollectors = bigArrays().newObjectArray(1); } @Override @@ -258,7 +259,14 @@ public InternalAggregation buildEmptyAggregation() { protected void doClose() { Releasables.close(bucketOrds); Releasables.close(timestampOrds); - aggregatorCollectors.forEach((k, v) -> { v.close(); }); + Releasables.close(groupBucketValues); + for (int i = 0; i< aggregatorCollectors.size(); i++) { + AggregatorBucketFunction aggregatorBucketFunction = aggregatorCollectors.get(i); + if (aggregatorBucketFunction != null) { + aggregatorBucketFunction.close(); + } + } + Releasables.close(aggregatorCollectors); } public class Collector extends LeafBucketCollectorBase { @@ -440,11 +448,12 @@ private BytesRef packKey(BytesRef tsid) { */ public void collectTimeSeriesValues(long bucketOrd) throws IOException { if (needAggregator) { + aggregatorCollectors = bigArrays().grow(aggregatorCollectors, bucketOrd + 1); AggregatorBucketFunction aggregatorBucketFunction = aggregatorCollectors.get(bucketOrd); if (aggregatorBucketFunction == null) { AggregatorBucketFunction internal = aggregator.getAggregatorBucketFunction(bigArrays(), aggregatorParams); aggregatorBucketFunction = new TSIDBucketFunction(bigArrays(), internal); - aggregatorCollectors.put(bucketOrd, aggregatorBucketFunction); + aggregatorCollectors.set(bucketOrd, aggregatorBucketFunction); } for (Entry entry : timeBucketMetrics.entrySet()) { @@ -473,7 +482,8 @@ public void collectTimeSeriesValues(long bucketOrd) throws IOException { } else { Map tsids = new LinkedHashMap<>(); timeBucketMetrics.forEach((k, v) -> { tsids.put(k + offset, v.getAggregation(format, metadata())); }); - groupBucketValues.put(bucketOrd, tsids); + groupBucketValues = bigArrays().grow(groupBucketValues, bucketOrd + 1); + groupBucketValues.set(bucketOrd, tsids); } } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/CountValuesBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/CountValuesBucketFunction.java index 584a230f680d1..fc127de9ed62f 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/CountValuesBucketFunction.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/CountValuesBucketFunction.java @@ -22,7 +22,7 @@ public class CountValuesBucketFunction implements AggregatorBucketFunction { - private ObjectArray> values; // TODO change hashmap + private ObjectArray> values; private BigArrays bigArrays; public CountValuesBucketFunction(BigArrays bigArrays) { From dae6fc8e5891b325c31e7b54003e9d176579844b Mon Sep 17 00:00:00 2001 From: weizijun Date: Tue, 16 Aug 2022 17:01:57 +0800 Subject: [PATCH 47/53] support deferring --- .../TimeSeriesAggregationsIT.java | 12 +- ...meSeriesAggregationAggregationBuilder.java | 19 +- ...meSeriesAggregationAggregationFactory.java | 9 +- ...eSeriesAggregationAggregatorDeferring.java | 820 ++++++++++++++++++ ...meSeriesAggregationAggregatorSupplier.java | 1 + ...MetricTimeSeriesAggregationAggregator.java | 65 +- 6 files changed, 912 insertions(+), 14 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorDeferring.java diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java index 1e6ee4b3565a7..397dca243cb32 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java @@ -535,7 +535,7 @@ public void testBasicTimeSeriesAggregations() { Rounding.Prepared rounding = Rounding.builder(new TimeValue(fixedInterval.estimateMillis())).build().prepareForUnknown(); SearchResponse response = client().prepareSearch("index") .setSize(0) - .addAggregation(timeSeriesAggregation("by_ts").field("metric_0").interval(fixedInterval).size(data.size())) + .addAggregation(timeSeriesAggregation("by_ts").field("metric_0").interval(fixedInterval).deferring(randomBoolean()).size(data.size())) .get(); Aggregations aggregations = response.getAggregations(); assertNotNull(aggregations); @@ -579,6 +579,7 @@ public void testTimeSeriesAggregationsDownsample() { timeSeriesAggregation("by_ts").field("metric_0") .interval(fixedInterval) .downsample(fixedInterval, Function.sum_over_time, null) + .deferring(randomBoolean()) .size(data.size()) ) .get(); @@ -618,7 +619,13 @@ public void testTimeSeriesAggregationsAggregator() { Rounding.Prepared rounding = Rounding.builder(new TimeValue(fixedInterval.estimateMillis())).build().prepareForUnknown(); SearchResponse response = client().prepareSearch("index") .setSize(0) - .addAggregation(timeSeriesAggregation("by_ts").field("metric_0").interval(fixedInterval).aggregator("sum").size(data.size())) + .addAggregation( + timeSeriesAggregation("by_ts").field("metric_0") + .interval(fixedInterval) + .aggregator("sum") + .deferring(randomBoolean()) + .size(data.size()) + ) .get(); Aggregations aggregations = response.getAggregations(); assertNotNull(aggregations); @@ -676,6 +683,7 @@ public void testTimeSeriesAggregationsGroupBy() { .interval(fixedInterval) .downsample(fixedInterval, Function.max_over_time, null) .aggregator("sum") + .deferring(randomBoolean()) .size(data.size()) ) .get(); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java index bc6d54a6d2a21..293ae65cb2644 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java @@ -52,6 +52,7 @@ public class TimeSeriesAggregationAggregationBuilder extends ValuesSourceAggrega public static final ParseField SHARD_MIN_DOC_COUNT_FIELD = new ParseField("shard_min_doc_count"); public static final ParseField START_TIME_FIELD = new ParseField("start_time"); public static final ParseField END_TIME_FIELD = new ParseField("end_time"); + public static final ParseField DEFERRING_FIELD = new ParseField("deferring"); static final TermsAggregator.BucketCountThresholds DEFAULT_BUCKET_COUNT_THRESHOLDS = new TermsAggregator.BucketCountThresholds( 1, @@ -84,6 +85,7 @@ public class TimeSeriesAggregationAggregationBuilder extends ValuesSourceAggrega private BucketOrder order = BucketOrder.key(true); private Long startTime; private Long endTime; + private boolean deferring = true; static { ValuesSourceAggregationBuilder.declareFields(PARSER, false, true, false); @@ -123,6 +125,7 @@ public class TimeSeriesAggregationAggregationBuilder extends ValuesSourceAggrega PARSER.declareInt(TimeSeriesAggregationAggregationBuilder::shardMinDocCount, SHARD_MIN_DOC_COUNT_FIELD); PARSER.declareLong(TimeSeriesAggregationAggregationBuilder::startTime, START_TIME_FIELD); PARSER.declareLong(TimeSeriesAggregationAggregationBuilder::endTime, END_TIME_FIELD); + PARSER.declareBoolean(TimeSeriesAggregationAggregationBuilder::deferring, DEFERRING_FIELD); } public TimeSeriesAggregationAggregationBuilder(String name) { @@ -147,6 +150,7 @@ protected TimeSeriesAggregationAggregationBuilder( this.bucketCountThresholds = clone.bucketCountThresholds; this.startTime = clone.startTime; this.endTime = clone.endTime; + this.deferring = clone.deferring; } public TimeSeriesAggregationAggregationBuilder(StreamInput in) throws IOException { @@ -163,6 +167,7 @@ public TimeSeriesAggregationAggregationBuilder(StreamInput in) throws IOExceptio bucketCountThresholds = new TermsAggregator.BucketCountThresholds(in); startTime = in.readOptionalLong(); endTime = in.readOptionalLong(); + deferring = in.readBoolean(); } @Override @@ -179,6 +184,7 @@ protected void innerWriteTo(StreamOutput out) throws IOException { bucketCountThresholds.writeTo(out); out.writeOptionalLong(startTime); out.writeOptionalLong(endTime); + out.writeBoolean(deferring); } @Override @@ -217,6 +223,7 @@ protected ValuesSourceAggregatorFactory innerBuild( order, startTime != null ? startTime : -1, endTime != null ? endTime : -1, + deferring, config, context, parent, @@ -259,6 +266,7 @@ protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) if (endTime != null) { builder.field(END_TIME_FIELD.getPreferredName(), endTime); } + builder.field(DEFERRING_FIELD.getPreferredName(), deferring); return builder; } @@ -279,7 +287,7 @@ public String getType() { @Override public boolean isInSortOrderExecutionRequired() { - return true; + return false == deferring; } /** @@ -539,6 +547,15 @@ public TimeSeriesAggregationAggregationBuilder endTime(long endTime) { return this; } + public boolean isDeferring() { + return deferring; + } + + public TimeSeriesAggregationAggregationBuilder deferring(boolean deferring) { + this.deferring = deferring; + return this; + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java index 25bef1563e214..7831288fe0628 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java @@ -40,6 +40,7 @@ public class TimeSeriesAggregationAggregationFactory extends ValuesSourceAggrega private final TermsAggregator.BucketCountThresholds bucketCountThresholds; private final long startTime; private final long endTime; + private boolean deferring; private final BucketOrder order; private final ValuesSourceConfig config; private final TimeSeriesAggregationAggregatorSupplier aggregatorSupplier; @@ -58,6 +59,7 @@ public TimeSeriesAggregationAggregationFactory( BucketOrder order, long startTime, long endTime, + boolean deferring, ValuesSourceConfig config, AggregationContext context, AggregatorFactory parent, @@ -77,6 +79,7 @@ public TimeSeriesAggregationAggregationFactory( this.bucketCountThresholds = bucketCountThresholds; this.startTime = startTime; this.endTime = endTime; + this.deferring = deferring; this.order = order; this.config = config; this.aggregatorSupplier = aggregatorSupplier; @@ -86,7 +89,7 @@ public static void registerAggregators(ValuesSourceRegistry.Builder builder) { builder.register( TimeSeriesAggregationAggregationBuilder.REGISTRY_KEY, List.of(CoreValuesSourceType.NUMERIC), - TimeSeriesAggregationAggregator::new, + TimeSeriesAggregationAggregatorDeferring::new, true ); } @@ -102,7 +105,7 @@ protected Aggregator createUnmapped(Aggregator parent, Map metad thresholds.setShardSize(BucketUtils.suggestShardSideQueueSize(thresholds.getRequiredSize())); } thresholds.ensureValidity(); - return new TimeSeriesAggregationAggregator( + return new TimeSeriesAggregationAggregatorDeferring( name, factories, keyed, @@ -117,6 +120,7 @@ protected Aggregator createUnmapped(Aggregator parent, Map metad order, startTime, endTime, + deferring, config, context, parent, @@ -152,6 +156,7 @@ protected Aggregator doCreateInternal(Aggregator parent, CardinalityUpperBound c order, startTime, endTime, + deferring, config, context, parent, diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorDeferring.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorDeferring.java new file mode 100644 index 0000000000000..ac5df654aadd1 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorDeferring.java @@ -0,0 +1,820 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations.timeseries.aggregation; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SortedDocValues; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.search.CollectionTerminatedException; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.Scorable; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.apache.lucene.util.PriorityQueue; +import org.apache.lucene.util.packed.PackedInts; +import org.apache.lucene.util.packed.PackedLongValues; +import org.elasticsearch.cluster.metadata.DataStream; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.LongHash; +import org.elasticsearch.common.util.ObjectArray; +import org.elasticsearch.common.util.set.Sets; +import org.elasticsearch.core.CheckedConsumer; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.index.TimestampBounds; +import org.elasticsearch.index.fielddata.IndexFieldData; +import org.elasticsearch.index.fielddata.SortedBinaryDocValues; +import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; +import org.elasticsearch.index.fielddata.plain.SortedSetBytesLeafFieldData; +import org.elasticsearch.index.mapper.TimeSeriesIdFieldMapper; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.AggregationExecutionContext; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.BucketOrder; +import org.elasticsearch.search.aggregations.CardinalityUpperBound; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalOrder; +import org.elasticsearch.search.aggregations.LeafBucketCollector; +import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; +import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; +import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; +import org.elasticsearch.search.aggregations.bucket.terms.BucketPriorityQueue; +import org.elasticsearch.search.aggregations.bucket.terms.BytesKeyedBucketOrds; +import org.elasticsearch.search.aggregations.bucket.terms.LongKeyedBucketOrds; +import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator; +import org.elasticsearch.search.aggregations.support.AggregationContext; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.AggregatorBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.TSIDBucketFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesLineAggreagation; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.atomic.AtomicLong; + +import static org.elasticsearch.search.DocValueFormat.TIME_SERIES_ID; +import static org.elasticsearch.search.aggregations.InternalOrder.isKeyOrder; + +@SuppressWarnings({ "unchecked", "rawtypes" }) +public class TimeSeriesAggregationAggregatorDeferring extends BucketsAggregator { + private static final Logger logger = LogManager.getLogger(TimeSeriesAggregationAggregatorDeferring.class); + + protected BytesKeyedBucketOrds bucketOrds; + private LongKeyedBucketOrds timestampOrds; + private ValuesSource.Numeric valuesSource; + + private boolean keyed; + + private Set group; + private Set without; + private long interval; + private long offset; + private Aggregator aggregator; + private Map aggregatorParams; + protected long downsampleRange; + protected Function downsampleFunction; + protected Map downsampleParams; + private BucketOrder order; + private TermsAggregator.BucketCountThresholds bucketCountThresholds; + protected Comparator partiallyBuiltBucketComparator; + protected DocValueFormat format; + private TimestampBounds timestampBounds; + private long startTime; + private long endTime; + protected boolean deferring; + + private BytesRef preTsid; + private long preBucketOrdinal; + protected long preRounding = -1; + private RoundingInterval rounding; + private boolean needAggregator; + protected Map timeBucketMetrics; // TODO replace map + private ObjectArray> groupBucketValues; // TODO replace map + private ObjectArray aggregatorCollectors; + + private final IndexFieldData tsidFieldData; + + private List entries = new ArrayList<>(); + private AggregationExecutionContext aggCtx; + private PackedLongValues.Builder docDeltasBuilder; + private PackedLongValues.Builder bucketsBuilder; + private LongHash selectedBuckets; + + @SuppressWarnings("unchecked") + public TimeSeriesAggregationAggregatorDeferring( + String name, + AggregatorFactories factories, + boolean keyed, + List group, + List without, + DateHistogramInterval interval, + DateHistogramInterval offset, + Aggregator aggregator, + Map aggregatorParams, + Downsample downsample, + TermsAggregator.BucketCountThresholds bucketCountThresholds, + BucketOrder order, + long startTime, + long endTime, + boolean deferring, + ValuesSourceConfig valuesSourceConfig, + AggregationContext context, + org.elasticsearch.search.aggregations.Aggregator parent, + CardinalityUpperBound bucketCardinality, + Map metadata + ) throws IOException { + super(name, factories, context, parent, bucketCardinality, metadata); + this.keyed = keyed; + this.group = group != null ? Sets.newHashSet(group) : Set.of(); + this.without = without != null ? Sets.newHashSet(without) : Set.of(); + this.interval = interval != null ? interval.estimateMillis() : -1; + if (this.interval <= 0) { + throw new IllegalArgumentException("time_series_aggregation invalid interval [" + interval + "]"); + } + this.startTime = startTime; + this.endTime = endTime; + this.deferring = deferring; + this.rounding = new RoundingInterval(this.startTime, this.interval); + this.offset = offset != null ? offset.estimateMillis() : 0; + this.aggregator = aggregator; + this.aggregatorParams = aggregatorParams; + this.needAggregator = this.aggregator != null; + this.downsampleRange = downsample != null && downsample.getRange() != null ? downsample.getRange().estimateMillis() : -1; + this.downsampleFunction = downsample != null && downsample.getFunction() != null + ? downsample.getFunction() + : (downsampleRange > 0 ? Function.origin_value : Function.last); + if (this.downsampleRange <= 0) { + this.downsampleRange = this.interval; + } + this.downsampleParams = downsample != null && downsample.getParameters() != null + ? new HashMap<>(downsample.getParameters()) + : new HashMap<>(); + this.downsampleParams.put(Function.RANGE_FIELD, downsampleRange); + this.bucketCountThresholds = bucketCountThresholds; + this.order = order == null ? BucketOrder.key(true) : order; + this.partiallyBuiltBucketComparator = order.partiallyBuiltBucketComparator(b -> b.bucketOrd, this); + this.bucketOrds = BytesKeyedBucketOrds.build(bigArrays(), bucketCardinality); + this.timestampOrds = LongKeyedBucketOrds.build(bigArrays(), CardinalityUpperBound.MANY); + if (valuesSourceConfig != null) { + this.valuesSource = valuesSourceConfig.hasValues() ? (ValuesSource.Numeric) valuesSourceConfig.getValuesSource() : null; + this.format = valuesSourceConfig.format(); + } else { + this.valuesSource = null; + this.format = null; + } + this.timestampBounds = context.getIndexSettings().getTimestampBounds(); + + if ((this.group.size() > 0 || this.without.size() > 0) && this.aggregator == null) { + throw new IllegalArgumentException("time_series_aggregation group by must have an aggregator"); + } + + groupBucketValues = bigArrays().newObjectArray(1); + aggregatorCollectors = bigArrays().newObjectArray(1); + + tsidFieldData = (IndexFieldData) Objects.requireNonNull( + context.buildFieldContext("_tsid"), + "Cannot obtain tsid field" + ).indexFieldData(); + } + + @Override + public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { + InternalTimeSeriesAggregation.InternalBucket[][] allBucketsPerOrd = + new InternalTimeSeriesAggregation.InternalBucket[owningBucketOrds.length][]; + long[] otherDocCounts = new long[owningBucketOrds.length]; + for (int ordIdx = 0; ordIdx < owningBucketOrds.length; ordIdx++) { + BytesRef spareKey = new BytesRef(); + BytesKeyedBucketOrds.BucketOrdsEnum ordsEnum = bucketOrds.ordsEnum(owningBucketOrds[ordIdx]); + long bucketsInOrd = bucketOrds.bucketsInOrd(owningBucketOrds[ordIdx]); + int size = (int) Math.min(bucketsInOrd, bucketCountThresholds.getShardSize()); + PriorityQueue ordered = new BucketPriorityQueue<>( + size, + partiallyBuiltBucketComparator + ); + while (ordsEnum.next()) { + long docCount = bucketDocCount(ordsEnum.ord()); + otherDocCounts[ordIdx] += docCount; + if (docCount < bucketCountThresholds.getShardMinDocCount()) { + continue; + } + ordsEnum.readValue(spareKey); + long ord = ordsEnum.ord(); + InternalTimeSeriesAggregation.InternalBucket bucket = new InternalTimeSeriesAggregation.InternalBucket( + TimeSeriesIdFieldMapper.decodeTsid(spareKey), + docCount, + null, + null, + keyed, + false, + 0 + ); + bucket.bucketOrd = ord; + ordered.insertWithOverflow(bucket); + } + + // Get the top buckets + InternalTimeSeriesAggregation.InternalBucket[] bucketsForOrd = new InternalTimeSeriesAggregation.InternalBucket[ordered.size()]; + allBucketsPerOrd[ordIdx] = bucketsForOrd; + List bucketList = new ArrayList<>(); + for (int b = ordered.size() - 1; b >= 0; --b) { + InternalTimeSeriesAggregation.InternalBucket bucket = ordered.pop(); + allBucketsPerOrd[ordIdx][b] = bucket; + otherDocCounts[ordIdx] -= allBucketsPerOrd[ordIdx][b].getDocCount(); + bucketList.add(bucket); + } + + if (deferring) { + if (this.selectedBuckets != null) { + throw new IllegalStateException("Already been replayed"); + } + this.selectedBuckets = new LongHash(ordered.size(), BigArrays.NON_RECYCLING_INSTANCE); + for (InternalTimeSeriesAggregation.InternalBucket bucket : bucketList) { + selectedBuckets.add(bucket.bucketOrd); + } + + PriorityQueue queue = new PriorityQueue<>(entries.size()) { + @Override + protected boolean lessThan(LeafWalker a, LeafWalker b) { + return a.getTimestamp() > b.getTimestamp(); + } + }; + + List leafWalkers = new ArrayList<>(); + for (Entry entry : entries) { + assert entry.docDeltas.size() > 0 : "segment should have at least one document to replay, got 0"; + try { + final PackedLongValues.Iterator docDeltaIterator = entry.docDeltas.iterator(); + final PackedLongValues.Iterator buckets = entry.buckets.iterator(); + LeafWalker leafWalker = new LeafWalker(entry.aggCtx.getLeafReaderContext(), docDeltaIterator, buckets); + if (leafWalker.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) { + leafWalkers.add(leafWalker); + } + } catch (CollectionTerminatedException e) { + // collection was terminated prematurely + // continue with the following leaf + } + } + + while (populateQueue(leafWalkers, queue)) { + do { + LeafWalker walker = queue.top(); + walker.collectCurrent(); + if (walker.nextDoc() == DocIdSetIterator.NO_MORE_DOCS || walker.shouldPop()) { + queue.pop(); + } else { + queue.updateTop(); + } + } while (queue.size() > 0); + } + + /** + * collect the last tsid + */ + if (timeBucketMetrics != null && timeBucketMetrics.size() > 0) { + collectTimeSeriesValues(preBucketOrdinal); + } + } + + for (InternalTimeSeriesAggregation.InternalBucket bucket : bucketList) { + long ord = bucket.bucketOrd; + Map values = new LinkedHashMap<>(); + if (needAggregator) { + AggregatorBucketFunction aggregatorBucketFunction = aggregatorCollectors.get(ord); + LongKeyedBucketOrds.BucketOrdsEnum timeOrdsEnum = timestampOrds.ordsEnum(ord); + while (timeOrdsEnum.next()) { + values.put( + timeOrdsEnum.value() + offset, + aggregatorBucketFunction.getAggregation(timeOrdsEnum.ord(), aggregatorParams, format, metadata()) + ); + } + } else { + values = groupBucketValues.get(ord); + if (values == null) { + values = new LinkedHashMap<>(); + } + } + bucket.metricAggregation = new TimeSeriesLineAggreagation(TimeSeriesLineAggreagation.NAME, values, format, metadata()); + } + } + + buildSubAggsForAllBuckets(allBucketsPerOrd, b -> b.bucketOrd, (b, a) -> b.aggregations = a); + + InternalAggregation[] result = new InternalAggregation[owningBucketOrds.length]; + for (int ordIdx = 0; ordIdx < owningBucketOrds.length; ordIdx++) { + result[ordIdx] = buildResult(otherDocCounts[ordIdx], allBucketsPerOrd[ordIdx]); + } + return result; + } + + @Override + public InternalAggregation buildEmptyAggregation() { + return new InternalTimeSeriesAggregation( + name, + order, + order, + bucketCountThresholds.getRequiredSize(), + bucketCountThresholds.getMinDocCount(), + bucketCountThresholds.getShardSize(), + false, + 0, + new ArrayList<>(), + false, + 0, + metadata() + ); + } + + @Override + protected void doClose() { + Releasables.close(bucketOrds); + Releasables.close(timestampOrds); + Releasables.close(groupBucketValues); + for (int i = 0; i < aggregatorCollectors.size(); i++) { + AggregatorBucketFunction aggregatorBucketFunction = aggregatorCollectors.get(i); + if (aggregatorBucketFunction != null) { + aggregatorBucketFunction.close(); + } + } + Releasables.close(aggregatorCollectors); + } + + public class Collector extends LeafBucketCollectorBase { + final SortedNumericDoubleValues values; + final AggregationExecutionContext aggCtx; + final LeafBucketCollector sub; + final CheckedConsumer docConsumer; + + public Collector( + LeafBucketCollector sub, + SortedNumericDoubleValues values, + AggregationExecutionContext aggCtx, + CheckedConsumer docConsumer + ) { + super(sub, values); + this.sub = sub; + this.values = values; + this.aggCtx = aggCtx; + this.docConsumer = docConsumer; + } + + @Override + public void collect(int doc, long bucket) throws IOException { + BytesRef newTsid = aggCtx.getTsid(); + + if (preTsid == null) { + reset(newTsid, bucket); + } else if (false == preTsid.equals(newTsid)) { + collectTimeSeriesValues(preBucketOrdinal); + reset(newTsid, bucket); + } + + if (preRounding < 0 || aggCtx.getTimestamp() < preRounding - interval) { + preRounding = rounding.nextRoundingValue(aggCtx.getTimestamp()); + } + + // calculate the value of the current doc + docConsumer.accept(doc); + collectBucket(sub, doc, preBucketOrdinal); + } + + private void reset(BytesRef tsid, long bucket) { + timeBucketMetrics = new TreeMap<>(); + preTsid = BytesRef.deepCopyOf(tsid); + preRounding = -1; + + BytesRef bucketValue = needAggregator ? packKey(preTsid) : preTsid; + long bucketOrdinal = bucketOrds.add(bucket, bucketValue); + if (bucketOrdinal < 0) { // already seen + bucketOrdinal = -1 - bucketOrdinal; + grow(bucketOrdinal + 1); + } + preBucketOrdinal = bucketOrdinal; + } + } + + public class DeferringCollector extends LeafBucketCollector { + final SortedNumericDoubleValues values; + final AggregationExecutionContext aggCtx; + final CheckedConsumer docConsumer; + + public DeferringCollector( + SortedNumericDoubleValues values, + AggregationExecutionContext aggCtx, + CheckedConsumer docConsumer + ) { + this.values = values; + this.aggCtx = aggCtx; + this.docConsumer = docConsumer; + } + + @Override + public void collect(int doc, long bucket) throws IOException { + BytesRef newTsid = aggCtx.getTsid(); + + if (preTsid == null) { + reset(newTsid, bucket); + } else if (false == preTsid.equals(newTsid)) { + collectTimeSeriesValues(preBucketOrdinal); + reset(newTsid, bucket); + } + + if (preRounding < 0 || aggCtx.getTimestamp() < preRounding - interval) { + preRounding = rounding.nextRoundingValue(aggCtx.getTimestamp()); + } + + // calculate the value of the current doc + docConsumer.accept(doc); + } + + private void reset(BytesRef tsid, long bucket) { + timeBucketMetrics = new TreeMap<>(); + preTsid = BytesRef.deepCopyOf(tsid); + preRounding = -1; + preBucketOrdinal = bucket; + } + } + + @Override + protected LeafBucketCollector getLeafCollector(AggregationExecutionContext context, LeafBucketCollector sub) throws IOException { + if (valuesSource == null) { + return new LeafBucketCollector() { + @Override + public void setScorer(Scorable arg0) throws IOException { + // no-op + } + + @Override + public void collect(int doc, long bucket) { + // no-op + } + + @Override + public boolean isNoop() { + return false; + } + }; + } + + if (deferring) { + final SortedBinaryDocValues tsids = tsidFieldData.load(context.getLeafReaderContext()).getBytesValues(); + BytesRef tsid = null; + final AtomicLong currentBucketOrdinal = new AtomicLong(); + finishLeaf(); + return new LeafBucketCollectorBase(sub, null) { + int lastDoc = 0; + + @Override + public void collect(int doc, long bucket) throws IOException { + if (tsids.advanceExact(doc)) { + BytesRef newTsid = tsids.nextValue(); + if (tsid == null) { + reset(newTsid, bucket); + } else if (false == tsid.equals(newTsid)) { + reset(newTsid, bucket); + } + } + + if (aggCtx == null) { + aggCtx = context; + docDeltasBuilder = PackedLongValues.packedBuilder(PackedInts.DEFAULT); + bucketsBuilder = PackedLongValues.packedBuilder(PackedInts.DEFAULT); + } + docDeltasBuilder.add(doc - lastDoc); + bucketsBuilder.add(currentBucketOrdinal.get()); + lastDoc = doc; + + collectBucket(sub, doc, currentBucketOrdinal.get()); + } + + private void reset(BytesRef tsid, long bucket) { + tsid = BytesRef.deepCopyOf(tsid); + + BytesRef bucketValue = needAggregator ? packKey(tsid) : tsid; + long bucketOrdinal = bucketOrds.add(bucket, bucketValue); + if (bucketOrdinal < 0) { // already seen + bucketOrdinal = -1 - bucketOrdinal; + grow(bucketOrdinal + 1); + } + currentBucketOrdinal.set(bucketOrdinal); + } + }; + } else { + return getCollector(sub, context); + } + } + + protected LeafBucketCollector getCollector( + LeafBucketCollector sub, + AggregationExecutionContext aggCtx + ) throws IOException { + final SortedNumericDoubleValues values = valuesSource.doubleValues(aggCtx.getLeafReaderContext()); + CheckedConsumer docConsumer = (doc) -> { + if (aggCtx.getTimestamp() + downsampleRange < preRounding) { + return; + } + + if (values.advanceExact(doc)) { + final int valuesCount = values.docValueCount(); + for (int i = 0; i < valuesCount; i++) { + double value = values.nextValue(); + if (false == timeBucketMetrics.containsKey(preRounding)) { + downsampleParams.put(Function.ROUNDING_FIELD, preRounding); + timeBucketMetrics.put(preRounding, downsampleFunction.getFunction(downsampleParams)); + } + for (Map.Entry entry : timeBucketMetrics.entrySet()) { + Long timestamp = entry.getKey(); + AggregatorFunction function = entry.getValue(); + if (aggCtx.getTimestamp() + downsampleRange >= timestamp) { + function.collect(new TimePoint(aggCtx.getTimestamp(), value)); + } else { + break; + } + } + } + } + }; + if (deferring) { + return new DeferringCollector(values, aggCtx, docConsumer); + } else { + return new Collector(sub, values, aggCtx, docConsumer); + } + } + + @Override + protected void doPostCollection() throws IOException { + if (deferring) { + finishLeaf(); + } else { + /** + * collect the last tsid + */ + if (timeBucketMetrics != null && timeBucketMetrics.size() > 0) { + collectTimeSeriesValues(preBucketOrdinal); + } + } + + } + + /** + * Button up the builders for the current leaf. + */ + private void finishLeaf() { + if (aggCtx != null) { + assert docDeltasBuilder != null && bucketsBuilder != null; + assert docDeltasBuilder.size() > 0; + entries.add(new Entry(aggCtx, docDeltasBuilder.build(), bucketsBuilder.build())); + clearLeaf(); + } + } + + /** + * Clear the status for the current leaf. + */ + private void clearLeaf() { + aggCtx = null; + docDeltasBuilder = null; + bucketsBuilder = null; + } + + InternalTimeSeriesAggregation buildResult(long otherDocCount, InternalTimeSeriesAggregation.InternalBucket[] topBuckets) { + final BucketOrder reduceOrder; + if (isKeyOrder(order) == false) { + reduceOrder = InternalOrder.key(true); + Arrays.sort(topBuckets, reduceOrder.comparator()); + } else { + reduceOrder = order; + } + return new InternalTimeSeriesAggregation( + name, + reduceOrder, + order, + bucketCountThresholds.getRequiredSize(), + bucketCountThresholds.getMinDocCount(), + bucketCountThresholds.getShardSize(), + false, + otherDocCount, + List.of(topBuckets), + keyed, + 0, + metadata() + ); + } + + /** + * decode the tsid and pack the bucket key from the group and without config + */ + private BytesRef packKey(BytesRef tsid) { + if (group.size() == 0 && without.size() == 0) { + return new BytesRef(new byte[] { 0 }); + } + + Map tsidMap = TimeSeriesIdFieldMapper.decodeTsid(tsid); + Map groupMap = new LinkedHashMap<>(); + tsidMap.forEach((key, value) -> { + if (group.size() > 0) { + if (group.contains(key) && false == without.contains(key)) { + groupMap.put(key, value); + } + } else if (without.size() > 0) { + if (false == without.contains(key)) { + groupMap.put(key, value); + } + } + }); + return TIME_SERIES_ID.parseBytesRef(groupMap); + } + + /** + * collect the value of one time series line + */ + public void collectTimeSeriesValues(long bucketOrd) throws IOException { + if (needAggregator) { + aggregatorCollectors = bigArrays().grow(aggregatorCollectors, bucketOrd + 1); + AggregatorBucketFunction aggregatorBucketFunction = aggregatorCollectors.get(bucketOrd); + if (aggregatorBucketFunction == null) { + AggregatorBucketFunction internal = aggregator.getAggregatorBucketFunction(bigArrays(), aggregatorParams); + aggregatorBucketFunction = new TSIDBucketFunction(bigArrays(), internal); + aggregatorCollectors.set(bucketOrd, aggregatorBucketFunction); + } + + for (Map.Entry entry : timeBucketMetrics.entrySet()) { + Long timestamp = entry.getKey(); + AggregatorFunction value = entry.getValue(); + if (logger.isTraceEnabled()) { + logger.trace( + "collect time_series, time={}, value={}, tsid={}, hashcode={}", + timestamp, + value.get(), + TimeSeriesIdFieldMapper.decodeTsid(preTsid), + this.hashCode() + ); + } + + long ord = timestampOrds.add(bucketOrd, timestamp); + if (ord < 0) { // already seen + ord = -1 - ord; + } + if (timestamp - interval <= timestampBounds.startTime() || timestamp > timestampBounds.endTime()) { + aggregatorBucketFunction.collect(new TSIDValue(preTsid, value.getAggregation(format, metadata())), ord); + } else { + aggregatorBucketFunction.collect(new TSIDValue(preTsid, value.get()), ord); + } + } + } else { + Map tsids = new LinkedHashMap<>(); + timeBucketMetrics.forEach((k, v) -> { tsids.put(k + offset, v.getAggregation(format, metadata())); }); + groupBucketValues = bigArrays().grow(groupBucketValues, bucketOrd + 1); + groupBucketValues.set(bucketOrd, tsids); + } + } + + static class Entry { + final AggregationExecutionContext aggCtx; + final PackedLongValues docDeltas; + final PackedLongValues buckets; + + Entry(AggregationExecutionContext aggCtx, PackedLongValues docDeltas, PackedLongValues buckets) { + this.aggCtx = Objects.requireNonNull(aggCtx); + this.docDeltas = Objects.requireNonNull(docDeltas); + this.buckets = Objects.requireNonNull(buckets); + } + } + + // Re-populate the queue with walkers on the same TSID. + private boolean populateQueue(List leafWalkers, PriorityQueue queue) throws IOException { + BytesRef currentTsid = null; + assert queue.size() == 0; + Iterator it = leafWalkers.iterator(); + while (it.hasNext()) { + LeafWalker leafWalker = it.next(); + if (leafWalker.docId == DocIdSetIterator.NO_MORE_DOCS) { + // If a walker is exhausted then we can remove it from consideration + // entirely + it.remove(); + continue; + } + BytesRef tsid = leafWalker.getTsid(); + if (currentTsid == null) { + currentTsid = tsid; + } + int comp = tsid.compareTo(currentTsid); + if (comp == 0) { + queue.add(leafWalker); + } else if (comp < 0) { + // We've found a walker on a lower TSID, so we remove all walkers + // collected so far from the queue and reset our comparison TSID + // to be the lower value + queue.clear(); + queue.add(leafWalker); + currentTsid = tsid; + } + } + assert queueAllHaveTsid(queue, currentTsid); + // If all walkers are exhausted then nothing will have been added to the queue + // and we're done + return queue.size() > 0; + } + + private static boolean queueAllHaveTsid(PriorityQueue queue, BytesRef tsid) throws IOException { + for (LeafWalker leafWalker : queue) { + BytesRef walkerId = leafWalker.tsids.lookupOrd(leafWalker.tsids.ordValue()); + assert walkerId.equals(tsid) : tsid.utf8ToString() + " != " + walkerId.utf8ToString(); + } + return true; + } + + class LeafWalker { + private final LeafBucketCollector collector; + private final SortedDocValues tsids; + private final SortedNumericDocValues timestamps; + private final BytesRefBuilder scratch = new BytesRefBuilder(); + int docId = 0; + long currentBucket = 0; + int tsidOrd; + long timestamp; + + final PackedLongValues.Iterator docDeltaIterator; + final PackedLongValues.Iterator buckets; + + LeafWalker(LeafReaderContext context, PackedLongValues.Iterator docDeltaIterator, PackedLongValues.Iterator buckets) + throws IOException { + this.docDeltaIterator = docDeltaIterator; + this.buckets = buckets; + AggregationExecutionContext aggContext = new AggregationExecutionContext(context, scratch::get, () -> timestamp); + this.collector = getCollector(null, aggContext); + tsids = DocValues.getSorted(context.reader(), TimeSeriesIdFieldMapper.NAME); + timestamps = DocValues.getSortedNumeric(context.reader(), DataStream.TimestampField.FIXED_TIMESTAMP_FIELD); + } + + void collectCurrent() throws IOException { + assert tsids.docID() == docId; + assert timestamps.docID() == docId; + collector.collect(docId, currentBucket); + } + + int nextDoc() throws IOException { + if (docId == DocIdSetIterator.NO_MORE_DOCS) { + return DocIdSetIterator.NO_MORE_DOCS; + } + + do { + if (false == docDeltaIterator.hasNext()) { + docId = DocIdSetIterator.NO_MORE_DOCS; + break; + } + docId += docDeltaIterator.next(); + final long bucket = buckets.next(); + final long rebasedBucket = selectedBuckets.find(bucket); + if (rebasedBucket != -1 && isValidDoc(docId)) { + currentBucket = bucket; + timestamp = timestamps.nextValue(); + break; + } + } while (docId != DocIdSetIterator.NO_MORE_DOCS); + return docId; + } + + private boolean isValidDoc(int docId) throws IOException { + return tsids.advanceExact(docId) && timestamps.advanceExact(docId); + } + + BytesRef getTsid() throws IOException { + tsidOrd = tsids.ordValue(); + scratch.copyBytes(tsids.lookupOrd(tsidOrd)); + return scratch.get(); + } + + public long getTimestamp() { + return timestamp; + } + + // true if the TSID ord has changed since the last time we checked + boolean shouldPop() throws IOException { + if (tsidOrd != tsids.ordValue()) { + return true; + } else { + return false; + } + } + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java index fb1e28cc9a38f..9b1c4812f2c13 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java @@ -38,6 +38,7 @@ Aggregator build( BucketOrder order, long startTime, long endTime, + boolean deferring, ValuesSourceConfig valuesSourceConfig, AggregationContext context, Aggregator parent, diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java index 7ded9cbf7c09b..f4d6487857418 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java @@ -9,6 +9,7 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.Scorable; +import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.Tuple; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; import org.elasticsearch.search.aggregations.AggregationExecutionContext; @@ -23,7 +24,9 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.timeseries.aggregation.Downsample; import org.elasticsearch.search.aggregations.timeseries.aggregation.Function; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregationAggregator; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregationAggregatorDeferring; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; import org.elasticsearch.xpack.aggregatemetric.aggregations.support.AggregateMetricsValuesSource; import org.elasticsearch.xpack.aggregatemetric.mapper.AggregateDoubleMetricFieldMapper.Metric; @@ -34,7 +37,7 @@ import java.util.Map.Entry; @SuppressWarnings({ "unchecked", "rawtypes" }) -public class AggregateMetricTimeSeriesAggregationAggregator extends TimeSeriesAggregationAggregator { +public class AggregateMetricTimeSeriesAggregationAggregator extends TimeSeriesAggregationAggregatorDeferring { private final AggregateMetricsValuesSource.AggregateDoubleMetric valuesSource; public AggregateMetricTimeSeriesAggregationAggregator( @@ -52,6 +55,7 @@ public AggregateMetricTimeSeriesAggregationAggregator( BucketOrder order, long startTime, long endTime, + boolean deferring, ValuesSourceConfig valuesSourceConfig, AggregationContext context, Aggregator parent, @@ -73,6 +77,7 @@ public AggregateMetricTimeSeriesAggregationAggregator( order, startTime, endTime, + deferring, null, context, parent, @@ -90,8 +95,7 @@ public AggregateMetricTimeSeriesAggregationAggregator( } @Override - protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBucketCollector sub, AggregationExecutionContext aggCtx) - throws IOException { + protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException { if (valuesSource == null) { return new LeafBucketCollector() { @Override @@ -110,14 +114,51 @@ public boolean isNoop() { } }; } + return getCollector(sub, aggCtx); + } + + @Override + protected LeafBucketCollector getCollector( + LeafBucketCollector sub, + AggregationExecutionContext aggCtx + ) throws IOException { Metric metricType = getAggregateMetric(); if (metricType != null) { - final SortedNumericDoubleValues values = valuesSource.getAggregateMetricValues(context, metricType); - return getCollector(sub, aggCtx, values); + final SortedNumericDoubleValues values = valuesSource.getAggregateMetricValues(aggCtx.getLeafReaderContext(), metricType); + CheckedConsumer docConsumer = (doc) -> { + if (aggCtx.getTimestamp() + downsampleRange < preRounding) { + return; + } + + if (values.advanceExact(doc)) { + final int valuesCount = values.docValueCount(); + for (int i = 0; i < valuesCount; i++) { + double value = values.nextValue(); + if (false == timeBucketMetrics.containsKey(preRounding)) { + downsampleParams.put(Function.ROUNDING_FIELD, preRounding); + timeBucketMetrics.put(preRounding, downsampleFunction.getFunction(downsampleParams)); + } + for (Map.Entry entry : timeBucketMetrics.entrySet()) { + Long timestamp = entry.getKey(); + AggregatorFunction function = entry.getValue(); + if (aggCtx.getTimestamp() + downsampleRange >= timestamp) { + function.collect(new TimePoint(aggCtx.getTimestamp(), value)); + } else { + break; + } + } + } + } + }; + if (deferring) { + return new DeferringCollector(values, aggCtx, docConsumer); + } else { + return new Collector(sub, values, aggCtx, docConsumer); + } } else { - final SortedNumericDoubleValues aggregateSums = valuesSource.getAggregateMetricValues(context, Metric.sum); - final SortedNumericDoubleValues aggregateValueCounts = valuesSource.getAggregateMetricValues(context, Metric.value_count); - return new Collector(sub, aggregateSums, aggCtx, (doc) -> { + final SortedNumericDoubleValues aggregateSums = valuesSource.getAggregateMetricValues(aggCtx.getLeafReaderContext(), Metric.sum); + final SortedNumericDoubleValues aggregateValueCounts = valuesSource.getAggregateMetricValues(aggCtx.getLeafReaderContext(), Metric.value_count); + CheckedConsumer docConsumer = (doc) -> { if (aggCtx.getTimestamp() + downsampleRange < preRounding) { return; } @@ -153,7 +194,13 @@ public boolean isNoop() { break; } } - }); + }; + + if (deferring) { + return new DeferringCollector(aggregateSums, aggCtx, docConsumer); + } else { + return new Collector(sub, aggregateSums, aggCtx, docConsumer); + } } } From 6e7c4d28aba7d79ea4b881f329fa596c74e5d1c3 Mon Sep 17 00:00:00 2001 From: weizijun Date: Thu, 18 Aug 2022 17:51:58 +0800 Subject: [PATCH 48/53] improve fetch tsid --- ...eSeriesAggregationAggregatorDeferring.java | 35 +++++++------------ 1 file changed, 12 insertions(+), 23 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorDeferring.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorDeferring.java index ac5df654aadd1..7feab4f55a8e7 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorDeferring.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorDeferring.java @@ -30,10 +30,7 @@ import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.Releasables; import org.elasticsearch.index.TimestampBounds; -import org.elasticsearch.index.fielddata.IndexFieldData; -import org.elasticsearch.index.fielddata.SortedBinaryDocValues; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; -import org.elasticsearch.index.fielddata.plain.SortedSetBytesLeafFieldData; import org.elasticsearch.index.mapper.TimeSeriesIdFieldMapper; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.AggregationExecutionContext; @@ -70,6 +67,7 @@ import java.util.Objects; import java.util.Set; import java.util.TreeMap; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import static org.elasticsearch.search.DocValueFormat.TIME_SERIES_ID; @@ -112,8 +110,6 @@ public class TimeSeriesAggregationAggregatorDeferring extends BucketsAggregator private ObjectArray> groupBucketValues; // TODO replace map private ObjectArray aggregatorCollectors; - private final IndexFieldData tsidFieldData; - private List entries = new ArrayList<>(); private AggregationExecutionContext aggCtx; private PackedLongValues.Builder docDeltasBuilder; @@ -190,11 +186,6 @@ public TimeSeriesAggregationAggregatorDeferring( groupBucketValues = bigArrays().newObjectArray(1); aggregatorCollectors = bigArrays().newObjectArray(1); - - tsidFieldData = (IndexFieldData) Objects.requireNonNull( - context.buildFieldContext("_tsid"), - "Cannot obtain tsid field" - ).indexFieldData(); } @Override @@ -476,8 +467,8 @@ public boolean isNoop() { } if (deferring) { - final SortedBinaryDocValues tsids = tsidFieldData.load(context.getLeafReaderContext()).getBytesValues(); - BytesRef tsid = null; + SortedDocValues tsids = DocValues.getSorted(context.getLeafReaderContext().reader(), TimeSeriesIdFieldMapper.NAME); + final AtomicInteger tsidOrd = new AtomicInteger(-1); final AtomicLong currentBucketOrdinal = new AtomicLong(); finishLeaf(); return new LeafBucketCollectorBase(sub, null) { @@ -486,11 +477,11 @@ public boolean isNoop() { @Override public void collect(int doc, long bucket) throws IOException { if (tsids.advanceExact(doc)) { - BytesRef newTsid = tsids.nextValue(); - if (tsid == null) { - reset(newTsid, bucket); - } else if (false == tsid.equals(newTsid)) { - reset(newTsid, bucket); + int newTsidOrd = tsids.ordValue(); + if (tsidOrd.get() < 0) { + reset(newTsidOrd, bucket); + } else if (tsidOrd.get() != newTsidOrd) { + reset(newTsidOrd, bucket); } } @@ -506,8 +497,9 @@ public void collect(int doc, long bucket) throws IOException { collectBucket(sub, doc, currentBucketOrdinal.get()); } - private void reset(BytesRef tsid, long bucket) { - tsid = BytesRef.deepCopyOf(tsid); + private void reset(int newTsidOrd, long bucket) throws IOException { + tsidOrd.set(newTsidOrd); + BytesRef tsid = tsids.lookupOrd(newTsidOrd); BytesRef bucketValue = needAggregator ? packKey(tsid) : tsid; long bucketOrdinal = bucketOrds.add(bucket, bucketValue); @@ -523,10 +515,7 @@ private void reset(BytesRef tsid, long bucket) { } } - protected LeafBucketCollector getCollector( - LeafBucketCollector sub, - AggregationExecutionContext aggCtx - ) throws IOException { + protected LeafBucketCollector getCollector(LeafBucketCollector sub, AggregationExecutionContext aggCtx) throws IOException { final SortedNumericDoubleValues values = valuesSource.doubleValues(aggCtx.getLeafReaderContext()); CheckedConsumer docConsumer = (doc) -> { if (aggCtx.getTimestamp() + downsampleRange < preRounding) { From 4a75da2bf0e94dcdbec106239dcea524b6906691 Mon Sep 17 00:00:00 2001 From: weizijun Date: Thu, 18 Aug 2022 17:53:20 +0800 Subject: [PATCH 49/53] fix aggregate_metric_double agg --- .../TimeSeriesAggregationAggregatorDeferring.java | 8 ++++++++ .../AggregateMetricTimeSeriesAggregationAggregator.java | 2 +- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorDeferring.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorDeferring.java index 7feab4f55a8e7..72fd809c59e89 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorDeferring.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorDeferring.java @@ -466,6 +466,14 @@ public boolean isNoop() { }; } + return getLeafCollectorInternal(context, sub, aggContext); + } + + protected LeafBucketCollector getLeafCollectorInternal( + LeafReaderContext context, + LeafBucketCollector sub, + AggregationExecutionContext aggContext + ) throws IOException { if (deferring) { SortedDocValues tsids = DocValues.getSorted(context.getLeafReaderContext().reader(), TimeSeriesIdFieldMapper.NAME); final AtomicInteger tsidOrd = new AtomicInteger(-1); diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java index f4d6487857418..e846cfe94696e 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java @@ -114,7 +114,7 @@ public boolean isNoop() { } }; } - return getCollector(sub, aggCtx); + return getLeafCollectorInternal(context, sub, aggCtx); } @Override From a0f29f5f24c366dc6db72b8cf4f38633df50f1da Mon Sep 17 00:00:00 2001 From: weizijun Date: Thu, 18 Aug 2022 17:55:22 +0800 Subject: [PATCH 50/53] fixup --- .../TimeSeriesAggregationAggregatorDeferring.java | 8 ++++---- .../AggregateMetricTimeSeriesAggregationAggregator.java | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorDeferring.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorDeferring.java index 72fd809c59e89..0350bb4a43636 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorDeferring.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorDeferring.java @@ -466,7 +466,7 @@ public boolean isNoop() { }; } - return getLeafCollectorInternal(context, sub, aggContext); + return getLeafCollectorInternal(context.getLeafReaderContext(), sub, context); } protected LeafBucketCollector getLeafCollectorInternal( @@ -475,7 +475,7 @@ protected LeafBucketCollector getLeafCollectorInternal( AggregationExecutionContext aggContext ) throws IOException { if (deferring) { - SortedDocValues tsids = DocValues.getSorted(context.getLeafReaderContext().reader(), TimeSeriesIdFieldMapper.NAME); + SortedDocValues tsids = DocValues.getSorted(context.reader(), TimeSeriesIdFieldMapper.NAME); final AtomicInteger tsidOrd = new AtomicInteger(-1); final AtomicLong currentBucketOrdinal = new AtomicLong(); finishLeaf(); @@ -494,7 +494,7 @@ public void collect(int doc, long bucket) throws IOException { } if (aggCtx == null) { - aggCtx = context; + aggCtx = aggContext; docDeltasBuilder = PackedLongValues.packedBuilder(PackedInts.DEFAULT); bucketsBuilder = PackedLongValues.packedBuilder(PackedInts.DEFAULT); } @@ -519,7 +519,7 @@ private void reset(int newTsidOrd, long bucket) throws IOException { } }; } else { - return getCollector(sub, context); + return getCollector(sub, aggContext); } } diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java index e846cfe94696e..a7755ef8652b7 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java @@ -114,7 +114,7 @@ public boolean isNoop() { } }; } - return getLeafCollectorInternal(context, sub, aggCtx); + return getLeafCollectorInternal(aggCtx.getLeafReaderContext(), sub, aggCtx); } @Override From d50661626aa9d33b7d3e021fbf250307fed8546e Mon Sep 17 00:00:00 2001 From: weizijun Date: Thu, 18 Aug 2022 17:56:31 +0800 Subject: [PATCH 51/53] rm old aggregator --- ...meSeriesAggregationAggregationFactory.java | 4 +- .../TimeSeriesAggregationAggregator.java | 416 ++++++++- ...eSeriesAggregationAggregatorDeferring.java | 817 ------------------ ...MetricTimeSeriesAggregationAggregator.java | 4 +- 4 files changed, 375 insertions(+), 866 deletions(-) delete mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorDeferring.java diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java index 7831288fe0628..ebed2951d59cd 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java @@ -89,7 +89,7 @@ public static void registerAggregators(ValuesSourceRegistry.Builder builder) { builder.register( TimeSeriesAggregationAggregationBuilder.REGISTRY_KEY, List.of(CoreValuesSourceType.NUMERIC), - TimeSeriesAggregationAggregatorDeferring::new, + TimeSeriesAggregationAggregator::new, true ); } @@ -105,7 +105,7 @@ protected Aggregator createUnmapped(Aggregator parent, Map metad thresholds.setShardSize(BucketUtils.suggestShardSideQueueSize(thresholds.getRequiredSize())); } thresholds.ensureValidity(); - return new TimeSeriesAggregationAggregatorDeferring( + return new TimeSeriesAggregationAggregator( name, factories, keyed, diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index b5ed8eed8fb87..fc8758810a78c 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -10,10 +10,21 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.lucene.index.DocValues; import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SortedDocValues; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.search.CollectionTerminatedException; +import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.Scorable; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; import org.apache.lucene.util.PriorityQueue; +import org.apache.lucene.util.packed.PackedInts; +import org.apache.lucene.util.packed.PackedLongValues; +import org.elasticsearch.cluster.metadata.DataStream; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.LongHash; import org.elasticsearch.common.util.ObjectArray; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.core.CheckedConsumer; @@ -49,12 +60,15 @@ import java.util.Arrays; import java.util.Comparator; import java.util.HashMap; +import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.Map.Entry; +import java.util.Objects; import java.util.Set; import java.util.TreeMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import static org.elasticsearch.search.DocValueFormat.TIME_SERIES_ID; import static org.elasticsearch.search.aggregations.InternalOrder.isKeyOrder; @@ -85,6 +99,7 @@ public class TimeSeriesAggregationAggregator extends BucketsAggregator { private TimestampBounds timestampBounds; private long startTime; private long endTime; + protected boolean deferring; private BytesRef preTsid; private long preBucketOrdinal; @@ -95,6 +110,12 @@ public class TimeSeriesAggregationAggregator extends BucketsAggregator { private ObjectArray> groupBucketValues; // TODO replace map private ObjectArray aggregatorCollectors; + private List entries = new ArrayList<>(); + private AggregationExecutionContext aggCtx; + private PackedLongValues.Builder docDeltasBuilder; + private PackedLongValues.Builder bucketsBuilder; + private LongHash selectedBuckets; + @SuppressWarnings("unchecked") public TimeSeriesAggregationAggregator( String name, @@ -111,6 +132,7 @@ public TimeSeriesAggregationAggregator( BucketOrder order, long startTime, long endTime, + boolean deferring, ValuesSourceConfig valuesSourceConfig, AggregationContext context, org.elasticsearch.search.aggregations.Aggregator parent, @@ -127,6 +149,7 @@ public TimeSeriesAggregationAggregator( } this.startTime = startTime; this.endTime = endTime; + this.deferring = deferring; this.rounding = new RoundingInterval(this.startTime, this.interval); this.offset = offset != null ? offset.estimateMillis() : 0; this.aggregator = aggregator; @@ -203,8 +226,67 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I // Get the top buckets InternalTimeSeriesAggregation.InternalBucket[] bucketsForOrd = new InternalTimeSeriesAggregation.InternalBucket[ordered.size()]; allBucketsPerOrd[ordIdx] = bucketsForOrd; + List bucketList = new ArrayList<>(); for (int b = ordered.size() - 1; b >= 0; --b) { InternalTimeSeriesAggregation.InternalBucket bucket = ordered.pop(); + allBucketsPerOrd[ordIdx][b] = bucket; + otherDocCounts[ordIdx] -= allBucketsPerOrd[ordIdx][b].getDocCount(); + bucketList.add(bucket); + } + + if (deferring) { + if (this.selectedBuckets != null) { + throw new IllegalStateException("Already been replayed"); + } + this.selectedBuckets = new LongHash(ordered.size(), BigArrays.NON_RECYCLING_INSTANCE); + for (InternalTimeSeriesAggregation.InternalBucket bucket : bucketList) { + selectedBuckets.add(bucket.bucketOrd); + } + + PriorityQueue queue = new PriorityQueue<>(entries.size()) { + @Override + protected boolean lessThan(LeafWalker a, LeafWalker b) { + return a.getTimestamp() > b.getTimestamp(); + } + }; + + List leafWalkers = new ArrayList<>(); + for (Entry entry : entries) { + assert entry.docDeltas.size() > 0 : "segment should have at least one document to replay, got 0"; + try { + final PackedLongValues.Iterator docDeltaIterator = entry.docDeltas.iterator(); + final PackedLongValues.Iterator buckets = entry.buckets.iterator(); + LeafWalker leafWalker = new LeafWalker(entry.aggCtx.getLeafReaderContext(), docDeltaIterator, buckets); + if (leafWalker.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) { + leafWalkers.add(leafWalker); + } + } catch (CollectionTerminatedException e) { + // collection was terminated prematurely + // continue with the following leaf + } + } + + while (populateQueue(leafWalkers, queue)) { + do { + LeafWalker walker = queue.top(); + walker.collectCurrent(); + if (walker.nextDoc() == DocIdSetIterator.NO_MORE_DOCS || walker.shouldPop()) { + queue.pop(); + } else { + queue.updateTop(); + } + } while (queue.size() > 0); + } + + /** + * collect the last tsid + */ + if (timeBucketMetrics != null && timeBucketMetrics.size() > 0) { + collectTimeSeriesValues(preBucketOrdinal); + } + } + + for (InternalTimeSeriesAggregation.InternalBucket bucket : bucketList) { long ord = bucket.bucketOrd; Map values = new LinkedHashMap<>(); if (needAggregator) { @@ -223,8 +305,6 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I } } bucket.metricAggregation = new TimeSeriesLineAggreagation(TimeSeriesLineAggreagation.NAME, values, format, metadata()); - allBucketsPerOrd[ordIdx][b] = bucket; - otherDocCounts[ordIdx] -= allBucketsPerOrd[ordIdx][b].getDocCount(); } } @@ -260,7 +340,7 @@ protected void doClose() { Releasables.close(bucketOrds); Releasables.close(timestampOrds); Releasables.close(groupBucketValues); - for (int i = 0; i< aggregatorCollectors.size(); i++) { + for (int i = 0; i < aggregatorCollectors.size(); i++) { AggregatorBucketFunction aggregatorBucketFunction = aggregatorCollectors.get(i); if (aggregatorBucketFunction != null) { aggregatorBucketFunction.close(); @@ -323,12 +403,129 @@ private void reset(BytesRef tsid, long bucket) { } } - protected LeafBucketCollector getCollector( + public class DeferringCollector extends LeafBucketCollector { + final SortedNumericDoubleValues values; + final AggregationExecutionContext aggCtx; + final CheckedConsumer docConsumer; + + public DeferringCollector( + SortedNumericDoubleValues values, + AggregationExecutionContext aggCtx, + CheckedConsumer docConsumer + ) { + this.values = values; + this.aggCtx = aggCtx; + this.docConsumer = docConsumer; + } + + @Override + public void collect(int doc, long bucket) throws IOException { + BytesRef newTsid = aggCtx.getTsid(); + + if (preTsid == null) { + reset(newTsid, bucket); + } else if (false == preTsid.equals(newTsid)) { + collectTimeSeriesValues(preBucketOrdinal); + reset(newTsid, bucket); + } + + if (preRounding < 0 || aggCtx.getTimestamp() < preRounding - interval) { + preRounding = rounding.nextRoundingValue(aggCtx.getTimestamp()); + } + + // calculate the value of the current doc + docConsumer.accept(doc); + } + + private void reset(BytesRef tsid, long bucket) { + timeBucketMetrics = new TreeMap<>(); + preTsid = BytesRef.deepCopyOf(tsid); + preRounding = -1; + preBucketOrdinal = bucket; + } + } + + @Override + protected LeafBucketCollector getLeafCollector(AggregationExecutionContext context, LeafBucketCollector sub) throws IOException { + if (valuesSource == null) { + return new LeafBucketCollector() { + @Override + public void setScorer(Scorable arg0) throws IOException { + // no-op + } + + @Override + public void collect(int doc, long bucket) { + // no-op + } + + @Override + public boolean isNoop() { + return false; + } + }; + } + + return getLeafCollectorInternal(context.getLeafReaderContext(), sub, context); + } + + protected LeafBucketCollector getLeafCollectorInternal( + LeafReaderContext context, LeafBucketCollector sub, - AggregationExecutionContext aggCtx, - SortedNumericDoubleValues values - ) { - return new Collector(sub, values, aggCtx, (doc) -> { + AggregationExecutionContext aggContext + ) throws IOException { + if (deferring) { + SortedDocValues tsids = DocValues.getSorted(context.reader(), TimeSeriesIdFieldMapper.NAME); + final AtomicInteger tsidOrd = new AtomicInteger(-1); + final AtomicLong currentBucketOrdinal = new AtomicLong(); + finishLeaf(); + return new LeafBucketCollectorBase(sub, null) { + int lastDoc = 0; + + @Override + public void collect(int doc, long bucket) throws IOException { + if (tsids.advanceExact(doc)) { + int newTsidOrd = tsids.ordValue(); + if (tsidOrd.get() < 0) { + reset(newTsidOrd, bucket); + } else if (tsidOrd.get() != newTsidOrd) { + reset(newTsidOrd, bucket); + } + } + + if (aggCtx == null) { + aggCtx = aggContext; + docDeltasBuilder = PackedLongValues.packedBuilder(PackedInts.DEFAULT); + bucketsBuilder = PackedLongValues.packedBuilder(PackedInts.DEFAULT); + } + docDeltasBuilder.add(doc - lastDoc); + bucketsBuilder.add(currentBucketOrdinal.get()); + lastDoc = doc; + + collectBucket(sub, doc, currentBucketOrdinal.get()); + } + + private void reset(int newTsidOrd, long bucket) throws IOException { + tsidOrd.set(newTsidOrd); + BytesRef tsid = tsids.lookupOrd(newTsidOrd); + + BytesRef bucketValue = needAggregator ? packKey(tsid) : tsid; + long bucketOrdinal = bucketOrds.add(bucket, bucketValue); + if (bucketOrdinal < 0) { // already seen + bucketOrdinal = -1 - bucketOrdinal; + grow(bucketOrdinal + 1); + } + currentBucketOrdinal.set(bucketOrdinal); + } + }; + } else { + return getCollector(sub, aggContext); + } + } + + protected LeafBucketCollector getCollector(LeafBucketCollector sub, AggregationExecutionContext aggCtx) throws IOException { + final SortedNumericDoubleValues values = valuesSource.doubleValues(aggCtx.getLeafReaderContext()); + CheckedConsumer docConsumer = (doc) -> { if (aggCtx.getTimestamp() + downsampleRange < preRounding) { return; } @@ -341,7 +538,7 @@ protected LeafBucketCollector getCollector( downsampleParams.put(Function.ROUNDING_FIELD, preRounding); timeBucketMetrics.put(preRounding, downsampleFunction.getFunction(downsampleParams)); } - for (Entry entry : timeBucketMetrics.entrySet()) { + for (Map.Entry entry : timeBucketMetrics.entrySet()) { Long timestamp = entry.getKey(); AggregatorFunction function = entry.getValue(); if (aggCtx.getTimestamp() + downsampleRange >= timestamp) { @@ -352,47 +549,48 @@ protected LeafBucketCollector getCollector( } } } - }); + }; + if (deferring) { + return new DeferringCollector(values, aggCtx, docConsumer); + } else { + return new Collector(sub, values, aggCtx, docConsumer); + } } @Override - protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBucketCollector sub) throws IOException { - // TODO: remove this method in a follow up PR - throw new UnsupportedOperationException("Shouldn't be here"); - } - - protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBucketCollector sub, AggregationExecutionContext aggCtx) - throws IOException { - if (valuesSource == null) { - return new LeafBucketCollector() { - @Override - public void setScorer(Scorable arg0) throws IOException { - // no-op - } + protected void doPostCollection() throws IOException { + if (deferring) { + finishLeaf(); + } else { + /** + * collect the last tsid + */ + if (timeBucketMetrics != null && timeBucketMetrics.size() > 0) { + collectTimeSeriesValues(preBucketOrdinal); + } + } - @Override - public void collect(int doc, long bucket) { - // no-op - } + } - @Override - public boolean isNoop() { - return false; - } - }; + /** + * Button up the builders for the current leaf. + */ + private void finishLeaf() { + if (aggCtx != null) { + assert docDeltasBuilder != null && bucketsBuilder != null; + assert docDeltasBuilder.size() > 0; + entries.add(new Entry(aggCtx, docDeltasBuilder.build(), bucketsBuilder.build())); + clearLeaf(); } - final SortedNumericDoubleValues values = valuesSource.doubleValues(context); - return getCollector(sub, aggCtx, values); } - @Override - protected void doPostCollection() throws IOException { - /** - * collect the last tsid - */ - if (timeBucketMetrics != null && timeBucketMetrics.size() > 0) { - collectTimeSeriesValues(preBucketOrdinal); - } + /** + * Clear the status for the current leaf. + */ + private void clearLeaf() { + aggCtx = null; + docDeltasBuilder = null; + bucketsBuilder = null; } InternalTimeSeriesAggregation buildResult(long otherDocCount, InternalTimeSeriesAggregation.InternalBucket[] topBuckets) { @@ -456,7 +654,7 @@ public void collectTimeSeriesValues(long bucketOrd) throws IOException { aggregatorCollectors.set(bucketOrd, aggregatorBucketFunction); } - for (Entry entry : timeBucketMetrics.entrySet()) { + for (Map.Entry entry : timeBucketMetrics.entrySet()) { Long timestamp = entry.getKey(); AggregatorFunction value = entry.getValue(); if (logger.isTraceEnabled()) { @@ -486,4 +684,134 @@ public void collectTimeSeriesValues(long bucketOrd) throws IOException { groupBucketValues.set(bucketOrd, tsids); } } + + static class Entry { + final AggregationExecutionContext aggCtx; + final PackedLongValues docDeltas; + final PackedLongValues buckets; + + Entry(AggregationExecutionContext aggCtx, PackedLongValues docDeltas, PackedLongValues buckets) { + this.aggCtx = Objects.requireNonNull(aggCtx); + this.docDeltas = Objects.requireNonNull(docDeltas); + this.buckets = Objects.requireNonNull(buckets); + } + } + + // Re-populate the queue with walkers on the same TSID. + private boolean populateQueue(List leafWalkers, PriorityQueue queue) throws IOException { + BytesRef currentTsid = null; + assert queue.size() == 0; + Iterator it = leafWalkers.iterator(); + while (it.hasNext()) { + LeafWalker leafWalker = it.next(); + if (leafWalker.docId == DocIdSetIterator.NO_MORE_DOCS) { + // If a walker is exhausted then we can remove it from consideration + // entirely + it.remove(); + continue; + } + BytesRef tsid = leafWalker.getTsid(); + if (currentTsid == null) { + currentTsid = tsid; + } + int comp = tsid.compareTo(currentTsid); + if (comp == 0) { + queue.add(leafWalker); + } else if (comp < 0) { + // We've found a walker on a lower TSID, so we remove all walkers + // collected so far from the queue and reset our comparison TSID + // to be the lower value + queue.clear(); + queue.add(leafWalker); + currentTsid = tsid; + } + } + assert queueAllHaveTsid(queue, currentTsid); + // If all walkers are exhausted then nothing will have been added to the queue + // and we're done + return queue.size() > 0; + } + + private static boolean queueAllHaveTsid(PriorityQueue queue, BytesRef tsid) throws IOException { + for (LeafWalker leafWalker : queue) { + BytesRef walkerId = leafWalker.tsids.lookupOrd(leafWalker.tsids.ordValue()); + assert walkerId.equals(tsid) : tsid.utf8ToString() + " != " + walkerId.utf8ToString(); + } + return true; + } + + class LeafWalker { + private final LeafBucketCollector collector; + private final SortedDocValues tsids; + private final SortedNumericDocValues timestamps; + private final BytesRefBuilder scratch = new BytesRefBuilder(); + int docId = 0; + long currentBucket = 0; + int tsidOrd; + long timestamp; + + final PackedLongValues.Iterator docDeltaIterator; + final PackedLongValues.Iterator buckets; + + LeafWalker(LeafReaderContext context, PackedLongValues.Iterator docDeltaIterator, PackedLongValues.Iterator buckets) + throws IOException { + this.docDeltaIterator = docDeltaIterator; + this.buckets = buckets; + AggregationExecutionContext aggContext = new AggregationExecutionContext(context, scratch::get, () -> timestamp); + this.collector = getCollector(null, aggContext); + tsids = DocValues.getSorted(context.reader(), TimeSeriesIdFieldMapper.NAME); + timestamps = DocValues.getSortedNumeric(context.reader(), DataStream.TimestampField.FIXED_TIMESTAMP_FIELD); + } + + void collectCurrent() throws IOException { + assert tsids.docID() == docId; + assert timestamps.docID() == docId; + collector.collect(docId, currentBucket); + } + + int nextDoc() throws IOException { + if (docId == DocIdSetIterator.NO_MORE_DOCS) { + return DocIdSetIterator.NO_MORE_DOCS; + } + + do { + if (false == docDeltaIterator.hasNext()) { + docId = DocIdSetIterator.NO_MORE_DOCS; + break; + } + docId += docDeltaIterator.next(); + final long bucket = buckets.next(); + final long rebasedBucket = selectedBuckets.find(bucket); + if (rebasedBucket != -1 && isValidDoc(docId)) { + currentBucket = bucket; + timestamp = timestamps.nextValue(); + break; + } + } while (docId != DocIdSetIterator.NO_MORE_DOCS); + return docId; + } + + private boolean isValidDoc(int docId) throws IOException { + return tsids.advanceExact(docId) && timestamps.advanceExact(docId); + } + + BytesRef getTsid() throws IOException { + tsidOrd = tsids.ordValue(); + scratch.copyBytes(tsids.lookupOrd(tsidOrd)); + return scratch.get(); + } + + public long getTimestamp() { + return timestamp; + } + + // true if the TSID ord has changed since the last time we checked + boolean shouldPop() throws IOException { + if (tsidOrd != tsids.ordValue()) { + return true; + } else { + return false; + } + } + } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorDeferring.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorDeferring.java deleted file mode 100644 index 0350bb4a43636..0000000000000 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorDeferring.java +++ /dev/null @@ -1,817 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0 and the Server Side Public License, v 1; you may not use this file except - * in compliance with, at your election, the Elastic License 2.0 or the Server - * Side Public License, v 1. - */ - -package org.elasticsearch.search.aggregations.timeseries.aggregation; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.apache.lucene.index.DocValues; -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.SortedDocValues; -import org.apache.lucene.index.SortedNumericDocValues; -import org.apache.lucene.search.CollectionTerminatedException; -import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.search.Scorable; -import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.BytesRefBuilder; -import org.apache.lucene.util.PriorityQueue; -import org.apache.lucene.util.packed.PackedInts; -import org.apache.lucene.util.packed.PackedLongValues; -import org.elasticsearch.cluster.metadata.DataStream; -import org.elasticsearch.common.util.BigArrays; -import org.elasticsearch.common.util.LongHash; -import org.elasticsearch.common.util.ObjectArray; -import org.elasticsearch.common.util.set.Sets; -import org.elasticsearch.core.CheckedConsumer; -import org.elasticsearch.core.Releasables; -import org.elasticsearch.index.TimestampBounds; -import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; -import org.elasticsearch.index.mapper.TimeSeriesIdFieldMapper; -import org.elasticsearch.search.DocValueFormat; -import org.elasticsearch.search.aggregations.AggregationExecutionContext; -import org.elasticsearch.search.aggregations.AggregatorFactories; -import org.elasticsearch.search.aggregations.BucketOrder; -import org.elasticsearch.search.aggregations.CardinalityUpperBound; -import org.elasticsearch.search.aggregations.InternalAggregation; -import org.elasticsearch.search.aggregations.InternalOrder; -import org.elasticsearch.search.aggregations.LeafBucketCollector; -import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; -import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; -import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; -import org.elasticsearch.search.aggregations.bucket.terms.BucketPriorityQueue; -import org.elasticsearch.search.aggregations.bucket.terms.BytesKeyedBucketOrds; -import org.elasticsearch.search.aggregations.bucket.terms.LongKeyedBucketOrds; -import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator; -import org.elasticsearch.search.aggregations.support.AggregationContext; -import org.elasticsearch.search.aggregations.support.ValuesSource; -import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; -import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.AggregatorBucketFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.bucketfunction.TSIDBucketFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; -import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TimeSeriesLineAggreagation; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Comparator; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Set; -import java.util.TreeMap; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; - -import static org.elasticsearch.search.DocValueFormat.TIME_SERIES_ID; -import static org.elasticsearch.search.aggregations.InternalOrder.isKeyOrder; - -@SuppressWarnings({ "unchecked", "rawtypes" }) -public class TimeSeriesAggregationAggregatorDeferring extends BucketsAggregator { - private static final Logger logger = LogManager.getLogger(TimeSeriesAggregationAggregatorDeferring.class); - - protected BytesKeyedBucketOrds bucketOrds; - private LongKeyedBucketOrds timestampOrds; - private ValuesSource.Numeric valuesSource; - - private boolean keyed; - - private Set group; - private Set without; - private long interval; - private long offset; - private Aggregator aggregator; - private Map aggregatorParams; - protected long downsampleRange; - protected Function downsampleFunction; - protected Map downsampleParams; - private BucketOrder order; - private TermsAggregator.BucketCountThresholds bucketCountThresholds; - protected Comparator partiallyBuiltBucketComparator; - protected DocValueFormat format; - private TimestampBounds timestampBounds; - private long startTime; - private long endTime; - protected boolean deferring; - - private BytesRef preTsid; - private long preBucketOrdinal; - protected long preRounding = -1; - private RoundingInterval rounding; - private boolean needAggregator; - protected Map timeBucketMetrics; // TODO replace map - private ObjectArray> groupBucketValues; // TODO replace map - private ObjectArray aggregatorCollectors; - - private List entries = new ArrayList<>(); - private AggregationExecutionContext aggCtx; - private PackedLongValues.Builder docDeltasBuilder; - private PackedLongValues.Builder bucketsBuilder; - private LongHash selectedBuckets; - - @SuppressWarnings("unchecked") - public TimeSeriesAggregationAggregatorDeferring( - String name, - AggregatorFactories factories, - boolean keyed, - List group, - List without, - DateHistogramInterval interval, - DateHistogramInterval offset, - Aggregator aggregator, - Map aggregatorParams, - Downsample downsample, - TermsAggregator.BucketCountThresholds bucketCountThresholds, - BucketOrder order, - long startTime, - long endTime, - boolean deferring, - ValuesSourceConfig valuesSourceConfig, - AggregationContext context, - org.elasticsearch.search.aggregations.Aggregator parent, - CardinalityUpperBound bucketCardinality, - Map metadata - ) throws IOException { - super(name, factories, context, parent, bucketCardinality, metadata); - this.keyed = keyed; - this.group = group != null ? Sets.newHashSet(group) : Set.of(); - this.without = without != null ? Sets.newHashSet(without) : Set.of(); - this.interval = interval != null ? interval.estimateMillis() : -1; - if (this.interval <= 0) { - throw new IllegalArgumentException("time_series_aggregation invalid interval [" + interval + "]"); - } - this.startTime = startTime; - this.endTime = endTime; - this.deferring = deferring; - this.rounding = new RoundingInterval(this.startTime, this.interval); - this.offset = offset != null ? offset.estimateMillis() : 0; - this.aggregator = aggregator; - this.aggregatorParams = aggregatorParams; - this.needAggregator = this.aggregator != null; - this.downsampleRange = downsample != null && downsample.getRange() != null ? downsample.getRange().estimateMillis() : -1; - this.downsampleFunction = downsample != null && downsample.getFunction() != null - ? downsample.getFunction() - : (downsampleRange > 0 ? Function.origin_value : Function.last); - if (this.downsampleRange <= 0) { - this.downsampleRange = this.interval; - } - this.downsampleParams = downsample != null && downsample.getParameters() != null - ? new HashMap<>(downsample.getParameters()) - : new HashMap<>(); - this.downsampleParams.put(Function.RANGE_FIELD, downsampleRange); - this.bucketCountThresholds = bucketCountThresholds; - this.order = order == null ? BucketOrder.key(true) : order; - this.partiallyBuiltBucketComparator = order.partiallyBuiltBucketComparator(b -> b.bucketOrd, this); - this.bucketOrds = BytesKeyedBucketOrds.build(bigArrays(), bucketCardinality); - this.timestampOrds = LongKeyedBucketOrds.build(bigArrays(), CardinalityUpperBound.MANY); - if (valuesSourceConfig != null) { - this.valuesSource = valuesSourceConfig.hasValues() ? (ValuesSource.Numeric) valuesSourceConfig.getValuesSource() : null; - this.format = valuesSourceConfig.format(); - } else { - this.valuesSource = null; - this.format = null; - } - this.timestampBounds = context.getIndexSettings().getTimestampBounds(); - - if ((this.group.size() > 0 || this.without.size() > 0) && this.aggregator == null) { - throw new IllegalArgumentException("time_series_aggregation group by must have an aggregator"); - } - - groupBucketValues = bigArrays().newObjectArray(1); - aggregatorCollectors = bigArrays().newObjectArray(1); - } - - @Override - public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { - InternalTimeSeriesAggregation.InternalBucket[][] allBucketsPerOrd = - new InternalTimeSeriesAggregation.InternalBucket[owningBucketOrds.length][]; - long[] otherDocCounts = new long[owningBucketOrds.length]; - for (int ordIdx = 0; ordIdx < owningBucketOrds.length; ordIdx++) { - BytesRef spareKey = new BytesRef(); - BytesKeyedBucketOrds.BucketOrdsEnum ordsEnum = bucketOrds.ordsEnum(owningBucketOrds[ordIdx]); - long bucketsInOrd = bucketOrds.bucketsInOrd(owningBucketOrds[ordIdx]); - int size = (int) Math.min(bucketsInOrd, bucketCountThresholds.getShardSize()); - PriorityQueue ordered = new BucketPriorityQueue<>( - size, - partiallyBuiltBucketComparator - ); - while (ordsEnum.next()) { - long docCount = bucketDocCount(ordsEnum.ord()); - otherDocCounts[ordIdx] += docCount; - if (docCount < bucketCountThresholds.getShardMinDocCount()) { - continue; - } - ordsEnum.readValue(spareKey); - long ord = ordsEnum.ord(); - InternalTimeSeriesAggregation.InternalBucket bucket = new InternalTimeSeriesAggregation.InternalBucket( - TimeSeriesIdFieldMapper.decodeTsid(spareKey), - docCount, - null, - null, - keyed, - false, - 0 - ); - bucket.bucketOrd = ord; - ordered.insertWithOverflow(bucket); - } - - // Get the top buckets - InternalTimeSeriesAggregation.InternalBucket[] bucketsForOrd = new InternalTimeSeriesAggregation.InternalBucket[ordered.size()]; - allBucketsPerOrd[ordIdx] = bucketsForOrd; - List bucketList = new ArrayList<>(); - for (int b = ordered.size() - 1; b >= 0; --b) { - InternalTimeSeriesAggregation.InternalBucket bucket = ordered.pop(); - allBucketsPerOrd[ordIdx][b] = bucket; - otherDocCounts[ordIdx] -= allBucketsPerOrd[ordIdx][b].getDocCount(); - bucketList.add(bucket); - } - - if (deferring) { - if (this.selectedBuckets != null) { - throw new IllegalStateException("Already been replayed"); - } - this.selectedBuckets = new LongHash(ordered.size(), BigArrays.NON_RECYCLING_INSTANCE); - for (InternalTimeSeriesAggregation.InternalBucket bucket : bucketList) { - selectedBuckets.add(bucket.bucketOrd); - } - - PriorityQueue queue = new PriorityQueue<>(entries.size()) { - @Override - protected boolean lessThan(LeafWalker a, LeafWalker b) { - return a.getTimestamp() > b.getTimestamp(); - } - }; - - List leafWalkers = new ArrayList<>(); - for (Entry entry : entries) { - assert entry.docDeltas.size() > 0 : "segment should have at least one document to replay, got 0"; - try { - final PackedLongValues.Iterator docDeltaIterator = entry.docDeltas.iterator(); - final PackedLongValues.Iterator buckets = entry.buckets.iterator(); - LeafWalker leafWalker = new LeafWalker(entry.aggCtx.getLeafReaderContext(), docDeltaIterator, buckets); - if (leafWalker.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) { - leafWalkers.add(leafWalker); - } - } catch (CollectionTerminatedException e) { - // collection was terminated prematurely - // continue with the following leaf - } - } - - while (populateQueue(leafWalkers, queue)) { - do { - LeafWalker walker = queue.top(); - walker.collectCurrent(); - if (walker.nextDoc() == DocIdSetIterator.NO_MORE_DOCS || walker.shouldPop()) { - queue.pop(); - } else { - queue.updateTop(); - } - } while (queue.size() > 0); - } - - /** - * collect the last tsid - */ - if (timeBucketMetrics != null && timeBucketMetrics.size() > 0) { - collectTimeSeriesValues(preBucketOrdinal); - } - } - - for (InternalTimeSeriesAggregation.InternalBucket bucket : bucketList) { - long ord = bucket.bucketOrd; - Map values = new LinkedHashMap<>(); - if (needAggregator) { - AggregatorBucketFunction aggregatorBucketFunction = aggregatorCollectors.get(ord); - LongKeyedBucketOrds.BucketOrdsEnum timeOrdsEnum = timestampOrds.ordsEnum(ord); - while (timeOrdsEnum.next()) { - values.put( - timeOrdsEnum.value() + offset, - aggregatorBucketFunction.getAggregation(timeOrdsEnum.ord(), aggregatorParams, format, metadata()) - ); - } - } else { - values = groupBucketValues.get(ord); - if (values == null) { - values = new LinkedHashMap<>(); - } - } - bucket.metricAggregation = new TimeSeriesLineAggreagation(TimeSeriesLineAggreagation.NAME, values, format, metadata()); - } - } - - buildSubAggsForAllBuckets(allBucketsPerOrd, b -> b.bucketOrd, (b, a) -> b.aggregations = a); - - InternalAggregation[] result = new InternalAggregation[owningBucketOrds.length]; - for (int ordIdx = 0; ordIdx < owningBucketOrds.length; ordIdx++) { - result[ordIdx] = buildResult(otherDocCounts[ordIdx], allBucketsPerOrd[ordIdx]); - } - return result; - } - - @Override - public InternalAggregation buildEmptyAggregation() { - return new InternalTimeSeriesAggregation( - name, - order, - order, - bucketCountThresholds.getRequiredSize(), - bucketCountThresholds.getMinDocCount(), - bucketCountThresholds.getShardSize(), - false, - 0, - new ArrayList<>(), - false, - 0, - metadata() - ); - } - - @Override - protected void doClose() { - Releasables.close(bucketOrds); - Releasables.close(timestampOrds); - Releasables.close(groupBucketValues); - for (int i = 0; i < aggregatorCollectors.size(); i++) { - AggregatorBucketFunction aggregatorBucketFunction = aggregatorCollectors.get(i); - if (aggregatorBucketFunction != null) { - aggregatorBucketFunction.close(); - } - } - Releasables.close(aggregatorCollectors); - } - - public class Collector extends LeafBucketCollectorBase { - final SortedNumericDoubleValues values; - final AggregationExecutionContext aggCtx; - final LeafBucketCollector sub; - final CheckedConsumer docConsumer; - - public Collector( - LeafBucketCollector sub, - SortedNumericDoubleValues values, - AggregationExecutionContext aggCtx, - CheckedConsumer docConsumer - ) { - super(sub, values); - this.sub = sub; - this.values = values; - this.aggCtx = aggCtx; - this.docConsumer = docConsumer; - } - - @Override - public void collect(int doc, long bucket) throws IOException { - BytesRef newTsid = aggCtx.getTsid(); - - if (preTsid == null) { - reset(newTsid, bucket); - } else if (false == preTsid.equals(newTsid)) { - collectTimeSeriesValues(preBucketOrdinal); - reset(newTsid, bucket); - } - - if (preRounding < 0 || aggCtx.getTimestamp() < preRounding - interval) { - preRounding = rounding.nextRoundingValue(aggCtx.getTimestamp()); - } - - // calculate the value of the current doc - docConsumer.accept(doc); - collectBucket(sub, doc, preBucketOrdinal); - } - - private void reset(BytesRef tsid, long bucket) { - timeBucketMetrics = new TreeMap<>(); - preTsid = BytesRef.deepCopyOf(tsid); - preRounding = -1; - - BytesRef bucketValue = needAggregator ? packKey(preTsid) : preTsid; - long bucketOrdinal = bucketOrds.add(bucket, bucketValue); - if (bucketOrdinal < 0) { // already seen - bucketOrdinal = -1 - bucketOrdinal; - grow(bucketOrdinal + 1); - } - preBucketOrdinal = bucketOrdinal; - } - } - - public class DeferringCollector extends LeafBucketCollector { - final SortedNumericDoubleValues values; - final AggregationExecutionContext aggCtx; - final CheckedConsumer docConsumer; - - public DeferringCollector( - SortedNumericDoubleValues values, - AggregationExecutionContext aggCtx, - CheckedConsumer docConsumer - ) { - this.values = values; - this.aggCtx = aggCtx; - this.docConsumer = docConsumer; - } - - @Override - public void collect(int doc, long bucket) throws IOException { - BytesRef newTsid = aggCtx.getTsid(); - - if (preTsid == null) { - reset(newTsid, bucket); - } else if (false == preTsid.equals(newTsid)) { - collectTimeSeriesValues(preBucketOrdinal); - reset(newTsid, bucket); - } - - if (preRounding < 0 || aggCtx.getTimestamp() < preRounding - interval) { - preRounding = rounding.nextRoundingValue(aggCtx.getTimestamp()); - } - - // calculate the value of the current doc - docConsumer.accept(doc); - } - - private void reset(BytesRef tsid, long bucket) { - timeBucketMetrics = new TreeMap<>(); - preTsid = BytesRef.deepCopyOf(tsid); - preRounding = -1; - preBucketOrdinal = bucket; - } - } - - @Override - protected LeafBucketCollector getLeafCollector(AggregationExecutionContext context, LeafBucketCollector sub) throws IOException { - if (valuesSource == null) { - return new LeafBucketCollector() { - @Override - public void setScorer(Scorable arg0) throws IOException { - // no-op - } - - @Override - public void collect(int doc, long bucket) { - // no-op - } - - @Override - public boolean isNoop() { - return false; - } - }; - } - - return getLeafCollectorInternal(context.getLeafReaderContext(), sub, context); - } - - protected LeafBucketCollector getLeafCollectorInternal( - LeafReaderContext context, - LeafBucketCollector sub, - AggregationExecutionContext aggContext - ) throws IOException { - if (deferring) { - SortedDocValues tsids = DocValues.getSorted(context.reader(), TimeSeriesIdFieldMapper.NAME); - final AtomicInteger tsidOrd = new AtomicInteger(-1); - final AtomicLong currentBucketOrdinal = new AtomicLong(); - finishLeaf(); - return new LeafBucketCollectorBase(sub, null) { - int lastDoc = 0; - - @Override - public void collect(int doc, long bucket) throws IOException { - if (tsids.advanceExact(doc)) { - int newTsidOrd = tsids.ordValue(); - if (tsidOrd.get() < 0) { - reset(newTsidOrd, bucket); - } else if (tsidOrd.get() != newTsidOrd) { - reset(newTsidOrd, bucket); - } - } - - if (aggCtx == null) { - aggCtx = aggContext; - docDeltasBuilder = PackedLongValues.packedBuilder(PackedInts.DEFAULT); - bucketsBuilder = PackedLongValues.packedBuilder(PackedInts.DEFAULT); - } - docDeltasBuilder.add(doc - lastDoc); - bucketsBuilder.add(currentBucketOrdinal.get()); - lastDoc = doc; - - collectBucket(sub, doc, currentBucketOrdinal.get()); - } - - private void reset(int newTsidOrd, long bucket) throws IOException { - tsidOrd.set(newTsidOrd); - BytesRef tsid = tsids.lookupOrd(newTsidOrd); - - BytesRef bucketValue = needAggregator ? packKey(tsid) : tsid; - long bucketOrdinal = bucketOrds.add(bucket, bucketValue); - if (bucketOrdinal < 0) { // already seen - bucketOrdinal = -1 - bucketOrdinal; - grow(bucketOrdinal + 1); - } - currentBucketOrdinal.set(bucketOrdinal); - } - }; - } else { - return getCollector(sub, aggContext); - } - } - - protected LeafBucketCollector getCollector(LeafBucketCollector sub, AggregationExecutionContext aggCtx) throws IOException { - final SortedNumericDoubleValues values = valuesSource.doubleValues(aggCtx.getLeafReaderContext()); - CheckedConsumer docConsumer = (doc) -> { - if (aggCtx.getTimestamp() + downsampleRange < preRounding) { - return; - } - - if (values.advanceExact(doc)) { - final int valuesCount = values.docValueCount(); - for (int i = 0; i < valuesCount; i++) { - double value = values.nextValue(); - if (false == timeBucketMetrics.containsKey(preRounding)) { - downsampleParams.put(Function.ROUNDING_FIELD, preRounding); - timeBucketMetrics.put(preRounding, downsampleFunction.getFunction(downsampleParams)); - } - for (Map.Entry entry : timeBucketMetrics.entrySet()) { - Long timestamp = entry.getKey(); - AggregatorFunction function = entry.getValue(); - if (aggCtx.getTimestamp() + downsampleRange >= timestamp) { - function.collect(new TimePoint(aggCtx.getTimestamp(), value)); - } else { - break; - } - } - } - } - }; - if (deferring) { - return new DeferringCollector(values, aggCtx, docConsumer); - } else { - return new Collector(sub, values, aggCtx, docConsumer); - } - } - - @Override - protected void doPostCollection() throws IOException { - if (deferring) { - finishLeaf(); - } else { - /** - * collect the last tsid - */ - if (timeBucketMetrics != null && timeBucketMetrics.size() > 0) { - collectTimeSeriesValues(preBucketOrdinal); - } - } - - } - - /** - * Button up the builders for the current leaf. - */ - private void finishLeaf() { - if (aggCtx != null) { - assert docDeltasBuilder != null && bucketsBuilder != null; - assert docDeltasBuilder.size() > 0; - entries.add(new Entry(aggCtx, docDeltasBuilder.build(), bucketsBuilder.build())); - clearLeaf(); - } - } - - /** - * Clear the status for the current leaf. - */ - private void clearLeaf() { - aggCtx = null; - docDeltasBuilder = null; - bucketsBuilder = null; - } - - InternalTimeSeriesAggregation buildResult(long otherDocCount, InternalTimeSeriesAggregation.InternalBucket[] topBuckets) { - final BucketOrder reduceOrder; - if (isKeyOrder(order) == false) { - reduceOrder = InternalOrder.key(true); - Arrays.sort(topBuckets, reduceOrder.comparator()); - } else { - reduceOrder = order; - } - return new InternalTimeSeriesAggregation( - name, - reduceOrder, - order, - bucketCountThresholds.getRequiredSize(), - bucketCountThresholds.getMinDocCount(), - bucketCountThresholds.getShardSize(), - false, - otherDocCount, - List.of(topBuckets), - keyed, - 0, - metadata() - ); - } - - /** - * decode the tsid and pack the bucket key from the group and without config - */ - private BytesRef packKey(BytesRef tsid) { - if (group.size() == 0 && without.size() == 0) { - return new BytesRef(new byte[] { 0 }); - } - - Map tsidMap = TimeSeriesIdFieldMapper.decodeTsid(tsid); - Map groupMap = new LinkedHashMap<>(); - tsidMap.forEach((key, value) -> { - if (group.size() > 0) { - if (group.contains(key) && false == without.contains(key)) { - groupMap.put(key, value); - } - } else if (without.size() > 0) { - if (false == without.contains(key)) { - groupMap.put(key, value); - } - } - }); - return TIME_SERIES_ID.parseBytesRef(groupMap); - } - - /** - * collect the value of one time series line - */ - public void collectTimeSeriesValues(long bucketOrd) throws IOException { - if (needAggregator) { - aggregatorCollectors = bigArrays().grow(aggregatorCollectors, bucketOrd + 1); - AggregatorBucketFunction aggregatorBucketFunction = aggregatorCollectors.get(bucketOrd); - if (aggregatorBucketFunction == null) { - AggregatorBucketFunction internal = aggregator.getAggregatorBucketFunction(bigArrays(), aggregatorParams); - aggregatorBucketFunction = new TSIDBucketFunction(bigArrays(), internal); - aggregatorCollectors.set(bucketOrd, aggregatorBucketFunction); - } - - for (Map.Entry entry : timeBucketMetrics.entrySet()) { - Long timestamp = entry.getKey(); - AggregatorFunction value = entry.getValue(); - if (logger.isTraceEnabled()) { - logger.trace( - "collect time_series, time={}, value={}, tsid={}, hashcode={}", - timestamp, - value.get(), - TimeSeriesIdFieldMapper.decodeTsid(preTsid), - this.hashCode() - ); - } - - long ord = timestampOrds.add(bucketOrd, timestamp); - if (ord < 0) { // already seen - ord = -1 - ord; - } - if (timestamp - interval <= timestampBounds.startTime() || timestamp > timestampBounds.endTime()) { - aggregatorBucketFunction.collect(new TSIDValue(preTsid, value.getAggregation(format, metadata())), ord); - } else { - aggregatorBucketFunction.collect(new TSIDValue(preTsid, value.get()), ord); - } - } - } else { - Map tsids = new LinkedHashMap<>(); - timeBucketMetrics.forEach((k, v) -> { tsids.put(k + offset, v.getAggregation(format, metadata())); }); - groupBucketValues = bigArrays().grow(groupBucketValues, bucketOrd + 1); - groupBucketValues.set(bucketOrd, tsids); - } - } - - static class Entry { - final AggregationExecutionContext aggCtx; - final PackedLongValues docDeltas; - final PackedLongValues buckets; - - Entry(AggregationExecutionContext aggCtx, PackedLongValues docDeltas, PackedLongValues buckets) { - this.aggCtx = Objects.requireNonNull(aggCtx); - this.docDeltas = Objects.requireNonNull(docDeltas); - this.buckets = Objects.requireNonNull(buckets); - } - } - - // Re-populate the queue with walkers on the same TSID. - private boolean populateQueue(List leafWalkers, PriorityQueue queue) throws IOException { - BytesRef currentTsid = null; - assert queue.size() == 0; - Iterator it = leafWalkers.iterator(); - while (it.hasNext()) { - LeafWalker leafWalker = it.next(); - if (leafWalker.docId == DocIdSetIterator.NO_MORE_DOCS) { - // If a walker is exhausted then we can remove it from consideration - // entirely - it.remove(); - continue; - } - BytesRef tsid = leafWalker.getTsid(); - if (currentTsid == null) { - currentTsid = tsid; - } - int comp = tsid.compareTo(currentTsid); - if (comp == 0) { - queue.add(leafWalker); - } else if (comp < 0) { - // We've found a walker on a lower TSID, so we remove all walkers - // collected so far from the queue and reset our comparison TSID - // to be the lower value - queue.clear(); - queue.add(leafWalker); - currentTsid = tsid; - } - } - assert queueAllHaveTsid(queue, currentTsid); - // If all walkers are exhausted then nothing will have been added to the queue - // and we're done - return queue.size() > 0; - } - - private static boolean queueAllHaveTsid(PriorityQueue queue, BytesRef tsid) throws IOException { - for (LeafWalker leafWalker : queue) { - BytesRef walkerId = leafWalker.tsids.lookupOrd(leafWalker.tsids.ordValue()); - assert walkerId.equals(tsid) : tsid.utf8ToString() + " != " + walkerId.utf8ToString(); - } - return true; - } - - class LeafWalker { - private final LeafBucketCollector collector; - private final SortedDocValues tsids; - private final SortedNumericDocValues timestamps; - private final BytesRefBuilder scratch = new BytesRefBuilder(); - int docId = 0; - long currentBucket = 0; - int tsidOrd; - long timestamp; - - final PackedLongValues.Iterator docDeltaIterator; - final PackedLongValues.Iterator buckets; - - LeafWalker(LeafReaderContext context, PackedLongValues.Iterator docDeltaIterator, PackedLongValues.Iterator buckets) - throws IOException { - this.docDeltaIterator = docDeltaIterator; - this.buckets = buckets; - AggregationExecutionContext aggContext = new AggregationExecutionContext(context, scratch::get, () -> timestamp); - this.collector = getCollector(null, aggContext); - tsids = DocValues.getSorted(context.reader(), TimeSeriesIdFieldMapper.NAME); - timestamps = DocValues.getSortedNumeric(context.reader(), DataStream.TimestampField.FIXED_TIMESTAMP_FIELD); - } - - void collectCurrent() throws IOException { - assert tsids.docID() == docId; - assert timestamps.docID() == docId; - collector.collect(docId, currentBucket); - } - - int nextDoc() throws IOException { - if (docId == DocIdSetIterator.NO_MORE_DOCS) { - return DocIdSetIterator.NO_MORE_DOCS; - } - - do { - if (false == docDeltaIterator.hasNext()) { - docId = DocIdSetIterator.NO_MORE_DOCS; - break; - } - docId += docDeltaIterator.next(); - final long bucket = buckets.next(); - final long rebasedBucket = selectedBuckets.find(bucket); - if (rebasedBucket != -1 && isValidDoc(docId)) { - currentBucket = bucket; - timestamp = timestamps.nextValue(); - break; - } - } while (docId != DocIdSetIterator.NO_MORE_DOCS); - return docId; - } - - private boolean isValidDoc(int docId) throws IOException { - return tsids.advanceExact(docId) && timestamps.advanceExact(docId); - } - - BytesRef getTsid() throws IOException { - tsidOrd = tsids.ordValue(); - scratch.copyBytes(tsids.lookupOrd(tsidOrd)); - return scratch.get(); - } - - public long getTimestamp() { - return timestamp; - } - - // true if the TSID ord has changed since the last time we checked - boolean shouldPop() throws IOException { - if (tsidOrd != tsids.ordValue()) { - return true; - } else { - return false; - } - } - } -} diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java index a7755ef8652b7..b1834644aa90a 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java @@ -7,7 +7,6 @@ package org.elasticsearch.xpack.aggregatemetric.aggregations.metrics; -import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.Scorable; import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.Tuple; @@ -26,7 +25,6 @@ import org.elasticsearch.search.aggregations.timeseries.aggregation.Function; import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregationAggregator; -import org.elasticsearch.search.aggregations.timeseries.aggregation.TimeSeriesAggregationAggregatorDeferring; import org.elasticsearch.search.aggregations.timeseries.aggregation.function.AggregatorFunction; import org.elasticsearch.xpack.aggregatemetric.aggregations.support.AggregateMetricsValuesSource; import org.elasticsearch.xpack.aggregatemetric.mapper.AggregateDoubleMetricFieldMapper.Metric; @@ -37,7 +35,7 @@ import java.util.Map.Entry; @SuppressWarnings({ "unchecked", "rawtypes" }) -public class AggregateMetricTimeSeriesAggregationAggregator extends TimeSeriesAggregationAggregatorDeferring { +public class AggregateMetricTimeSeriesAggregationAggregator extends TimeSeriesAggregationAggregator { private final AggregateMetricsValuesSource.AggregateDoubleMetric valuesSource; public AggregateMetricTimeSeriesAggregationAggregator( From 9bf11ac7f7e950d43a24c0ca8bbe090da9acaea0 Mon Sep 17 00:00:00 2001 From: weizijun Date: Fri, 19 Aug 2022 14:44:42 +0800 Subject: [PATCH 52/53] remove deferring parameter --- .../TimeSeriesAggregationsIT.java | 12 +- ...meSeriesAggregationAggregationBuilder.java | 19 +- ...meSeriesAggregationAggregationFactory.java | 5 - .../TimeSeriesAggregationAggregator.java | 241 ++++++------------ ...meSeriesAggregationAggregatorSupplier.java | 1 - ...MetricTimeSeriesAggregationAggregator.java | 15 +- 6 files changed, 86 insertions(+), 207 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java index 397dca243cb32..1e6ee4b3565a7 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/TimeSeriesAggregationsIT.java @@ -535,7 +535,7 @@ public void testBasicTimeSeriesAggregations() { Rounding.Prepared rounding = Rounding.builder(new TimeValue(fixedInterval.estimateMillis())).build().prepareForUnknown(); SearchResponse response = client().prepareSearch("index") .setSize(0) - .addAggregation(timeSeriesAggregation("by_ts").field("metric_0").interval(fixedInterval).deferring(randomBoolean()).size(data.size())) + .addAggregation(timeSeriesAggregation("by_ts").field("metric_0").interval(fixedInterval).size(data.size())) .get(); Aggregations aggregations = response.getAggregations(); assertNotNull(aggregations); @@ -579,7 +579,6 @@ public void testTimeSeriesAggregationsDownsample() { timeSeriesAggregation("by_ts").field("metric_0") .interval(fixedInterval) .downsample(fixedInterval, Function.sum_over_time, null) - .deferring(randomBoolean()) .size(data.size()) ) .get(); @@ -619,13 +618,7 @@ public void testTimeSeriesAggregationsAggregator() { Rounding.Prepared rounding = Rounding.builder(new TimeValue(fixedInterval.estimateMillis())).build().prepareForUnknown(); SearchResponse response = client().prepareSearch("index") .setSize(0) - .addAggregation( - timeSeriesAggregation("by_ts").field("metric_0") - .interval(fixedInterval) - .aggregator("sum") - .deferring(randomBoolean()) - .size(data.size()) - ) + .addAggregation(timeSeriesAggregation("by_ts").field("metric_0").interval(fixedInterval).aggregator("sum").size(data.size())) .get(); Aggregations aggregations = response.getAggregations(); assertNotNull(aggregations); @@ -683,7 +676,6 @@ public void testTimeSeriesAggregationsGroupBy() { .interval(fixedInterval) .downsample(fixedInterval, Function.max_over_time, null) .aggregator("sum") - .deferring(randomBoolean()) .size(data.size()) ) .get(); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java index 293ae65cb2644..d82aef6e04303 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java @@ -52,7 +52,6 @@ public class TimeSeriesAggregationAggregationBuilder extends ValuesSourceAggrega public static final ParseField SHARD_MIN_DOC_COUNT_FIELD = new ParseField("shard_min_doc_count"); public static final ParseField START_TIME_FIELD = new ParseField("start_time"); public static final ParseField END_TIME_FIELD = new ParseField("end_time"); - public static final ParseField DEFERRING_FIELD = new ParseField("deferring"); static final TermsAggregator.BucketCountThresholds DEFAULT_BUCKET_COUNT_THRESHOLDS = new TermsAggregator.BucketCountThresholds( 1, @@ -85,7 +84,6 @@ public class TimeSeriesAggregationAggregationBuilder extends ValuesSourceAggrega private BucketOrder order = BucketOrder.key(true); private Long startTime; private Long endTime; - private boolean deferring = true; static { ValuesSourceAggregationBuilder.declareFields(PARSER, false, true, false); @@ -125,7 +123,6 @@ public class TimeSeriesAggregationAggregationBuilder extends ValuesSourceAggrega PARSER.declareInt(TimeSeriesAggregationAggregationBuilder::shardMinDocCount, SHARD_MIN_DOC_COUNT_FIELD); PARSER.declareLong(TimeSeriesAggregationAggregationBuilder::startTime, START_TIME_FIELD); PARSER.declareLong(TimeSeriesAggregationAggregationBuilder::endTime, END_TIME_FIELD); - PARSER.declareBoolean(TimeSeriesAggregationAggregationBuilder::deferring, DEFERRING_FIELD); } public TimeSeriesAggregationAggregationBuilder(String name) { @@ -150,7 +147,6 @@ protected TimeSeriesAggregationAggregationBuilder( this.bucketCountThresholds = clone.bucketCountThresholds; this.startTime = clone.startTime; this.endTime = clone.endTime; - this.deferring = clone.deferring; } public TimeSeriesAggregationAggregationBuilder(StreamInput in) throws IOException { @@ -167,7 +163,6 @@ public TimeSeriesAggregationAggregationBuilder(StreamInput in) throws IOExceptio bucketCountThresholds = new TermsAggregator.BucketCountThresholds(in); startTime = in.readOptionalLong(); endTime = in.readOptionalLong(); - deferring = in.readBoolean(); } @Override @@ -184,7 +179,6 @@ protected void innerWriteTo(StreamOutput out) throws IOException { bucketCountThresholds.writeTo(out); out.writeOptionalLong(startTime); out.writeOptionalLong(endTime); - out.writeBoolean(deferring); } @Override @@ -223,7 +217,6 @@ protected ValuesSourceAggregatorFactory innerBuild( order, startTime != null ? startTime : -1, endTime != null ? endTime : -1, - deferring, config, context, parent, @@ -266,7 +259,6 @@ protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) if (endTime != null) { builder.field(END_TIME_FIELD.getPreferredName(), endTime); } - builder.field(DEFERRING_FIELD.getPreferredName(), deferring); return builder; } @@ -287,7 +279,7 @@ public String getType() { @Override public boolean isInSortOrderExecutionRequired() { - return false == deferring; + return false; } /** @@ -547,15 +539,6 @@ public TimeSeriesAggregationAggregationBuilder endTime(long endTime) { return this; } - public boolean isDeferring() { - return deferring; - } - - public TimeSeriesAggregationAggregationBuilder deferring(boolean deferring) { - this.deferring = deferring; - return this; - } - @Override public boolean equals(Object o) { if (this == o) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java index ebed2951d59cd..25bef1563e214 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java @@ -40,7 +40,6 @@ public class TimeSeriesAggregationAggregationFactory extends ValuesSourceAggrega private final TermsAggregator.BucketCountThresholds bucketCountThresholds; private final long startTime; private final long endTime; - private boolean deferring; private final BucketOrder order; private final ValuesSourceConfig config; private final TimeSeriesAggregationAggregatorSupplier aggregatorSupplier; @@ -59,7 +58,6 @@ public TimeSeriesAggregationAggregationFactory( BucketOrder order, long startTime, long endTime, - boolean deferring, ValuesSourceConfig config, AggregationContext context, AggregatorFactory parent, @@ -79,7 +77,6 @@ public TimeSeriesAggregationAggregationFactory( this.bucketCountThresholds = bucketCountThresholds; this.startTime = startTime; this.endTime = endTime; - this.deferring = deferring; this.order = order; this.config = config; this.aggregatorSupplier = aggregatorSupplier; @@ -120,7 +117,6 @@ protected Aggregator createUnmapped(Aggregator parent, Map metad order, startTime, endTime, - deferring, config, context, parent, @@ -156,7 +152,6 @@ protected Aggregator doCreateInternal(Aggregator parent, CardinalityUpperBound c order, startTime, endTime, - deferring, config, context, parent, diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index fc8758810a78c..c4df98831801a 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -99,7 +99,6 @@ public class TimeSeriesAggregationAggregator extends BucketsAggregator { private TimestampBounds timestampBounds; private long startTime; private long endTime; - protected boolean deferring; private BytesRef preTsid; private long preBucketOrdinal; @@ -132,7 +131,6 @@ public TimeSeriesAggregationAggregator( BucketOrder order, long startTime, long endTime, - boolean deferring, ValuesSourceConfig valuesSourceConfig, AggregationContext context, org.elasticsearch.search.aggregations.Aggregator parent, @@ -149,7 +147,6 @@ public TimeSeriesAggregationAggregator( } this.startTime = startTime; this.endTime = endTime; - this.deferring = deferring; this.rounding = new RoundingInterval(this.startTime, this.interval); this.offset = offset != null ? offset.estimateMillis() : 0; this.aggregator = aggregator; @@ -223,6 +220,11 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I ordered.insertWithOverflow(bucket); } + if (this.selectedBuckets != null) { + throw new IllegalStateException("Already been replayed"); + } + this.selectedBuckets = new LongHash(ordered.size(), BigArrays.NON_RECYCLING_INSTANCE); + // Get the top buckets InternalTimeSeriesAggregation.InternalBucket[] bucketsForOrd = new InternalTimeSeriesAggregation.InternalBucket[ordered.size()]; allBucketsPerOrd[ordIdx] = bucketsForOrd; @@ -232,58 +234,49 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I allBucketsPerOrd[ordIdx][b] = bucket; otherDocCounts[ordIdx] -= allBucketsPerOrd[ordIdx][b].getDocCount(); bucketList.add(bucket); + selectedBuckets.add(bucket.bucketOrd); } - if (deferring) { - if (this.selectedBuckets != null) { - throw new IllegalStateException("Already been replayed"); - } - this.selectedBuckets = new LongHash(ordered.size(), BigArrays.NON_RECYCLING_INSTANCE); - for (InternalTimeSeriesAggregation.InternalBucket bucket : bucketList) { - selectedBuckets.add(bucket.bucketOrd); + PriorityQueue queue = new PriorityQueue<>(entries.size()) { + @Override + protected boolean lessThan(LeafWalker a, LeafWalker b) { + return a.getTimestamp() > b.getTimestamp(); } + }; - PriorityQueue queue = new PriorityQueue<>(entries.size()) { - @Override - protected boolean lessThan(LeafWalker a, LeafWalker b) { - return a.getTimestamp() > b.getTimestamp(); - } - }; - - List leafWalkers = new ArrayList<>(); - for (Entry entry : entries) { - assert entry.docDeltas.size() > 0 : "segment should have at least one document to replay, got 0"; - try { - final PackedLongValues.Iterator docDeltaIterator = entry.docDeltas.iterator(); - final PackedLongValues.Iterator buckets = entry.buckets.iterator(); - LeafWalker leafWalker = new LeafWalker(entry.aggCtx.getLeafReaderContext(), docDeltaIterator, buckets); - if (leafWalker.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) { - leafWalkers.add(leafWalker); - } - } catch (CollectionTerminatedException e) { - // collection was terminated prematurely - // continue with the following leaf + List leafWalkers = new ArrayList<>(); + for (Entry entry : entries) { + assert entry.docDeltas.size() > 0 : "segment should have at least one document to replay, got 0"; + try { + final PackedLongValues.Iterator docDeltaIterator = entry.docDeltas.iterator(); + final PackedLongValues.Iterator buckets = entry.buckets.iterator(); + LeafWalker leafWalker = new LeafWalker(entry.aggCtx.getLeafReaderContext(), docDeltaIterator, buckets); + if (leafWalker.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) { + leafWalkers.add(leafWalker); } + } catch (CollectionTerminatedException e) { + // collection was terminated prematurely + // continue with the following leaf } + } - while (populateQueue(leafWalkers, queue)) { - do { - LeafWalker walker = queue.top(); - walker.collectCurrent(); - if (walker.nextDoc() == DocIdSetIterator.NO_MORE_DOCS || walker.shouldPop()) { - queue.pop(); - } else { - queue.updateTop(); - } - } while (queue.size() > 0); - } + while (populateQueue(leafWalkers, queue)) { + do { + LeafWalker walker = queue.top(); + walker.collectCurrent(); + if (walker.nextDoc() == DocIdSetIterator.NO_MORE_DOCS || walker.shouldPop()) { + queue.pop(); + } else { + queue.updateTop(); + } + } while (queue.size() > 0); + } - /** - * collect the last tsid - */ - if (timeBucketMetrics != null && timeBucketMetrics.size() > 0) { - collectTimeSeriesValues(preBucketOrdinal); - } + /** + * collect the last tsid + */ + if (timeBucketMetrics != null && timeBucketMetrics.size() > 0) { + collectTimeSeriesValues(preBucketOrdinal); } for (InternalTimeSeriesAggregation.InternalBucket bucket : bucketList) { @@ -349,60 +342,6 @@ protected void doClose() { Releasables.close(aggregatorCollectors); } - public class Collector extends LeafBucketCollectorBase { - final SortedNumericDoubleValues values; - final AggregationExecutionContext aggCtx; - final LeafBucketCollector sub; - final CheckedConsumer docConsumer; - - public Collector( - LeafBucketCollector sub, - SortedNumericDoubleValues values, - AggregationExecutionContext aggCtx, - CheckedConsumer docConsumer - ) { - super(sub, values); - this.sub = sub; - this.values = values; - this.aggCtx = aggCtx; - this.docConsumer = docConsumer; - } - - @Override - public void collect(int doc, long bucket) throws IOException { - BytesRef newTsid = aggCtx.getTsid(); - - if (preTsid == null) { - reset(newTsid, bucket); - } else if (false == preTsid.equals(newTsid)) { - collectTimeSeriesValues(preBucketOrdinal); - reset(newTsid, bucket); - } - - if (preRounding < 0 || aggCtx.getTimestamp() < preRounding - interval) { - preRounding = rounding.nextRoundingValue(aggCtx.getTimestamp()); - } - - // calculate the value of the current doc - docConsumer.accept(doc); - collectBucket(sub, doc, preBucketOrdinal); - } - - private void reset(BytesRef tsid, long bucket) { - timeBucketMetrics = new TreeMap<>(); - preTsid = BytesRef.deepCopyOf(tsid); - preRounding = -1; - - BytesRef bucketValue = needAggregator ? packKey(preTsid) : preTsid; - long bucketOrdinal = bucketOrds.add(bucket, bucketValue); - if (bucketOrdinal < 0) { // already seen - bucketOrdinal = -1 - bucketOrdinal; - grow(bucketOrdinal + 1); - } - preBucketOrdinal = bucketOrdinal; - } - } - public class DeferringCollector extends LeafBucketCollector { final SortedNumericDoubleValues values; final AggregationExecutionContext aggCtx; @@ -474,56 +413,52 @@ protected LeafBucketCollector getLeafCollectorInternal( LeafBucketCollector sub, AggregationExecutionContext aggContext ) throws IOException { - if (deferring) { - SortedDocValues tsids = DocValues.getSorted(context.reader(), TimeSeriesIdFieldMapper.NAME); - final AtomicInteger tsidOrd = new AtomicInteger(-1); - final AtomicLong currentBucketOrdinal = new AtomicLong(); - finishLeaf(); - return new LeafBucketCollectorBase(sub, null) { - int lastDoc = 0; - - @Override - public void collect(int doc, long bucket) throws IOException { - if (tsids.advanceExact(doc)) { - int newTsidOrd = tsids.ordValue(); - if (tsidOrd.get() < 0) { - reset(newTsidOrd, bucket); - } else if (tsidOrd.get() != newTsidOrd) { - reset(newTsidOrd, bucket); - } - } - - if (aggCtx == null) { - aggCtx = aggContext; - docDeltasBuilder = PackedLongValues.packedBuilder(PackedInts.DEFAULT); - bucketsBuilder = PackedLongValues.packedBuilder(PackedInts.DEFAULT); + SortedDocValues tsids = DocValues.getSorted(context.reader(), TimeSeriesIdFieldMapper.NAME); + final AtomicInteger tsidOrd = new AtomicInteger(-1); + final AtomicLong currentBucketOrdinal = new AtomicLong(); + finishLeaf(); + return new LeafBucketCollectorBase(sub, null) { + int lastDoc = 0; + + @Override + public void collect(int doc, long bucket) throws IOException { + if (tsids.advanceExact(doc)) { + int newTsidOrd = tsids.ordValue(); + if (tsidOrd.get() < 0) { + reset(newTsidOrd, bucket); + } else if (tsidOrd.get() != newTsidOrd) { + reset(newTsidOrd, bucket); } - docDeltasBuilder.add(doc - lastDoc); - bucketsBuilder.add(currentBucketOrdinal.get()); - lastDoc = doc; + } - collectBucket(sub, doc, currentBucketOrdinal.get()); + if (aggCtx == null) { + aggCtx = aggContext; + docDeltasBuilder = PackedLongValues.packedBuilder(PackedInts.DEFAULT); + bucketsBuilder = PackedLongValues.packedBuilder(PackedInts.DEFAULT); } + docDeltasBuilder.add(doc - lastDoc); + bucketsBuilder.add(currentBucketOrdinal.get()); + lastDoc = doc; + + collectBucket(sub, doc, currentBucketOrdinal.get()); + } - private void reset(int newTsidOrd, long bucket) throws IOException { - tsidOrd.set(newTsidOrd); - BytesRef tsid = tsids.lookupOrd(newTsidOrd); + private void reset(int newTsidOrd, long bucket) throws IOException { + tsidOrd.set(newTsidOrd); + BytesRef tsid = tsids.lookupOrd(newTsidOrd); - BytesRef bucketValue = needAggregator ? packKey(tsid) : tsid; - long bucketOrdinal = bucketOrds.add(bucket, bucketValue); - if (bucketOrdinal < 0) { // already seen - bucketOrdinal = -1 - bucketOrdinal; - grow(bucketOrdinal + 1); - } - currentBucketOrdinal.set(bucketOrdinal); + BytesRef bucketValue = needAggregator ? packKey(tsid) : tsid; + long bucketOrdinal = bucketOrds.add(bucket, bucketValue); + if (bucketOrdinal < 0) { // already seen + bucketOrdinal = -1 - bucketOrdinal; + grow(bucketOrdinal + 1); } - }; - } else { - return getCollector(sub, aggContext); - } + currentBucketOrdinal.set(bucketOrdinal); + } + }; } - protected LeafBucketCollector getCollector(LeafBucketCollector sub, AggregationExecutionContext aggCtx) throws IOException { + protected LeafBucketCollector getCollector(AggregationExecutionContext aggCtx) throws IOException { final SortedNumericDoubleValues values = valuesSource.doubleValues(aggCtx.getLeafReaderContext()); CheckedConsumer docConsumer = (doc) -> { if (aggCtx.getTimestamp() + downsampleRange < preRounding) { @@ -550,26 +485,12 @@ protected LeafBucketCollector getCollector(LeafBucketCollector sub, AggregationE } } }; - if (deferring) { - return new DeferringCollector(values, aggCtx, docConsumer); - } else { - return new Collector(sub, values, aggCtx, docConsumer); - } + return new DeferringCollector(values, aggCtx, docConsumer); } @Override protected void doPostCollection() throws IOException { - if (deferring) { - finishLeaf(); - } else { - /** - * collect the last tsid - */ - if (timeBucketMetrics != null && timeBucketMetrics.size() > 0) { - collectTimeSeriesValues(preBucketOrdinal); - } - } - + finishLeaf(); } /** @@ -758,7 +679,7 @@ class LeafWalker { this.docDeltaIterator = docDeltaIterator; this.buckets = buckets; AggregationExecutionContext aggContext = new AggregationExecutionContext(context, scratch::get, () -> timestamp); - this.collector = getCollector(null, aggContext); + this.collector = getCollector(aggContext); tsids = DocValues.getSorted(context.reader(), TimeSeriesIdFieldMapper.NAME); timestamps = DocValues.getSortedNumeric(context.reader(), DataStream.TimestampField.FIXED_TIMESTAMP_FIELD); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java index 9b1c4812f2c13..fb1e28cc9a38f 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java @@ -38,7 +38,6 @@ Aggregator build( BucketOrder order, long startTime, long endTime, - boolean deferring, ValuesSourceConfig valuesSourceConfig, AggregationContext context, Aggregator parent, diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java index b1834644aa90a..897e959f4582d 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java @@ -53,7 +53,6 @@ public AggregateMetricTimeSeriesAggregationAggregator( BucketOrder order, long startTime, long endTime, - boolean deferring, ValuesSourceConfig valuesSourceConfig, AggregationContext context, Aggregator parent, @@ -75,7 +74,6 @@ public AggregateMetricTimeSeriesAggregationAggregator( order, startTime, endTime, - deferring, null, context, parent, @@ -117,7 +115,6 @@ public boolean isNoop() { @Override protected LeafBucketCollector getCollector( - LeafBucketCollector sub, AggregationExecutionContext aggCtx ) throws IOException { Metric metricType = getAggregateMetric(); @@ -148,11 +145,7 @@ protected LeafBucketCollector getCollector( } } }; - if (deferring) { - return new DeferringCollector(values, aggCtx, docConsumer); - } else { - return new Collector(sub, values, aggCtx, docConsumer); - } + return new DeferringCollector(values, aggCtx, docConsumer); } else { final SortedNumericDoubleValues aggregateSums = valuesSource.getAggregateMetricValues(aggCtx.getLeafReaderContext(), Metric.sum); final SortedNumericDoubleValues aggregateValueCounts = valuesSource.getAggregateMetricValues(aggCtx.getLeafReaderContext(), Metric.value_count); @@ -194,11 +187,7 @@ protected LeafBucketCollector getCollector( } }; - if (deferring) { - return new DeferringCollector(aggregateSums, aggCtx, docConsumer); - } else { - return new Collector(sub, aggregateSums, aggCtx, docConsumer); - } + return new DeferringCollector(aggregateSums, aggCtx, docConsumer); } } From 6898e84e771ba602986fdcabbf6b44cc85b55702 Mon Sep 17 00:00:00 2001 From: weizijun Date: Fri, 19 Aug 2022 17:28:03 +0800 Subject: [PATCH 53/53] fix rounding --- .../aggregation/TimeSeriesAggregationAggregator.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java index c4df98831801a..706b83dc09d4b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -157,7 +157,7 @@ public TimeSeriesAggregationAggregator( ? downsample.getFunction() : (downsampleRange > 0 ? Function.origin_value : Function.last); if (this.downsampleRange <= 0) { - this.downsampleRange = this.interval; + this.downsampleRange = this.interval - 1; } this.downsampleParams = downsample != null && downsample.getParameters() != null ? new HashMap<>(downsample.getParameters()) @@ -368,7 +368,7 @@ public void collect(int doc, long bucket) throws IOException { reset(newTsid, bucket); } - if (preRounding < 0 || aggCtx.getTimestamp() < preRounding - interval) { + if (preRounding < 0 || (aggCtx.getTimestamp() <= preRounding - interval) && preRounding > startTime) { preRounding = rounding.nextRoundingValue(aggCtx.getTimestamp()); }