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..1e6ee4b3565a7 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,10 @@ 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.core.Tuple; import org.elasticsearch.index.mapper.DateFieldMapper; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; @@ -25,10 +28,14 @@ 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.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; @@ -40,6 +47,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; @@ -50,6 +58,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; @@ -521,4 +530,212 @@ 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(((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)); + }); + } + } + + 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") + .setSize(0) + .addAggregation( + timeSeriesAggregation("by_ts").field("metric_0") + .interval(fixedInterval) + .downsample(fixedInterval, Function.sum_over_time, null) + .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(((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)); + }); + } + } + + 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(((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)); + }); + } + } + + 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) + .downsample(fixedInterval, Function.max_over_time, null) + .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); + } + } + }); + + for (InternalTimeSeriesAggregation.InternalBucket bucket : timeSeries.getBuckets()) { + Map key = bucket.getKey(); + Map dataValues = aggResults.get(key); + dataValues.forEach((timestamp, metric) -> { + 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 caa44ced40a63..c9dd0168d47ab 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchModule.java +++ b/server/src/main/java/org/elasticsearch/search/SearchModule.java @@ -215,6 +215,16 @@ 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.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; import org.elasticsearch.search.fetch.FetchSubPhase; import org.elasticsearch.search.fetch.subphase.ExplainPhase; @@ -666,6 +676,24 @@ private ValuesSourceRegistry registerAggregations(List plugins) { ).addResultReader(InternalTimeSeries::new), builder ); + + registerAggregation( + new AggregationSpec( + TimeSeriesAggregationAggregationBuilder.NAME, + TimeSeriesAggregationAggregationBuilder::new, + TimeSeriesAggregationAggregationBuilder.PARSER + ).addResultReader(InternalTimeSeriesAggregation::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) + .addResultReader(TimeSeriesCountValues.NAME, TimeSeriesCountValues::new) + .addResultReader(TimeSeriesOriginValues.NAME, TimeSeriesOriginValues::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/timeseries/aggregation/Aggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java new file mode 100644 index 0000000000000..cd5b93859a413 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Aggregator.java @@ -0,0 +1,175 @@ +/* + * 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.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; +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.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; + +import java.util.Map; + +public enum Aggregator { + count { + @Override + public AggregatorFunction getAggregatorFunction(Map aggregatorParams) { + return new ValueCountFunction(); + } + + @Override + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, Map aggregatorParams) { + return new ValueCountBucketFunction(bigArrays); + } + }, + sum { + @Override + public AggregatorFunction getAggregatorFunction(Map aggregatorParams) { + return new SumFunction(); + } + + @Override + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, Map aggregatorParams) { + return new SumBucketFunction(bigArrays); + } + }, + min { + @Override + public AggregatorFunction getAggregatorFunction(Map aggregatorParams) { + return new MinFunction(); + } + + @Override + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, Map aggregatorParams) { + return new MinBucketFunction(bigArrays); + } + }, + max { + @Override + public AggregatorFunction getAggregatorFunction(Map aggregatorParams) { + return new MaxFunction(); + } + + @Override + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, Map aggregatorParams) { + return new MaxBucketFunction(bigArrays); + } + }, + avg { + @Override + public AggregatorFunction getAggregatorFunction(Map aggregatorParams) { + return new AvgFunction(); + } + + @Override + 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 = ((Double) aggregatorParams.get("size")).intValue(); + } + return new TopkFunction(size, true); + } + + @Override + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, Map aggregatorParams) { + int size = 0; + if (aggregatorParams != null && aggregatorParams.containsKey("size")) { + size = ((Double) aggregatorParams.get("size")).intValue(); + } + return new TopkBucketFunction(bigArrays, size, true); + } + }, + bottomk { + @Override + public AggregatorFunction getAggregatorFunction(Map aggregatorParams) { + int size = 0; + if (aggregatorParams != null && aggregatorParams.containsKey("size")) { + size = ((Double) aggregatorParams.get("size")).intValue(); + } + return new TopkFunction(size, false); + } + + @Override + public AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, Map aggregatorParams) { + int size = 0; + if (aggregatorParams != null && aggregatorParams.containsKey("size")) { + size = ((Double) aggregatorParams.get("size")).intValue(); + } + return new TopkBucketFunction(bigArrays, 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); + } + }, + 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) { + try { + return Aggregator.valueOf(name); + } catch (Exception e) { + throw new IllegalArgumentException("aggregator [" + name + "] not support"); + } + } + + /** + * get the aggregator function + */ + public abstract AggregatorFunction getAggregatorFunction(Map aggregatorParams); + + /** + * get the aggregator bucket function + */ + public abstract AggregatorBucketFunction getAggregatorBucketFunction(BigArrays bigArrays, Map aggregatorParams); +} 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..0dfc367aeaf25 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Downsample.java @@ -0,0 +1,109 @@ +/* + * 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; +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"); + + @SuppressWarnings("unchecked") + public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + NAME, + false, + (args, name) -> new Downsample((DateHistogramInterval) args[0], (Function) args[1], (Map) args[2]) + ); + + static { + PARSER.declareField( + ConstructingObjectParser.optionalConstructorArg(), + p -> new DateHistogramInterval(p.text()), + RANGE_FIELD, + ObjectParser.ValueType.STRING + ); + PARSER.declareField( + ConstructingObjectParser.optionalConstructorArg(), + p -> Function.resolve(p.text()), + 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 parameters; + + public Downsample(DateHistogramInterval range, Function function, Map parameters) { + this.range = range; + this.function = function; + this.parameters = parameters; + } + + public Downsample(StreamInput in) throws IOException { + this.range = in.readOptionalWriteable(DateHistogramInterval::new); + this.function = in.readOptionalEnum(Function.class); + this.parameters = in.readMap(); + } + + public DateHistogramInterval getRange() { + return range; + } + + public Function getFunction() { + return function; + } + + public Map getParameters() { + return parameters; + } + + public static Downsample fromXContent(XContentParser parser) { + return PARSER.apply(parser, null); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + if (function != null) { + builder.field(FUNCTION_FIELD.getPreferredName(), function); + } + if (range != null) { + builder.field(RANGE_FIELD.getPreferredName(), range); + } + if (parameters != null) { + builder.field(PARAMS_FIELD.getPreferredName(), parameters); + } + builder.endObject(); + return builder; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeOptionalWriteable(range); + out.writeOptionalEnum(function); + out.writeGenericMap(parameters); + } +} 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..309fa93ef9ef9 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/Function.java @@ -0,0 +1,375 @@ +/* + * 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.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; +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.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; +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; +import java.time.ZonedDateTime; +import java.util.Map; + +public enum Function { + count_over_time(ValueType.matrix) { + @Override + public AggregatorFunction getFunction(Map params) { + return new ValueCountFunction(); + } + }, + count_exact_over_time(ValueType.matrix) { + @Override + public AggregatorFunction getFunction(Map params) { + return new ValueCountExactFunction(); + } + }, + sum_over_time(ValueType.matrix) { + @Override + public AggregatorFunction getFunction(Map params) { + return new SumFunction(); + } + }, + min_over_time(ValueType.matrix) { + @Override + public AggregatorFunction getFunction(Map params) { + return new MinFunction(); + } + }, + max_over_time(ValueType.matrix) { + @Override + public AggregatorFunction getFunction(Map params) { + return new MaxFunction(); + } + }, + avg_over_time(ValueType.matrix) { + @Override + public AggregatorFunction getFunction(Map params) { + return new AvgFunction(); + } + }, + avg_exact_over_time(ValueType.matrix) { + @Override + public AggregatorFunction getFunction(Map params) { + return new AvgExactFunction(); + } + }, + last(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new LastFunction(value -> value); + } + }, + origin_value(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new OriginValuesFunction(); + } + }, + 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(ValueType.matrix) { + @Override + public AggregatorFunction getFunction(Map params) { + return new RateFunction((long) params.get(RANGE_FIELD), (long) params.get(ROUNDING_FIELD), false, false); + } + }, + 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(ValueType.matrix) { + @Override + public AggregatorFunction getFunction(Map params) { + return new IRateFunction(true); + } + }, + idelta(ValueType.matrix) { + @Override + public AggregatorFunction getFunction(Map params) { + return new IRateFunction(true); + } + }, + abs(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new LastFunction(Math::abs); + } + }, + ceil(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new LastFunction(Math::ceil); + } + }, + floor(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new LastFunction(Math::floor); + } + }, + exp(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new LastFunction(Math::exp); + } + }, + sqrt(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new LastFunction(Math::sqrt); + } + }, + ln(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new LastFunction(Math::log); + } + }, + log10(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new LastFunction(Math::log10); + } + }, + sin(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new LastFunction(Math::sin); + } + }, + cos(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new LastFunction(Math::cos); + } + }, + tan(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new LastFunction(Math::tan); + } + }, + asin(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new LastFunction(Math::asin); + } + }, + acos(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new LastFunction(Math::acos); + } + }, + atan(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new LastFunction(Math::atan); + } + }, + sinh(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new LastFunction(Math::sinh); + } + }, + cosh(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new LastFunction(Math::cosh); + } + }, + tanh(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new LastFunction(Math::tanh); + } + }, + rad(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new LastFunction(value -> value * Math.PI / 180); + } + }, + deg(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new LastFunction(value -> value * 180 / Math.PI); + } + }, + pi(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new LastFunction(value -> Math.PI); + } + }, + sgn(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new LastFunction(v -> { + if (v < 0) { + return -1d; + } else if (v > 0) { + return 1d; + } + return v; + }); + } + }, + timestamp(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new TimestampFunction(); + } + }, + day_of_month(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new DateFunction(ZonedDateTime::getDayOfMonth); + } + }, + day_of_week(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new DateFunction(zonedDateTime -> { + DayOfWeek dayOfWeek = zonedDateTime.getDayOfWeek(); + return dayOfWeek.getValue() % 7; + }); + } + }, + hour(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new DateFunction(ZonedDateTime::getHour); + } + }, + minute(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new DateFunction(ZonedDateTime::getMinute); + } + }, + month(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new DateFunction(ZonedDateTime::getMonthValue); + } + }, + year(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + return new DateFunction(ZonedDateTime::getMonthValue); + } + }, + clamp(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + double max = Double.MAX_VALUE; + if (params != null && params.containsKey(MAX_FIELD)) { + max = Double.valueOf(String.valueOf(params.get(MAX_FIELD))); + } + double min = Double.MIN_VALUE; + if (params != null && params.containsKey(MIN_FIELD)) { + min = Double.valueOf(String.valueOf(params.get(MIN_FIELD))); + } + return new ClampFunction(max, min); + } + }, + clamp_max(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + double max = Double.MAX_VALUE; + if (params != null && params.containsKey(MAX_FIELD)) { + max = Double.valueOf(String.valueOf(params.get(MAX_FIELD))); + } + return new ClampMaxFunction(max); + } + }, + clamp_min(ValueType.vector) { + @Override + public AggregatorFunction getFunction(Map params) { + double min = Double.MIN_VALUE; + if (params != null && params.containsKey(MIN_FIELD)) { + min = Double.valueOf(String.valueOf(params.get(MIN_FIELD))); + } + return new ClampMinFunction(min); + } + }, + quantile_over_time(ValueType.matrix) { + @Override + public AggregatorFunction getFunction(Map params) { + double quantile = 1f; + if (params != null && params.containsKey(QUANTILE_FIELD)) { + quantile = (double) params.get(QUANTILE_FIELD); + } + return new QuantileFunction(quantile); + } + }, + last_over_time(ValueType.matrix) { + @Override + 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"; + + 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); + } catch (Exception e) { + throw new IllegalArgumentException("function [" + name + "] not support"); + } + } + + /** + * get the function + */ + public abstract AggregatorFunction getFunction(Map params); + + public enum ValueType { + vector, + matrix + } +} 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..3e84007a581b7 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregation.java @@ -0,0 +1,491 @@ +/* + * 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.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.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.Objects; + +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 Logger logger = LogManager.getLogger(InternalTimeSeriesAggregation.class); + + 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 InternalAggregation metricAggregation; + protected boolean showDocCountError; + protected long docCountError; + protected InternalAggregations aggregations; + + public InternalBucket( + Map key, + long docCount, + InternalAggregation metricAggregation, + InternalAggregations aggregations, + boolean keyed, + boolean showDocCountError, + long docCountError + ) { + this.key = key; + this.docCount = docCount; + this.aggregations = aggregations; + this.keyed = keyed; + this.metricAggregation = metricAggregation; + 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(); + metricAggregation = in.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.writeNamedWriteable(metricAggregation); + if (showDocCountError) { + // TODO recover -Dtests.seed=142C4BE4C242FF8B + // 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; + } + + public InternalAggregation getMetricAggregation() { + return metricAggregation; + } + + @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()); + metricAggregation.doXContentBody(builder, params); + 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(metricAggregation, that.metricAggregation); + } + + @Override + public int hashCode() { + return Objects.hash(getClass(), key, keyed, docCount, metricAggregation); + } + + @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.metricAggregation, + 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.metricAggregation, + 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 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; + List aggregationsList = new ArrayList<>(buckets.size()); + List metricAggregationsList = new ArrayList<>(buckets.size()); + 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.metricAggregation, + bucket.aggregations, + bucket.keyed, + bucket.showDocCountError, + docCountError + ); + } else { + reduced.docCount += bucket.docCount; + } + + metricAggregationsList.add(bucket.metricAggregation); + aggregationsList.add(bucket.aggregations); + } + + reduced.metricAggregation = reduced.metricAggregation.reduce(metricAggregationsList, context); + 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..b1bfa6235fbe6 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/ParsedTimeSeriesAggregation.java @@ -0,0 +1,202 @@ +/* + * 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/RoundingInterval.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingInterval.java new file mode 100644 index 0000000000000..e2594dd8b8dba --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/RoundingInterval.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; + +import org.elasticsearch.common.Rounding; +import org.elasticsearch.core.TimeValue; + +public class RoundingInterval { + private final long startTime; + private final long interval; + private final Rounding.Prepared rounding; + + public RoundingInterval(long startTime, long interval) { + this.startTime = startTime; + this.interval = interval; + if (this.startTime <= 0) { + rounding = Rounding.builder(new TimeValue(this.interval)).build().prepareForUnknown(); + } else { + rounding = null; + } + } + + public long nextRoundingValue(long utcMillis) { + if (rounding != null) { + return rounding.nextRoundingValue(utcMillis - 1); + } else { + long step = (utcMillis - 1 - startTime) / interval + 1; + return startTime + interval * step; + } + } +} 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..ab79f1260c632 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TSIDValue.java @@ -0,0 +1,21 @@ +/* + * 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) { + this.tsid = tsid; + this.value = value; + } + + public BytesRef tsid; + public Value value; +} 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/TimeSeriesAggregation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregation.java new file mode 100644 index 0000000000000..9ecd4c0364a22 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregation.java @@ -0,0 +1,29 @@ +/* + * 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); +} 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..d82aef6e04303 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationBuilder.java @@ -0,0 +1,591 @@ +/* + * 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 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"); + 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"); + public static final ParseField START_TIME_FIELD = new ParseField("start_time"); + public static final ParseField END_TIME_FIELD = new ParseField("end_time"); + + 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 Aggregator aggregator; + private Map aggregatorParams; + private Downsample downsample; + private TermsAggregator.BucketCountThresholds bucketCountThresholds = new TermsAggregator.BucketCountThresholds( + DEFAULT_BUCKET_COUNT_THRESHOLDS + ); + private BucketOrder order = BucketOrder.key(true); + private Long startTime; + private Long endTime; + + 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.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, + (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); + PARSER.declareLong(TimeSeriesAggregationAggregationBuilder::startTime, START_TIME_FIELD); + PARSER.declareLong(TimeSeriesAggregationAggregationBuilder::endTime, END_TIME_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.aggregatorParams = clone.aggregatorParams; + 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 { + super(in); + keyed = in.readBoolean(); + group = in.readOptionalStringList(); + without = in.readOptionalStringList(); + 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); + startTime = in.readOptionalLong(); + endTime = in.readOptionalLong(); + } + + @Override + protected void innerWriteTo(StreamOutput out) throws IOException { + out.writeBoolean(keyed); + out.writeOptionalStringCollection(group); + out.writeOptionalStringCollection(without); + out.writeOptionalWriteable(interval); + out.writeOptionalWriteable(offset); + out.writeOptionalEnum(aggregator); + out.writeGenericMap(aggregatorParams); + out.writeOptionalWriteable(downsample); + order.writeTo(out); + bucketCountThresholds.writeTo(out); + out.writeOptionalLong(startTime); + out.writeOptionalLong(endTime); + } + + @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, + aggregatorParams, + downsample, + bucketCountThresholds, + order, + startTime != null ? startTime : -1, + endTime != null ? endTime : -1, + 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 (aggregatorParams != null) { + builder.field(AGGREGATOR_PARAMS_FIELD.getPreferredName(), aggregatorParams); + } + if (downsample != null) { + builder.field(DOWNSAMPLE_FIELD.getPreferredName(), downsample); + } + 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; + } + + @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 false; + } + + /** + * 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 Aggregator 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.resolve(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) + */ + 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; + } + + /** + * 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, Map params) { + this.downsample = new Downsample(range, function, params); + 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) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (false == 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) + && aggregator == that.aggregator + && Objects.equals(aggregatorParams, that.aggregatorParams) + && Objects.equals(downsample, that.downsample) + && Objects.equals(bucketCountThresholds, that.bucketCountThresholds) + && Objects.equals(order, that.order) + && Objects.equals(startTime, that.startTime) + && Objects.equals(endTime, that.endTime); + } + + @Override + public int hashCode() { + return Objects.hash( + super.hashCode(), + keyed, + group, + without, + interval, + offset, + aggregator, + aggregatorParams, + downsample, + bucketCountThresholds, + order, + startTime, + endTime + ); + } + + @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..25bef1563e214 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregationFactory.java @@ -0,0 +1,162 @@ +/* + * 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 org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator aggregator; + 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; + + public TimeSeriesAggregationAggregationFactory( + String name, + boolean keyed, + List group, + List without, + DateHistogramInterval interval, + DateHistogramInterval offset, + org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator aggregator, + Map aggregatorParams, + Downsample downsample, + TermsAggregator.BucketCountThresholds bucketCountThresholds, + BucketOrder order, + long startTime, + long endTime, + 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.aggregatorParams = aggregatorParams; + this.downsample = downsample; + this.bucketCountThresholds = bucketCountThresholds; + this.startTime = startTime; + this.endTime = endTime; + 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, + aggregatorParams, + downsample, + thresholds, + order, + startTime, + endTime, + 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, + aggregatorParams, + downsample, + thresholds, + order, + startTime, + endTime, + 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..706b83dc09d4b --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregator.java @@ -0,0 +1,738 @@ +/* + * 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 TimeSeriesAggregationAggregator extends BucketsAggregator { + private static final Logger logger = LogManager.getLogger(TimeSeriesAggregationAggregator.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; + + 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 TimeSeriesAggregationAggregator( + 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, + 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.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 - 1; + } + 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); + } + + 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; + 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); + 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 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 > startTime) { + 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 { + 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); + } + }; + } + + protected LeafBucketCollector getCollector(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; + } + } + } + } + }; + return new DeferringCollector(values, aggCtx, docConsumer); + } + + @Override + protected void doPostCollection() throws IOException { + finishLeaf(); + } + + /** + * 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(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 new file mode 100644 index 0000000000000..fb1e28cc9a38f --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/TimeSeriesAggregationAggregatorSupplier.java @@ -0,0 +1,47 @@ +/* + * 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, + org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator aggregator, + Map aggregatorParams, + Downsample downsample, + TermsAggregator.BucketCountThresholds bucketCountThresholds, + BucketOrder order, + long startTime, + long endTime, + 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/bucketfunction/AggregatorBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AggregatorBucketFunction.java new file mode 100644 index 0000000000000..a807029857334 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AggregatorBucketFunction.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.bucketfunction; + +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, Map aggregatorParams, 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/AvgBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/AvgBucketFunction.java new file mode 100644 index 0000000000000..0d5caeb82e1a5 --- /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.timeseries.aggregation.Aggregator; + +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 Aggregator.avg.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, Map aggregatorParams, 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/CountValuesBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/CountValuesBucketFunction.java new file mode 100644 index 0000000000000..fc127de9ed62f --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/CountValuesBucketFunction.java @@ -0,0 +1,70 @@ +/* + * 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.timeseries.aggregation.Aggregator; +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 ObjectArray> values; + private BigArrays bigArrays; + + public CountValuesBucketFunction(BigArrays bigArrays) { + this.bigArrays = bigArrays; + this.values = bigArrays.newObjectArray(1); + } + + @Override + public String name() { + return Aggregator.count_values.name(); + } + + @Override + public void collect(Double number, long bucket) { + 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 + public InternalAggregation getAggregation( + long bucket, + Map aggregatorParams, + DocValueFormat formatter, + Map metadata + ) { + return new TimeSeriesCountValues(name(), values.get(bucket), formatter, metadata); + } + + @Override + public void close() { + Releasables.close(values); + } +} 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..e3e34b564ad70 --- /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.timeseries.aggregation.Aggregator; + +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 Aggregator.max.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, Map aggregatorParams, 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..ef3a7e00ad0e5 --- /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.timeseries.aggregation.Aggregator; + +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 Aggregator.min.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, Map aggregatorParams, 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..58b6152599055 --- /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.TimeSeriesLast; + +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, Map aggregatorParams, DocValueFormat formatter, Map metadata) { + return new TimeSeriesLast(name(), value, 0, formatter, metadata); + } + + @Override + public void close() { + + } +} 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..0c7c873da773d --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/QuantileBucketFunction.java @@ -0,0 +1,65 @@ +/* + * 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 org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator; + +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 Aggregator.quantile.name(); + } + + @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/bucketfunction/SumBucketFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/SumBucketFunction.java new file mode 100644 index 0000000000000..ba95bc93cf7c1 --- /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.timeseries.aggregation.Aggregator; + +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 Aggregator.sum.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, Map aggregatorParams, 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..ab14a71b7f075 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TSIDBucketFunction.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.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; +import org.elasticsearch.search.aggregations.timeseries.aggregation.internal.TSIDInternalAggregation; + +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 overflow 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. + */ +@SuppressWarnings({ "unchecked", "rawtypes" }) +public class TSIDBucketFunction implements AggregatorBucketFunction { + private final BigArrays bigArrays; + private ObjectArray> values; + private final AggregatorBucketFunction aggregatorBucketFunction; + + public TSIDBucketFunction(BigArrays bigArrays, AggregatorBucketFunction aggregatorBucketFunction) { + this.aggregatorBucketFunction = aggregatorBucketFunction; + this.bigArrays = bigArrays; + values = bigArrays.newObjectArray(1); + } + + @Override + public String name() { + return TSIDInternalAggregation.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.set(bucket, tsidValues); + } + tsidValues.put(tsidValue.tsid, (InternalAggregation) tsidValue.value); + } else if (aggregatorBucketFunction instanceof TopkBucketFunction) { + aggregatorBucketFunction.collect(tsidValue, bucket); + } 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 + ) { + 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 new file mode 100644 index 0000000000000..518021758e32a --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/bucketfunction/TopkBucketFunction.java @@ -0,0 +1,76 @@ +/* + * 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.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.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; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class TopkBucketFunction implements AggregatorBucketFunction> { + + private final BigArrays bigArrays; + private ObjectArray>> values; + private final int topkSize; + private final boolean isTop; + + public TopkBucketFunction(BigArrays bigArrays, int size, boolean isTop) { + this.bigArrays = bigArrays; + values = bigArrays.newObjectArray(1); + this.topkSize = size; + this.isTop = isTop; + } + + @Override + public String name() { + return Aggregator.topk.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.set(bucket, queue); + } + + queue.insertWithOverflow(number); + } + + @Override + 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 + public void close() { + Releasables.close(values); + } +} 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..cfdc68ca90d29 --- /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.timeseries.aggregation.Aggregator; + +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 Aggregator.count.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, Map aggregatorParams, 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/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/AggregatorFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AggregatorFunction.java new file mode 100644 index 0000000000000..551c7fd777d12 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AggregatorFunction.java @@ -0,0 +1,40 @@ +/* + * 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; + +/** + * 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); +} 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 new file mode 100644 index 0000000000000..eea43c6645294 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/AvgFunction.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.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 { + private double sum = 0; + private long count = 0; + + @Override + public void collect(TimePoint value) { + sum += value.getValue(); + count += 1; + } + + @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/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/CountValuesFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/CountValuesFunction.java new file mode 100644 index 0000000000000..bd2a806f4309b --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/CountValuesFunction.java @@ -0,0 +1,48 @@ +/* + * 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.TimeSeriesCountValues; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +public class CountValuesFunction implements AggregatorFunction> { + + private Map valueCount; + + public CountValuesFunction() { + valueCount = new HashMap<>(); + } + + @Override + public void collect(TimePoint value) { + long val = java.lang.Double.doubleToLongBits(value.getValue()); + AtomicInteger count = valueCount.get(val); + if (count == null) { + count = new AtomicInteger(0); + valueCount.put(val, count); + } + count.incrementAndGet(); + } + + @Override + public Map get() { + return valueCount; + } + + @Override + 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/function/DateFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/DateFunction.java new file mode 100644 index 0000000000000..95c74844391ce --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/DateFunction.java @@ -0,0 +1,33 @@ +/* + * 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; + +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/IRateFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/IRateFunction.java new file mode 100644 index 0000000000000..8c35786392b10 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/IRateFunction.java @@ -0,0 +1,76 @@ +/* + * 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; + +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 Double.NaN; + } + + 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 Double.NaN; + } + + 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 new file mode 100644 index 0000000000000..0368c6f7022a4 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/LastFunction.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.function; + +import java.util.function.Function; + +public class LastFunction extends AbstractLastFunction { + private Function function; + + public LastFunction(Function function) { + this.function = function; + } + + @Override + public Double interGet() { + return function.apply(point.getValue()); + } +} 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..261e01a20ad06 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/MaxFunction.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.MaxAggregationBuilder; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; + +import java.util.Map; + +public class MaxFunction implements AggregatorFunction { + private Double max = Double.NEGATIVE_INFINITY; + + @Override + public void collect(TimePoint value) { + this.max = Math.max(value.getValue(), 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..2153ce9310053 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/MinFunction.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.MinAggregationBuilder; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; + +import java.util.Map; + +public class MinFunction implements AggregatorFunction { + private Double min = Double.POSITIVE_INFINITY; + + @Override + public void collect(TimePoint value) { + this.min = Math.min(value.getValue(), 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/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/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); + } +} 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..f4ddd64bae7d0 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/RateFunction.java @@ -0,0 +1,138 @@ +/* + * 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) { + 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; + } + + if (currentSample.compareTo(value) > 0 && currentSample.getValue() < value.getValue()) { + totalRevertValue += value.getValue(); + } + + currentSample = value; + } + + @Override + 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 Double.NaN; + } + + double resultValue = lastSample.getValue() - firstSample.getValue(); + if (isCounter) { + resultValue += totalRevertValue; + } + + double durationToStart = (double) (firstSample.getTimestamp() - rangeStart) / 1000; + double durationToEnd = (double) (rangeEnd - lastSample.getTimestamp()) / 1000; + + double sampledInterval = (double) (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 / 1000); + } + + return resultValue; + } +} 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..f9984d415a9c8 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/SumFunction.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.SumAggregationBuilder; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; + +import java.util.Map; + +public class SumFunction implements AggregatorFunction { + private double sum = 0; + + @Override + public void collect(TimePoint value) { + this.sum += value.getValue(); + } + + @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/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; + } +} 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..1efec423df671 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/TopkFunction.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.function; + +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, boolean isTop) { + queue = getTopkQueue(size, isTop); + this.isTop = isTop; + this.topkSize = size; + } + + @Override + public void collect(TSIDValue value) { + queue.insertWithOverflow(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 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/function/ValueCountExactFunction.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountExactFunction.java new file mode 100644 index 0000000000000..31b1f900a4549 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountExactFunction.java @@ -0,0 +1,36 @@ +/* + * 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.Aggregator; +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(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 new file mode 100644 index 0000000000000..66048a4fcee5d --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/function/ValueCountFunction.java @@ -0,0 +1,36 @@ +/* + * 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.Aggregator; +import org.elasticsearch.search.aggregations.timeseries.aggregation.TimePoint; + +import java.util.Map; + +public class ValueCountFunction implements AggregatorFunction { + private long count = 0; + + @Override + public void collect(TimePoint value) { + count += 1; + } + + @Override + public Long get() { + return count; + } + + @Override + public InternalAggregation getAggregation(DocValueFormat formatter, Map 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 new file mode 100644 index 0000000000000..b1d05e3b2555b --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TSIDInternalAggregation.java @@ -0,0 +1,160 @@ +/* + * 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.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; +import java.util.ArrayList; +import java.util.List; +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 = "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 + ) { + super(name, metadata); + 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)); + } + + @Override + public String getWriteableName() { + return NAME; + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + out.writeNamedWriteable(formatter); + out.writeString(aggregator); + out.writeGenericMap(aggregatorParams); + 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()) { + Aggregator function = Aggregator.resolve(aggregator); + final AggregatorFunction aggregatorFunction = function.getAggregatorFunction(aggregatorParams); + tsidAgg.values.forEach( + (tsid, agg) -> { + 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()); + } else { + return tsidAgg; + } + } + + Map> reduced = new TreeMap<>(); + for (InternalAggregation aggregation : aggregations) { + TSIDInternalAggregation tsidAgg = (TSIDInternalAggregation) aggregation; + tsidAgg.values.forEach((tsid, value) -> { + List values = reduced.get(tsid); + if (values == null) { + values = new ArrayList<>(); + reduced.put(tsid, values); + } + values.add(value); + }); + } + + if (reduceContext.isFinalReduce()) { + Aggregator function = Aggregator.valueOf(aggregator); + 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); + if (aggregatorFunction instanceof TopkFunction) { + aggregatorFunction.collect(new TSIDValue<>(tsid, internalAggregation.value())); + } else { + aggregatorFunction.collect(new TimePoint(0, 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, aggregator, aggregatorParams, 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(), aggregator); + return builder; + } +} 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..bd9f4e112bfdf --- /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; + +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; + } + + public Map getValueCount() { + return valueCount; + } + + @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; + } +} 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..700fd396f9ede --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesIRate.java @@ -0,0 +1,153 @@ +/* + * 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; + +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/TimeSeriesLast.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesLast.java new file mode 100644 index 0000000000000..2872784be9d9a --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesLast.java @@ -0,0 +1,113 @@ +/* + * 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.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class TimeSeriesLast extends InternalNumericMetricsAggregation.SingleValue { + public static final String NAME = "time_series_last"; + + private final double last; + private final long timestamp; + + public TimeSeriesLast(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 TimeSeriesLast(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 NAME; + } + + public InternalAggregation finalizeSampling(SamplingContext samplingContext) { + return this; + } + + @Override + public double value() { + return last; + } + + public long getTimestamp() { + return timestamp; + } + + @Override + public TimeSeriesLast reduce(List aggregations, AggregationReduceContext reduceContext) { + double last = Double.NEGATIVE_INFINITY; + long timestamp = Long.MIN_VALUE; + for (InternalAggregation aggregation : aggregations) { + if (((TimeSeriesLast) aggregation).timestamp > timestamp) { + last = ((TimeSeriesLast) aggregation).last; + timestamp = ((TimeSeriesLast) aggregation).timestamp; + } + } + return new TimeSeriesLast(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 (false == super.equals(o)) { + return false; + } + TimeSeriesLast last1 = (TimeSeriesLast) 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/TimeSeriesLineAggreagation.java b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesLineAggreagation.java new file mode 100644 index 0000000000000..fddf8c811f70e --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesLineAggreagation.java @@ -0,0 +1,110 @@ +/* + * 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; + +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/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()); + } +} 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..5be1cf0bcbbd4 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesRate.java @@ -0,0 +1,191 @@ +/* + * 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.RateFunction; +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 = "time_series_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 = in.readOptionalWriteable(TimePoint::new); + firstSample = in.readOptionalWriteable(TimePoint::new); + 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); + out.writeOptionalWriteable(lastSample); + out.writeOptionalWriteable(firstSample); + 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 = RateFunction.extrapolatedRate( + range, + timestamp, + isCounter, + isRate, + lastSample, + firstSample, + count, + totalRevertValue + ); + } + 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); + if (timeSeriesRate.count == 0) { + continue; + } + 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) { + 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); + } + } +} 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..11bbe406b9456 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/timeseries/aggregation/internal/TimeSeriesTopk.java @@ -0,0 +1,117 @@ +/* + * 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.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.search.aggregations.timeseries.aggregation.function.TopkFunction; +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 = TopkFunction.getTopkQueue(topkSize, isTop); + + 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/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..d3d82c460844e --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/timeseries/aggregation/InternalTimeSeriesAggregationTests.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; + +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.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.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; + 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 TimeSeriesLineAggreagation( + TimeSeriesLineAggreagation.NAME, + Collections.emptyMap(), + DocValueFormat.RAW, + Collections.emptyMap() + ), + 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/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/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..7c9d26b075a9f --- /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)).field(randomAlphaOfLength(5)); + } + + @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..897e959f4582d --- /dev/null +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricTimeSeriesAggregationAggregator.java @@ -0,0 +1,215 @@ +/* + * 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.search.Scorable; +import org.elasticsearch.core.CheckedConsumer; +import org.elasticsearch.core.Tuple; +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.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.function.AggregatorFunction; +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; +import java.util.Map.Entry; + +@SuppressWarnings({ "unchecked", "rawtypes" }) +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, + org.elasticsearch.search.aggregations.timeseries.aggregation.Aggregator aggregator, + Map aggregatorParams, + Downsample downsample, + BucketCountThresholds bucketCountThresholds, + BucketOrder order, + long startTime, + long endTime, + ValuesSourceConfig valuesSourceConfig, + AggregationContext context, + Aggregator parent, + CardinalityUpperBound bucketCardinality, + Map metadata + ) throws IOException { + super( + name, + factories, + keyed, + group, + without, + interval, + offset, + aggregator, + aggregatorParams, + downsample, + bucketCountThresholds, + order, + startTime, + endTime, + 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_exact_over_time; + } + rewriteFunction(); + } + + @Override + protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, 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(aggCtx.getLeafReaderContext(), sub, aggCtx); + } + + @Override + protected LeafBucketCollector getCollector( + AggregationExecutionContext aggCtx + ) throws IOException { + Metric metricType = getAggregateMetric(); + if (metricType != null) { + 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; + } + } + } + } + }; + 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); + CheckedConsumer docConsumer = (doc) -> { + if (aggCtx.getTimestamp() + downsampleRange < preRounding) { + return; + } + + 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; + } + } + + 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 Tuple<>(sum, valueCount)); + } else { + break; + } + } + }; + + return new DeferringCollector(aggregateSums, aggCtx, docConsumer); + } + } + + 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: + return Metric.max; + case min_over_time: + return Metric.min; + case sum_over_time: + return Metric.sum; + case count_over_time: + 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 + ); + } }