diff --git a/docs/changelog/103511.yaml b/docs/changelog/103511.yaml new file mode 100644 index 0000000000000..20a48df914832 --- /dev/null +++ b/docs/changelog/103511.yaml @@ -0,0 +1,6 @@ +pr: 103511 +summary: Downsampling supports `date_histogram` with tz +area: Downsampling +type: bug +issues: + - 101309 diff --git a/docs/reference/data-streams/downsampling.asciidoc b/docs/reference/data-streams/downsampling.asciidoc index 5e31a90bfb959..cac73787fc018 100644 --- a/docs/reference/data-streams/downsampling.asciidoc +++ b/docs/reference/data-streams/downsampling.asciidoc @@ -135,7 +135,29 @@ downsampled. * For <>, only `fixed_intervals` (and not calendar-aware intervals) are supported. -* Only Coordinated Universal Time (UTC) date-times are supported. +* Timezone support comes with caveats: + +** Date histograms at intervals that are multiples of an hour are based on +values generated at UTC. This works well for timezones that are on the hour, e.g. ++5:00 or -3:00, but requires offsetting the reported time buckets, e.g. +`2020-01-01T10:30:00.000` instead of `2020-03-07T10:00:00.000` for +timezone +5:30 (India), if downsampling aggregates values per hour. In this case, +the results include the field `downsampled_results_offset: true`, to indicate that +the time buckets are shifted. This can be avoided if a downsampling interval of 15 +minutes is used, as it allows properly calculating hourly values for the shifted +buckets. + +** Date histograms at intervals that are multiples of a day are similarly +affected, in case downsampling aggregates values per day. In this case, the +beginning of each day is always calculated at UTC when generated the downsampled +values, so the time buckets need to be shifted, e.g. reported as +`2020-03-07T19:00:00.000` instead of `2020-03-07T00:00:00.000` for timezone `America/New_York`. +The field `downsampled_results_offset: true` is added in this case too. + +** Daylight savings and similar peculiarities around timezones affect +reported results, as <> +for date histogram aggregation. Besides, downsampling at daily interval +hinders tracking any information related to daylight savings changes. [discrete] [[downsampling-restrictions]] diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java index 5c19edc14075b..3674cef73ec3a 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersions.java +++ b/server/src/main/java/org/elasticsearch/TransportVersions.java @@ -191,6 +191,7 @@ static TransportVersion def(int id) { public static final TransportVersion MISSED_INDICES_UPDATE_EXCEPTION_ADDED = def(8_558_00_0); public static final TransportVersion INFERENCE_SERVICE_EMBEDDING_SIZE_ADDED = def(8_559_00_0); public static final TransportVersion ENRICH_ELASTICSEARCH_VERSION_REMOVED = def(8_560_00_0); + public static final TransportVersion DATE_HISTOGRAM_SUPPORT_DOWNSAMPLED_TZ = def(8_560_00_1); /* * STOP! READ THIS FIRST! No, really, diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetadata.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetadata.java index 742b52365c8d7..83b1c48e69eb9 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetadata.java @@ -26,7 +26,6 @@ import org.elasticsearch.cluster.routing.allocation.IndexMetadataUpdater; import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider; import org.elasticsearch.cluster.routing.allocation.decider.ShardsLimitAllocationDecider; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.io.stream.StreamInput; @@ -138,14 +137,9 @@ public class IndexMetadata implements Diffable, ToXContentFragmen EnumSet.of(ClusterBlockLevel.WRITE) ); - // TODO: refactor this method after adding more downsampling metadata - public boolean isDownsampledIndex() { - final String sourceIndex = settings.get(IndexMetadata.INDEX_DOWNSAMPLE_SOURCE_NAME_KEY); - final String indexDownsamplingStatus = settings.get(IndexMetadata.INDEX_DOWNSAMPLE_STATUS_KEY); - final boolean downsamplingSuccess = DownsampleTaskStatus.SUCCESS.name() - .toLowerCase(Locale.ROOT) - .equals(indexDownsamplingStatus != null ? indexDownsamplingStatus.toLowerCase(Locale.ROOT) : DownsampleTaskStatus.UNKNOWN); - return Strings.isNullOrEmpty(sourceIndex) == false && downsamplingSuccess; + @Nullable + public String getDownsamplingInterval() { + return settings.get(IndexMetadata.INDEX_DOWNSAMPLE_INTERVAL_KEY); } public enum State implements Writeable { @@ -1235,6 +1229,7 @@ public Index getResizeSourceIndex() { public static final String INDEX_DOWNSAMPLE_ORIGIN_UUID_KEY = "index.downsample.origin.uuid"; public static final String INDEX_DOWNSAMPLE_STATUS_KEY = "index.downsample.status"; + public static final String INDEX_DOWNSAMPLE_INTERVAL_KEY = "index.downsample.interval"; public static final Setting INDEX_DOWNSAMPLE_SOURCE_UUID = Setting.simpleString( INDEX_DOWNSAMPLE_SOURCE_UUID_KEY, Property.IndexScope, @@ -1277,6 +1272,14 @@ public String toString() { Property.InternalIndex ); + public static final Setting INDEX_DOWNSAMPLE_INTERVAL = Setting.simpleString( + INDEX_DOWNSAMPLE_INTERVAL_KEY, + "", + Property.IndexScope, + Property.InternalIndex, + Property.PrivateIndex + ); + // LIFECYCLE_NAME is here an as optimization, see LifecycleSettings.LIFECYCLE_NAME and // LifecycleSettings.LIFECYCLE_NAME_SETTING for the 'real' version public static final String LIFECYCLE_NAME = "index.lifecycle.name"; diff --git a/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java b/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java index f7c9e72d36326..c1b8d51c255db 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java @@ -76,6 +76,7 @@ public final class IndexScopedSettings extends AbstractScopedSettings { IndexMetadata.INDEX_DOWNSAMPLE_ORIGIN_NAME, IndexMetadata.INDEX_DOWNSAMPLE_ORIGIN_UUID, IndexMetadata.INDEX_DOWNSAMPLE_STATUS, + IndexMetadata.INDEX_DOWNSAMPLE_INTERVAL, SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_DEBUG_SETTING, SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_WARN_SETTING, SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_INFO_SETTING, diff --git a/server/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java b/server/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java index 30a85f4941105..9a8800c05bdb2 100644 --- a/server/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java +++ b/server/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java @@ -285,6 +285,13 @@ public IndexSettings getIndexSettings() { return indexSettings; } + /** + * Returns the MappingLookup for the queried index. + */ + public MappingLookup getMappingLookup() { + return mappingLookup; + } + /** * Given an index pattern, checks whether it matches against the current shard. The pattern * may represent a fully qualified index name if the search targets remote shards. diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java index 4f94e2061caa1..c164067ea6504 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java @@ -14,6 +14,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.core.TimeValue; +import org.elasticsearch.index.mapper.DataStreamTimestampFieldMapper; import org.elasticsearch.search.aggregations.AggregationBuilder; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactory; @@ -36,6 +37,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.SimpleTimeZone; import java.util.function.Consumer; import static java.util.Map.entry; @@ -406,23 +408,46 @@ protected ValuesSourceAggregatorFactory innerBuild( ) throws IOException { final DateIntervalWrapper.IntervalTypeEnum dateHistogramIntervalType = dateHistogramInterval.getIntervalType(); - if (context.getIndexSettings().getIndexMetadata().isDownsampledIndex() - && DateIntervalWrapper.IntervalTypeEnum.CALENDAR.equals(dateHistogramIntervalType)) { - throw new IllegalArgumentException( - config.getDescription() - + " is not supported for aggregation [" - + getName() - + "] with interval type [" - + dateHistogramIntervalType.getPreferredName() - + "]" - ); - } - + boolean downsampledResultsOffset = false; final ZoneId tz = timeZone(); - if (context.getIndexSettings().getIndexMetadata().isDownsampledIndex() && tz != null && ZoneId.of("UTC").equals(tz) == false) { - throw new IllegalArgumentException( - config.getDescription() + " is not supported for aggregation [" + getName() + "] with timezone [" + tz + "]" - ); + + String downsamplingInterval = context.getIndexSettings().getIndexMetadata().getDownsamplingInterval(); + if (downsamplingInterval != null) { + if (DateIntervalWrapper.IntervalTypeEnum.CALENDAR.equals(dateHistogramIntervalType)) { + throw new IllegalArgumentException( + config.getDescription() + + " is not supported for aggregation [" + + getName() + + "] with interval type [" + + dateHistogramIntervalType.getPreferredName() + + "]" + ); + } + + // Downsampled data in time-series indexes contain aggregated values that get calculated over UTC-based intervals. + // When they get aggregated using a different timezone, the resulting buckets may need to be offset to account for + // the difference between UTC (where stored data refers to) and the requested timezone. For instance: + // a. A TZ shifted by -01:15 over hourly downsampled data will lead to buckets with times XX:45, instead of XX:00 + // b. A TZ shifted by +07:00 over daily downsampled data will lead to buckets with times 07:00, instead of 00:00 + // c. Intervals over DST are approximate, not including gaps in time buckets. This applies to date histogram aggregation in + // general. + if (tz != null && ZoneId.of("UTC").equals(tz) == false && field().equals(DataStreamTimestampFieldMapper.DEFAULT_PATH)) { + + // Get the downsampling interval. + DateHistogramInterval interval = new DateHistogramInterval(downsamplingInterval); + long downsamplingResolution = interval.estimateMillis(); + long aggregationResolution = dateHistogramInterval.getAsFixedInterval().estimateMillis(); + + // If the aggregation resolution is not a multiple of the downsampling resolution, the reported time for each + // bucket needs to be shifted by the mod - in addition to rounding that's applied as usual. + // Note that the aggregation resolution gets shifted to match the specified timezone. Timezone.getOffset() normally expects + // a date but it can also process an offset (interval) in milliseconds as it uses the Unix epoch for reference. + long aggregationOffset = SimpleTimeZone.getTimeZone(tz).getOffset(aggregationResolution) % downsamplingResolution; + if (aggregationOffset != 0) { + downsampledResultsOffset = true; + offset += aggregationOffset; + } + } } DateHistogramAggregationSupplier aggregatorSupplier = context.getValuesSourceRegistry().getAggregator(REGISTRY_KEY, config); @@ -473,6 +498,7 @@ protected ValuesSourceAggregatorFactory innerBuild( order, keyed, minDocCount, + downsampledResultsOffset, rounding, roundedBounds, roundedHardBounds, diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationSupplier.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationSupplier.java index 1529d0fab6cc9..b3f002e8b83a7 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationSupplier.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationSupplier.java @@ -29,6 +29,7 @@ Aggregator build( BucketOrder order, boolean keyed, long minDocCount, + boolean downsampledResultsOffset, @Nullable LongBounds extendedBounds, @Nullable LongBounds hardBounds, ValuesSourceConfig valuesSourceConfig, diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java index ea08d5960d704..8f5323dfc9d2b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java @@ -79,6 +79,7 @@ public static Aggregator build( BucketOrder order, boolean keyed, long minDocCount, + boolean downsampledResultsOffset, @Nullable LongBounds extendedBounds, @Nullable LongBounds hardBounds, ValuesSourceConfig valuesSourceConfig, @@ -96,6 +97,7 @@ public static Aggregator build( order, keyed, minDocCount, + downsampledResultsOffset, extendedBounds, hardBounds, valuesSourceConfig, @@ -115,6 +117,7 @@ public static Aggregator build( order, keyed, minDocCount, + downsampledResultsOffset, extendedBounds, hardBounds, valuesSourceConfig, @@ -133,6 +136,7 @@ private static FromDateRange adaptIntoRangeOrNull( BucketOrder order, boolean keyed, long minDocCount, + boolean downsampledResultsOffset, @Nullable LongBounds extendedBounds, @Nullable LongBounds hardBounds, ValuesSourceConfig valuesSourceConfig, @@ -191,6 +195,7 @@ private static FromDateRange adaptIntoRangeOrNull( minDocCount, extendedBounds, keyed, + downsampledResultsOffset, fixedRoundingPoints ); } @@ -227,6 +232,7 @@ private static RangeAggregator.Range[] ranges(LongBounds hardBounds, long[] fixe private final boolean keyed; private final long minDocCount; + private final boolean downsampledResultsOffset; private final LongBounds extendedBounds; private final LongBounds hardBounds; @@ -240,6 +246,7 @@ private static RangeAggregator.Range[] ranges(LongBounds hardBounds, long[] fixe BucketOrder order, boolean keyed, long minDocCount, + boolean downsampledResultsOffset, @Nullable LongBounds extendedBounds, @Nullable LongBounds hardBounds, ValuesSourceConfig valuesSourceConfig, @@ -255,6 +262,7 @@ private static RangeAggregator.Range[] ranges(LongBounds hardBounds, long[] fixe order.validate(this); this.keyed = keyed; this.minDocCount = minDocCount; + this.downsampledResultsOffset = downsampledResultsOffset; this.extendedBounds = extendedBounds; this.hardBounds = hardBounds; // TODO: Stop using null here @@ -328,6 +336,7 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I emptyBucketInfo, formatter, keyed, + downsampledResultsOffset, metadata() ); }); @@ -347,6 +356,7 @@ public InternalAggregation buildEmptyAggregation() { emptyBucketInfo, formatter, keyed, + downsampledResultsOffset, metadata() ); } @@ -392,6 +402,7 @@ static class FromDateRange extends AdaptingAggregator implements SizedBucketAggr private final long minDocCount; private final LongBounds extendedBounds; private final boolean keyed; + private final boolean downsampledResultsOffset; private final long[] fixedRoundingPoints; FromDateRange( @@ -405,6 +416,7 @@ static class FromDateRange extends AdaptingAggregator implements SizedBucketAggr long minDocCount, LongBounds extendedBounds, boolean keyed, + boolean downsampledResultsOffset, long[] fixedRoundingPoints ) throws IOException { super(parent, subAggregators, delegate); @@ -416,6 +428,7 @@ static class FromDateRange extends AdaptingAggregator implements SizedBucketAggr this.minDocCount = minDocCount; this.extendedBounds = extendedBounds; this.keyed = keyed; + this.downsampledResultsOffset = downsampledResultsOffset; this.fixedRoundingPoints = fixedRoundingPoints; } @@ -454,6 +467,7 @@ protected InternalAggregation adapt(InternalAggregation delegateResult) { emptyBucketInfo, format, keyed, + downsampledResultsOffset, range.getMetadata() ); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorFactory.java index 1a75766c40a6b..bb12f4588ef80 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorFactory.java @@ -52,6 +52,7 @@ public static void registerAggregators(ValuesSourceRegistry.Builder builder) { order, keyed, minDocCount, + downsampledResultsOffset, extendedBounds, hardBounds, valuesSourceConfig, @@ -71,6 +72,7 @@ public static void registerAggregators(ValuesSourceRegistry.Builder builder) { order, keyed, minDocCount, + downsampledResultsOffset, extendedBounds, hardBounds, valuesSourceConfig, @@ -88,6 +90,7 @@ public static void registerAggregators(ValuesSourceRegistry.Builder builder) { private final BucketOrder order; private final boolean keyed; private final long minDocCount; + private final boolean downsampledResultsOffset; private final LongBounds extendedBounds; private final LongBounds hardBounds; private final Rounding rounding; @@ -98,6 +101,7 @@ public DateHistogramAggregatorFactory( BucketOrder order, boolean keyed, long minDocCount, + boolean downsampledResultsOffset, Rounding rounding, LongBounds extendedBounds, LongBounds hardBounds, @@ -111,6 +115,7 @@ public DateHistogramAggregatorFactory( this.aggregatorSupplier = aggregationSupplier; this.order = order; this.keyed = keyed; + this.downsampledResultsOffset = downsampledResultsOffset; this.minDocCount = minDocCount; this.extendedBounds = extendedBounds; this.hardBounds = hardBounds; @@ -139,6 +144,7 @@ protected Aggregator doCreateInternal(Aggregator parent, CardinalityUpperBound c order, keyed, minDocCount, + downsampledResultsOffset, extendedBounds, hardBounds, config, @@ -159,6 +165,7 @@ protected Aggregator createUnmapped(Aggregator parent, Map metad order, keyed, minDocCount, + downsampledResultsOffset, extendedBounds, hardBounds, config, diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateRangeHistogramAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateRangeHistogramAggregator.java index 34720f3f2f643..5fe44aa694cc5 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateRangeHistogramAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateRangeHistogramAggregator.java @@ -59,6 +59,7 @@ class DateRangeHistogramAggregator extends BucketsAggregator { private final boolean keyed; private final long minDocCount; + private final boolean downsampledResultsOffset; private final LongBounds extendedBounds; private final LongBounds hardBounds; @@ -71,6 +72,7 @@ class DateRangeHistogramAggregator extends BucketsAggregator { BucketOrder order, boolean keyed, long minDocCount, + boolean downsampledResultsOffset, @Nullable LongBounds extendedBounds, @Nullable LongBounds hardBounds, ValuesSourceConfig valuesSourceConfig, @@ -87,6 +89,7 @@ class DateRangeHistogramAggregator extends BucketsAggregator { order.validate(this); this.keyed = keyed; this.minDocCount = minDocCount; + this.downsampledResultsOffset = downsampledResultsOffset; this.extendedBounds = extendedBounds; this.hardBounds = hardBounds; // TODO: Stop using null here @@ -197,6 +200,7 @@ public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws I emptyBucketInfo, formatter, keyed, + downsampledResultsOffset, metadata() ); } @@ -217,6 +221,7 @@ public InternalAggregation buildEmptyAggregation() { emptyBucketInfo, formatter, keyed, + downsampledResultsOffset, metadata() ); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java index 4ffc9abdc2202..449326b1d69bb 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java @@ -9,6 +9,7 @@ import org.apache.lucene.util.CollectionUtil; import org.apache.lucene.util.PriorityQueue; +import org.elasticsearch.TransportVersions; import org.elasticsearch.common.Rounding; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -202,6 +203,7 @@ public int hashCode() { private final BucketOrder order; private final DocValueFormat format; private final boolean keyed; + private final boolean downsampledResultsOffset; private final long minDocCount; private final long offset; final EmptyBucketInfo emptyBucketInfo; @@ -215,6 +217,7 @@ public int hashCode() { EmptyBucketInfo emptyBucketInfo, DocValueFormat formatter, boolean keyed, + boolean downsampledResultsOffset, Map metadata ) { super(name, metadata); @@ -226,6 +229,7 @@ public int hashCode() { this.emptyBucketInfo = emptyBucketInfo; this.format = formatter; this.keyed = keyed; + this.downsampledResultsOffset = downsampledResultsOffset; } /** @@ -243,6 +247,11 @@ public InternalDateHistogram(StreamInput in) throws IOException { offset = in.readLong(); format = in.readNamedWriteable(DocValueFormat.class); keyed = in.readBoolean(); + if (in.getTransportVersion().onOrAfter(TransportVersions.DATE_HISTOGRAM_SUPPORT_DOWNSAMPLED_TZ)) { + downsampledResultsOffset = in.readBoolean(); + } else { + downsampledResultsOffset = false; + } buckets = in.readCollectionAsList(stream -> new Bucket(stream, keyed, format)); } @@ -256,6 +265,9 @@ protected void doWriteTo(StreamOutput out) throws IOException { out.writeLong(offset); out.writeNamedWriteable(format); out.writeBoolean(keyed); + if (out.getTransportVersion().onOrAfter(TransportVersions.DATE_HISTOGRAM_SUPPORT_DOWNSAMPLED_TZ)) { + out.writeBoolean(downsampledResultsOffset); + } out.writeCollection(buckets); } @@ -283,7 +295,18 @@ BucketOrder getOrder() { @Override public InternalDateHistogram create(List buckets) { - return new InternalDateHistogram(name, buckets, order, minDocCount, offset, emptyBucketInfo, format, keyed, metadata); + return new InternalDateHistogram( + name, + buckets, + order, + minDocCount, + offset, + emptyBucketInfo, + format, + keyed, + downsampledResultsOffset, + metadata + ); } @Override @@ -508,6 +531,7 @@ public InternalAggregation reduce(List aggregations, Aggreg emptyBucketInfo, format, keyed, + downsampledResultsOffset, getMetadata() ); } @@ -523,6 +547,7 @@ public InternalAggregation finalizeSampling(SamplingContext samplingContext) { emptyBucketInfo, format, keyed, + downsampledResultsOffset, getMetadata() ); } @@ -542,6 +567,12 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } else { builder.endArray(); } + if (downsampledResultsOffset) { + // Indicates that the dates reported in the buckets over downsampled indexes are offset + // to match the intervals at UTC, since downsampling always uses UTC-based intervals + // to calculate aggregated values. + builder.field("downsampled_results_offset", Boolean.TRUE); + } return builder; } @@ -570,7 +601,18 @@ public InternalAggregation createAggregation(List sourcePath(String fullName); + /** + * Returns the MappingLookup for the index, if one is initialized. + */ + @Nullable + public MappingLookup getMappingLookup() { + return null; + } + /** * Does this index have a {@code _doc_count} field in any segment? */ @@ -611,6 +620,11 @@ public Set sourcePath(String fullName) { return context.sourcePath(fullName); } + @Override + public MappingLookup getMappingLookup() { + return context.getMappingLookup(); + } + @Override public void close() { /* diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogramTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogramTests.java index 093ccc7181767..512784353a099 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogramTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogramTests.java @@ -109,7 +109,7 @@ private InternalDateHistogram createTestInstance( } } BucketOrder order = BucketOrder.key(randomBoolean()); - return new InternalDateHistogram(name, buckets, order, minDocCount, 0L, emptyBucketInfo, format, keyed, metadata); + return new InternalDateHistogram(name, buckets, order, minDocCount, 0L, emptyBucketInfo, format, keyed, false, metadata); } @Override @@ -210,7 +210,7 @@ protected InternalDateHistogram mutateInstance(InternalDateHistogram instance) { } default -> throw new AssertionError("Illegal randomisation branch"); } - return new InternalDateHistogram(name, buckets, order, minDocCount, offset, emptyBucketInfo, format, keyed, metadata); + return new InternalDateHistogram(name, buckets, order, minDocCount, offset, emptyBucketInfo, format, keyed, false, metadata); } public void testLargeReduce() { @@ -230,6 +230,7 @@ public void testLargeReduce() { ), DocValueFormat.RAW, false, + false, null ); expectReduceUsesTooManyBuckets(largeHisto, 100000); diff --git a/x-pack/plugin/downsample/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/downsample/30_date_histogram.yml b/x-pack/plugin/downsample/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/downsample/30_date_histogram.yml index b7f3ec7b8f384..831ad158deda4 100644 --- a/x-pack/plugin/downsample/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/downsample/30_date_histogram.yml +++ b/x-pack/plugin/downsample/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/downsample/30_date_histogram.yml @@ -13,8 +13,8 @@ setup: mode: time_series routing_path: [ uid ] time_series: - start_time: 2021-04-28T00:00:00Z - end_time: 2021-04-29T00:00:00Z + start_time: 2020-01-01T00:00:00Z + end_time: 2022-01-01T00:00:00Z mappings: properties: "@timestamp": @@ -39,12 +39,6 @@ setup: - '{ "index": {} }' - '{ "@timestamp": "2021-04-28T18:55:00Z", "uid": "004", "total_memory_used": 120770 }' - - do: - indices.put_settings: - index: test - body: - index.blocks.write: true - --- "Date histogram aggregation on time series index and rollup indices": - skip: @@ -52,6 +46,12 @@ setup: reason: "rollup: unsupported aggregations errors added in 8.5.0" features: close_to + - do: + indices.put_settings: + index: test + body: + index.blocks.write: true + - do: indices.downsample: index: test @@ -142,24 +142,6 @@ setup: - match: { aggregations.date_histogram.buckets.0.key_as_string: "2021-04-28T18:00:00.000Z"} - match: { aggregations.date_histogram.buckets.0.key: 1619632800000 } - # date histogram aggregation with non-utc timezone on rollup index not supported - - do: - catch: bad_request - search: - index: test-downsample - body: - size: 0 - aggs: - date_histogram: - date_histogram: - field: "@timestamp" - fixed_interval: 1h - time_zone: "America/New_York" - - - match: { status: 400 } - - match: { error.root_cause.0.type: illegal_argument_exception } - - match: { error.root_cause.0.reason: "Field [@timestamp] of type [date] is not supported for aggregation [date_histogram] with timezone [America/New_York]" } - # date histogram aggregation with non-utc timezone on time series index supported - do: search: @@ -247,3 +229,485 @@ setup: - match: { _shards.failures.0.index: "test-downsample" } - match: { _shards.failures.0.reason.type: illegal_argument_exception } - match: { _shards.failures.0.reason.reason: "Field [@timestamp] of type [date] is not supported for aggregation [date_histogram] with interval type [calendar_interval]" } + +--- +timezone support - 15m: + - skip: + version: " - 8.12.99" + reason: "timezone support added in 8.13" + + - do: + bulk: + refresh: true + index: test + body: + # Check timezone support + - '{ "index": {} }' + - '{ "@timestamp": "2021-04-28T10:05:00Z", "uid": "001", "total_memory_used": 106780 }' + - '{ "index": {} }' + - '{ "@timestamp": "2021-04-28T10:55:00Z", "uid": "001", "total_memory_used": 110450 }' + - '{ "index": {} }' + - '{ "@timestamp": "2021-04-28T11:05:00Z", "uid": "001", "total_memory_used": 110450 }' + - '{ "index": {} }' + - '{ "@timestamp": "2021-04-28T11:55:00Z", "uid": "001", "total_memory_used": 109009 }' + - '{ "index": {} }' + - '{ "@timestamp": "2021-04-28T12:05:00Z", "uid": "001", "total_memory_used": 120770 }' + # Check daylight savings + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T03:00:00Z", "uid": "001", "total_memory_used": 106780 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T03:50:00Z", "uid": "001", "total_memory_used": 110450 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T04:00:00Z", "uid": "001", "total_memory_used": 110450 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T04:50:00Z", "uid": "001", "total_memory_used": 109009 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T05:00:00Z", "uid": "001", "total_memory_used": 109009 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T06:00:00Z", "uid": "001", "total_memory_used": 106780 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T07:50:00Z", "uid": "001", "total_memory_used": 110450 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T08:00:00Z", "uid": "001", "total_memory_used": 110450 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T08:50:00Z", "uid": "001", "total_memory_used": 109009 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T09:00:00Z", "uid": "001", "total_memory_used": 109009 }' + + - do: + indices.put_settings: + index: test + body: + index.blocks.write: true + + - do: + indices.downsample: + index: test + target_index: test-downsample + body: > + { + "fixed_interval": "15m" + } + + - is_true: acknowledged + + - do: + search: + index: test-downsample + body: + size: 0 + aggs: + date_histogram: + date_histogram: + field: "@timestamp" + fixed_interval: 1h + time_zone: "America/New_York" + query: + range: + "@timestamp": + gt: "2021-04-28T15:00:00Z" + lt: "2021-04-29T15:00:00Z" + + - match: { hits.total.value: 4 } + - length: { aggregations.date_histogram.buckets: 1 } + - match: { aggregations.date_histogram.buckets.0.doc_count: 4 } + - match: { aggregations.date_histogram.buckets.0.key_as_string: "2021-04-28T14:00:00.000-04:00" } + - match: { aggregations.date_histogram.buckets.0.key: 1619632800000 } + - is_false: aggregations.date_histogram.downsampled_results_offset + + - do: + search: + index: test-downsample + body: + size: 0 + aggs: + date_histogram: + date_histogram: + field: "@timestamp" + fixed_interval: 1h + time_zone: "-01:15" + query: + range: + "@timestamp": + gt: "2021-04-27T15:00:00Z" + lt: "2021-04-28T15:00:00Z" + + - match: { hits.total.value: 5 } + - length: { aggregations.date_histogram.buckets: 3 } + - match: { aggregations.date_histogram.buckets.0.doc_count: 1 } + - match: { aggregations.date_histogram.buckets.0.key_as_string: "2021-04-28T08:00:00.000-01:15" } + - match: { aggregations.date_histogram.buckets.1.doc_count: 2 } + - match: { aggregations.date_histogram.buckets.1.key_as_string: "2021-04-28T09:00:00.000-01:15" } + - match: { aggregations.date_histogram.buckets.2.doc_count: 2 } + - match: { aggregations.date_histogram.buckets.2.key_as_string: "2021-04-28T10:00:00.000-01:15" } + - is_false: aggregations.date_histogram.downsampled_results_offset + + - do: + search: + index: test-downsample + body: + size: 0 + aggs: + date_histogram: + date_histogram: + field: "@timestamp" + fixed_interval: 1h + time_zone: "+02:15" + query: + range: + "@timestamp": + gt: "2021-04-27T15:00:00Z" + lt: "2021-04-28T15:00:00Z" + + - match: { hits.total.value: 5 } + - length: { aggregations.date_histogram.buckets: 3 } + - match: { aggregations.date_histogram.buckets.0.doc_count: 1 } + - match: { aggregations.date_histogram.buckets.0.key_as_string: "2021-04-28T12:00:00.000+02:15" } + - match: { aggregations.date_histogram.buckets.1.doc_count: 2 } + - match: { aggregations.date_histogram.buckets.1.key_as_string: "2021-04-28T13:00:00.000+02:15" } + - match: { aggregations.date_histogram.buckets.2.doc_count: 2 } + - match: { aggregations.date_histogram.buckets.2.key_as_string: "2021-04-28T14:00:00.000+02:15" } + - is_false: aggregations.date_histogram.downsampled_results_offset + + # Check timezone with daylight savings + - do: + search: + index: test-downsample + body: + size: 0 + aggs: + date_histogram: + date_histogram: + field: "@timestamp" + fixed_interval: 1h + time_zone: "America/New_York" + query: + range: + "@timestamp": + gt: "2020-03-08T00:00:00Z" + lt: "2020-03-10T00:00:00Z" + + - match: { hits.total.value: 10 } + - length: { aggregations.date_histogram.buckets: 7 } + - match: { aggregations.date_histogram.buckets.0.doc_count: 2 } + - match: { aggregations.date_histogram.buckets.0.key_as_string: "2020-03-08T23:00:00.000-04:00" } + - match: { aggregations.date_histogram.buckets.1.doc_count: 2 } + - match: { aggregations.date_histogram.buckets.1.key_as_string: "2020-03-09T00:00:00.000-04:00" } + - match: { aggregations.date_histogram.buckets.2.doc_count: 1 } + - match: { aggregations.date_histogram.buckets.2.key_as_string: "2020-03-09T01:00:00.000-04:00" } + - match: { aggregations.date_histogram.buckets.3.doc_count: 1 } + - match: { aggregations.date_histogram.buckets.3.key_as_string: "2020-03-09T02:00:00.000-04:00" } + - match: { aggregations.date_histogram.buckets.4.doc_count: 1 } + - match: { aggregations.date_histogram.buckets.4.key_as_string: "2020-03-09T03:00:00.000-04:00" } + - match: { aggregations.date_histogram.buckets.5.doc_count: 2 } + - match: { aggregations.date_histogram.buckets.5.key_as_string: "2020-03-09T04:00:00.000-04:00" } + - match: { aggregations.date_histogram.buckets.6.doc_count: 1 } + - match: { aggregations.date_histogram.buckets.6.key_as_string: "2020-03-09T05:00:00.000-04:00" } + - is_false: aggregations.date_histogram.downsampled_results_offset + + - do: + search: + index: test-downsample + body: + size: 0 + aggs: + date_histogram: + date_histogram: + field: "@timestamp" + fixed_interval: 1d + time_zone: "America/New_York" + query: + range: + "@timestamp": + gt: "2020-03-08T00:00:00Z" + lt: "2020-03-10T00:00:00Z" + + - match: { hits.total.value: 10 } + - length: { aggregations.date_histogram.buckets: 2 } + - match: { aggregations.date_histogram.buckets.0.doc_count: 2 } + - match: { aggregations.date_histogram.buckets.0.key_as_string: "2020-03-08T00:00:00.000-05:00" } + - match: { aggregations.date_histogram.buckets.1.doc_count: 8 } + - match: { aggregations.date_histogram.buckets.1.key_as_string: "2020-03-09T00:00:00.000-04:00" } + - is_false: aggregations.date_histogram.downsampled_results_offset + +--- +timezone support - 1h: + - skip: + version: " - 8.12.99" + reason: "timezone support added in 8.13" + + - do: + bulk: + refresh: true + index: test + body: + # Check timezone support + - '{ "index": {} }' + - '{ "@timestamp": "2021-04-28T10:05:00Z", "uid": "001", "total_memory_used": 106780 }' + - '{ "index": {} }' + - '{ "@timestamp": "2021-04-28T10:55:00Z", "uid": "001", "total_memory_used": 110450 }' + - '{ "index": {} }' + - '{ "@timestamp": "2021-04-28T11:05:00Z", "uid": "001", "total_memory_used": 110450 }' + - '{ "index": {} }' + - '{ "@timestamp": "2021-04-28T11:55:00Z", "uid": "001", "total_memory_used": 109009 }' + - '{ "index": {} }' + - '{ "@timestamp": "2021-04-28T12:05:00Z", "uid": "001", "total_memory_used": 120770 }' + # Check daylight savings + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T03:00:00Z", "uid": "001", "total_memory_used": 106780 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T03:50:00Z", "uid": "001", "total_memory_used": 110450 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T04:00:00Z", "uid": "001", "total_memory_used": 110450 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T04:50:00Z", "uid": "001", "total_memory_used": 109009 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T05:00:00Z", "uid": "001", "total_memory_used": 109009 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T06:00:00Z", "uid": "001", "total_memory_used": 106780 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T07:50:00Z", "uid": "001", "total_memory_used": 110450 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T08:00:00Z", "uid": "001", "total_memory_used": 110450 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T08:50:00Z", "uid": "001", "total_memory_used": 109009 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T09:00:00Z", "uid": "001", "total_memory_used": 109009 }' + + - do: + indices.put_settings: + index: test + body: + index.blocks.write: true + + - do: + indices.downsample: + index: test + target_index: test-downsample + body: > + { + "fixed_interval": "1h" + } + + - is_true: acknowledged + + - do: + search: + index: test-downsample + body: + size: 0 + aggs: + date_histogram: + date_histogram: + field: "@timestamp" + fixed_interval: 1h + time_zone: "America/New_York" + query: + range: + "@timestamp": + gt: "2021-04-28T15:00:00Z" + lt: "2021-04-29T15:00:00Z" + + - match: { hits.total.value: 4 } + - length: { aggregations.date_histogram.buckets: 1 } + - match: { aggregations.date_histogram.buckets.0.doc_count: 4 } + - match: { aggregations.date_histogram.buckets.0.key_as_string: "2021-04-28T14:00:00.000-04:00" } + - match: { aggregations.date_histogram.buckets.0.key: 1619632800000 } + - is_false: aggregations.date_histogram.downsampled_results_offset + + - do: + search: + index: test-downsample + body: + size: 0 + aggs: + date_histogram: + date_histogram: + field: "@timestamp" + fixed_interval: 1h + time_zone: "-01:15" + query: + range: + "@timestamp": + gt: "2021-04-27T15:00:00Z" + lt: "2021-04-28T15:00:00Z" + + - match: { hits.total.value: 3 } + - match: { aggregations.date_histogram.downsampled_results_offset: true } + - length: { aggregations.date_histogram.buckets: 3 } + - match: { aggregations.date_histogram.buckets.0.doc_count: 2 } + - match: { aggregations.date_histogram.buckets.0.key_as_string: "2021-04-28T08:45:00.000-01:15" } + - match: { aggregations.date_histogram.buckets.1.doc_count: 2 } + - match: { aggregations.date_histogram.buckets.1.key_as_string: "2021-04-28T09:45:00.000-01:15" } + - match: { aggregations.date_histogram.buckets.2.doc_count: 1 } + - match: { aggregations.date_histogram.buckets.2.key_as_string: "2021-04-28T10:45:00.000-01:15" } + + - do: + search: + index: test-downsample + body: + size: 0 + aggs: + date_histogram: + date_histogram: + field: "@timestamp" + fixed_interval: 1h + time_zone: "+02:15" + query: + range: + "@timestamp": + gt: "2021-04-27T15:00:00Z" + lt: "2021-04-28T15:00:00Z" + + - match: { hits.total.value: 3 } + - match: { aggregations.date_histogram.downsampled_results_offset: true } + - length: { aggregations.date_histogram.buckets: 3 } + - match: { aggregations.date_histogram.buckets.0.doc_count: 2 } + - match: { aggregations.date_histogram.buckets.0.key_as_string: "2021-04-28T12:15:00.000+02:15" } + - match: { aggregations.date_histogram.buckets.1.doc_count: 2 } + - match: { aggregations.date_histogram.buckets.1.key_as_string: "2021-04-28T13:15:00.000+02:15" } + - match: { aggregations.date_histogram.buckets.2.doc_count: 1 } + - match: { aggregations.date_histogram.buckets.2.key_as_string: "2021-04-28T14:15:00.000+02:15" } + + # Check timezone with daylight savings + - do: + search: + index: test-downsample + body: + size: 0 + aggs: + date_histogram: + date_histogram: + field: "@timestamp" + fixed_interval: 1h + time_zone: "America/New_York" + query: + range: + "@timestamp": + gt: "2020-03-08T00:00:00Z" + lt: "2020-03-10T00:00:00Z" + + - match: { hits.total.value: 7 } + - length: { aggregations.date_histogram.buckets: 7 } + - match: { aggregations.date_histogram.buckets.0.doc_count: 2 } + - match: { aggregations.date_histogram.buckets.0.key_as_string: "2020-03-08T23:00:00.000-04:00" } + - match: { aggregations.date_histogram.buckets.1.doc_count: 2 } + - match: { aggregations.date_histogram.buckets.1.key_as_string: "2020-03-09T00:00:00.000-04:00" } + - match: { aggregations.date_histogram.buckets.2.doc_count: 1 } + - match: { aggregations.date_histogram.buckets.2.key_as_string: "2020-03-09T01:00:00.000-04:00" } + - match: { aggregations.date_histogram.buckets.3.doc_count: 1 } + - match: { aggregations.date_histogram.buckets.3.key_as_string: "2020-03-09T02:00:00.000-04:00" } + - match: { aggregations.date_histogram.buckets.4.doc_count: 1 } + - match: { aggregations.date_histogram.buckets.4.key_as_string: "2020-03-09T03:00:00.000-04:00" } + - match: { aggregations.date_histogram.buckets.5.doc_count: 2 } + - match: { aggregations.date_histogram.buckets.5.key_as_string: "2020-03-09T04:00:00.000-04:00" } + - match: { aggregations.date_histogram.buckets.6.doc_count: 1 } + - match: { aggregations.date_histogram.buckets.6.key_as_string: "2020-03-09T05:00:00.000-04:00" } + - is_false: aggregations.date_histogram.downsampled_results_offset + + - do: + search: + index: test-downsample + body: + size: 0 + aggs: + date_histogram: + date_histogram: + field: "@timestamp" + fixed_interval: 1d + time_zone: "America/New_York" + query: + range: + "@timestamp": + gt: "2020-03-08T00:00:00Z" + lt: "2020-03-10T00:00:00Z" + + - match: { hits.total.value: 7 } + - length: { aggregations.date_histogram.buckets: 2 } + - match: { aggregations.date_histogram.buckets.0.doc_count: 2 } + - match: { aggregations.date_histogram.buckets.0.key_as_string: "2020-03-08T00:00:00.000-05:00" } + - match: { aggregations.date_histogram.buckets.1.doc_count: 8 } + - match: { aggregations.date_histogram.buckets.1.key_as_string: "2020-03-09T00:00:00.000-04:00" } + - is_false: aggregations.date_histogram.downsampled_results_offset + +--- +timezone support - 1d: + - skip: + version: " - 8.12.99" + reason: "timezone support added in 8.13" + + - do: + bulk: + refresh: true + index: test + body: + # Check daylight savings + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-08T03:00:00Z", "uid": "001", "total_memory_used": 106780 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-08T03:50:00Z", "uid": "001", "total_memory_used": 110450 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T03:00:00Z", "uid": "001", "total_memory_used": 106780 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T03:50:00Z", "uid": "001", "total_memory_used": 110450 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T04:00:00Z", "uid": "001", "total_memory_used": 110450 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T04:50:00Z", "uid": "001", "total_memory_used": 109009 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T05:00:00Z", "uid": "001", "total_memory_used": 109009 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T06:00:00Z", "uid": "001", "total_memory_used": 106780 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T07:50:00Z", "uid": "001", "total_memory_used": 110450 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T08:00:00Z", "uid": "001", "total_memory_used": 110450 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-09T08:50:00Z", "uid": "001", "total_memory_used": 109009 }' + - '{ "index": {} }' + - '{ "@timestamp": "2020-03-10T09:00:00Z", "uid": "001", "total_memory_used": 109009 }' + + - do: + indices.put_settings: + index: test + body: + index.blocks.write: true + + - do: + indices.downsample: + index: test + target_index: test-downsample + body: > + { + "fixed_interval": "1d" + } + + - is_true: acknowledged + + # Check timezone with daylight savings + - do: + search: + index: test-downsample + body: + size: 0 + aggs: + date_histogram: + date_histogram: + field: "@timestamp" + fixed_interval: 1d + time_zone: "America/New_York" + query: + range: + "@timestamp": + gt: "2020-03-01T00:00:00Z" + lt: "2020-03-30T00:00:00Z" + + - match: { hits.total.value: 3 } + - match: { aggregations.date_histogram.downsampled_results_offset: true } + - length: { aggregations.date_histogram.buckets: 3 } + - match: { aggregations.date_histogram.buckets.0.doc_count: 2 } + - match: { aggregations.date_histogram.buckets.0.key_as_string: "2020-03-07T19:00:00.000-05:00" } + - match: { aggregations.date_histogram.buckets.1.doc_count: 9 } + - match: { aggregations.date_histogram.buckets.1.key_as_string: "2020-03-08T19:00:00.000-04:00" } + - match: { aggregations.date_histogram.buckets.2.doc_count: 1 } + - match: { aggregations.date_histogram.buckets.2.key_as_string: "2020-03-09T19:00:00.000-04:00" } diff --git a/x-pack/plugin/downsample/src/internalClusterTest/java/org/elasticsearch/xpack/downsample/DataStreamLifecycleDownsampleDisruptionIT.java b/x-pack/plugin/downsample/src/internalClusterTest/java/org/elasticsearch/xpack/downsample/DataStreamLifecycleDownsampleDisruptionIT.java index 826c958de4c18..f6b623e60d61c 100644 --- a/x-pack/plugin/downsample/src/internalClusterTest/java/org/elasticsearch/xpack/downsample/DataStreamLifecycleDownsampleDisruptionIT.java +++ b/x-pack/plugin/downsample/src/internalClusterTest/java/org/elasticsearch/xpack/downsample/DataStreamLifecycleDownsampleDisruptionIT.java @@ -143,6 +143,7 @@ public boolean validateClusterForming() { Settings indexSettings = getSettingsResponse.getIndexToSettings().get(targetIndex); assertThat(indexSettings, is(notNullValue())); assertThat(IndexMetadata.INDEX_DOWNSAMPLE_STATUS.get(indexSettings), is(IndexMetadata.DownsampleTaskStatus.SUCCESS)); + assertEquals("5m", IndexMetadata.INDEX_DOWNSAMPLE_INTERVAL.get(indexSettings)); } catch (Exception e) { throw new AssertionError(e); } diff --git a/x-pack/plugin/downsample/src/main/java/org/elasticsearch/xpack/downsample/TransportDownsampleAction.java b/x-pack/plugin/downsample/src/main/java/org/elasticsearch/xpack/downsample/TransportDownsampleAction.java index 30b509022639d..cecd017c4f893 100644 --- a/x-pack/plugin/downsample/src/main/java/org/elasticsearch/xpack/downsample/TransportDownsampleAction.java +++ b/x-pack/plugin/downsample/src/main/java/org/elasticsearch/xpack/downsample/TransportDownsampleAction.java @@ -115,6 +115,7 @@ public class TransportDownsampleAction extends AcknowledgedTransportMasterNodeAc private final IndexScopedSettings indexScopedSettings; private final ThreadContext threadContext; private final PersistentTasksService persistentTasksService; + private String downsamplingInterval; private static final Set FORBIDDEN_SETTINGS = Set.of( IndexSettings.DEFAULT_PIPELINE.getKey(), @@ -284,6 +285,7 @@ protected void masterOperation( // Validate downsampling interval validateDownsamplingInterval(mapperService, request.getDownsampleConfig()); + downsamplingInterval = request.getDownsampleConfig().getInterval().toString(); final List dimensionFields = new ArrayList<>(); final List metricFields = new ArrayList<>(); @@ -888,6 +890,7 @@ public ClusterState execute(ClusterState currentState) { Settings.builder() .put(downsampleIndex.getSettings()) .put(IndexMetadata.INDEX_DOWNSAMPLE_STATUS.getKey(), DownsampleTaskStatus.SUCCESS) + .put(IndexMetadata.INDEX_DOWNSAMPLE_INTERVAL.getKey(), downsamplingInterval) .build(), downsampleIndexName ); diff --git a/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/DownsampleActionIT.java b/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/DownsampleActionIT.java index c44c15f7e9949..2c2cadd0f9eee 100644 --- a/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/DownsampleActionIT.java +++ b/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/DownsampleActionIT.java @@ -213,6 +213,7 @@ public void testRollupIndex() throws Exception { assertEquals(index, settings.get(IndexMetadata.INDEX_DOWNSAMPLE_SOURCE_NAME.getKey())); assertEquals(policy, settings.get(LifecycleSettings.LIFECYCLE_NAME_SETTING.getKey())); assertEquals(DownsampleTaskStatus.SUCCESS.toString(), settings.get(IndexMetadata.INDEX_DOWNSAMPLE_STATUS.getKey())); + assertEquals(fixedInterval.toString(), settings.get(IndexMetadata.INDEX_DOWNSAMPLE_INTERVAL.getKey())); }); assertBusy( () -> assertTrue("Alias [" + alias + "] does not point to index [" + rollupIndex + "]", aliasExists(rollupIndex, alias)) @@ -299,6 +300,7 @@ public void testRollupIndexInTheHotPhaseAfterRollover() throws Exception { assertEquals(originalIndex, settings.get(IndexMetadata.INDEX_DOWNSAMPLE_SOURCE_NAME.getKey())); assertEquals(policy, settings.get(LifecycleSettings.LIFECYCLE_NAME_SETTING.getKey())); assertEquals(DownsampleTaskStatus.SUCCESS.toString(), settings.get(IndexMetadata.INDEX_DOWNSAMPLE_STATUS.getKey())); + assertEquals(fixedInterval.toString(), settings.get(IndexMetadata.INDEX_DOWNSAMPLE_INTERVAL.getKey())); }); } @@ -345,6 +347,7 @@ public void testTsdbDataStreams() throws Exception { assertEquals(backingIndexName, settings.get(IndexMetadata.INDEX_DOWNSAMPLE_SOURCE_NAME.getKey())); assertEquals(policy, settings.get(LifecycleSettings.LIFECYCLE_NAME_SETTING.getKey())); assertEquals(DownsampleTaskStatus.SUCCESS.toString(), settings.get(IndexMetadata.INDEX_DOWNSAMPLE_STATUS.getKey())); + assertEquals(fixedInterval.toString(), settings.get(IndexMetadata.INDEX_DOWNSAMPLE_INTERVAL.getKey())); }); } @@ -477,6 +480,7 @@ public void testDownsampleTwice() throws Exception { assertEquals(downsampleIndexName, settings.get(IndexMetadata.INDEX_DOWNSAMPLE_SOURCE_NAME.getKey())); assertEquals(DownsampleTaskStatus.SUCCESS.toString(), settings.get(IndexMetadata.INDEX_DOWNSAMPLE_STATUS.getKey())); assertEquals(policy, settings.get(LifecycleSettings.LIFECYCLE_NAME_SETTING.getKey())); + assertEquals("1h", settings.get(IndexMetadata.INDEX_DOWNSAMPLE_INTERVAL.getKey())); }, 60, TimeUnit.SECONDS); } catch (AssertionError ae) { if (indexExists(firstBackingIndex)) { @@ -557,6 +561,7 @@ public void testDownsampleTwiceSameInterval() throws Exception { assertEquals(firstBackingIndex, settings.get(IndexMetadata.INDEX_DOWNSAMPLE_ORIGIN_NAME.getKey())); assertEquals(firstBackingIndex, settings.get(IndexMetadata.INDEX_DOWNSAMPLE_SOURCE_NAME.getKey())); assertEquals(DownsampleTaskStatus.SUCCESS.toString(), settings.get(IndexMetadata.INDEX_DOWNSAMPLE_STATUS.getKey())); + assertEquals("5m", settings.get(IndexMetadata.INDEX_DOWNSAMPLE_INTERVAL.getKey())); assertEquals(policy, settings.get(LifecycleSettings.LIFECYCLE_NAME_SETTING.getKey())); }, 60, TimeUnit.SECONDS);