Browse Source

Downsampling supports date_histogram with tz (#103511)

* Downsampling supports date_histogram with tz

This comes with caveats, for downsampled indexes at intervals more than
15 minutes. For instance,
 - 1-hour downsampling will produce inaccurate
results for 1-hour histograms on timezones shifted by XX:30
 - 1-day downsampling will produce inaccurate daily
histograms for not-UTC timezones as it tracks days at UTC.

Related to #101309

* Update docs/changelog/103511.yaml

* test daylight savings

* update documentation

* Offset time buckets over downsampled data with TZ

* Update docs/changelog/103511.yaml

* check for TSDS

* fixme for transport version

* add interval to index metadata

* add transport version

* bump up transport version

* address feedbcak

* spotless fix
Kostas Krikellas 1 year ago
parent
commit
c4c2ce83cb
18 changed files with 679 additions and 56 deletions
  1. 6 0
      docs/changelog/103511.yaml
  2. 23 1
      docs/reference/data-streams/downsampling.asciidoc
  3. 1 0
      server/src/main/java/org/elasticsearch/TransportVersions.java
  4. 12 9
      server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetadata.java
  5. 1 0
      server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java
  6. 7 0
      server/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java
  7. 42 16
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java
  8. 1 0
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationSupplier.java
  9. 14 0
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java
  10. 7 0
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorFactory.java
  11. 5 0
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateRangeHistogramAggregator.java
  12. 44 2
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java
  13. 14 0
      server/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java
  14. 3 2
      server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogramTests.java
  15. 490 26
      x-pack/plugin/downsample/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/downsample/30_date_histogram.yml
  16. 1 0
      x-pack/plugin/downsample/src/internalClusterTest/java/org/elasticsearch/xpack/downsample/DataStreamLifecycleDownsampleDisruptionIT.java
  17. 3 0
      x-pack/plugin/downsample/src/main/java/org/elasticsearch/xpack/downsample/TransportDownsampleAction.java
  18. 5 0
      x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/DownsampleActionIT.java

+ 6 - 0
docs/changelog/103511.yaml

@@ -0,0 +1,6 @@
+pr: 103511
+summary: Downsampling supports `date_histogram` with tz
+area: Downsampling
+type: bug
+issues:
+ - 101309

+ 23 - 1
docs/reference/data-streams/downsampling.asciidoc

@@ -135,7 +135,29 @@ downsampled.
 * For
 <<search-aggregations-bucket-datehistogram-aggregation,date histogram aggregations>>,
 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 <<datehistogram-aggregation-time-zone,documented>>
+for date histogram aggregation. Besides, downsampling at daily interval
+hinders tracking any information related to daylight savings changes.
 
 [discrete]
 [[downsampling-restrictions]]

+ 1 - 0
server/src/main/java/org/elasticsearch/TransportVersions.java

@@ -183,6 +183,7 @@ public class TransportVersions {
     public static final TransportVersion HOT_THREADS_AS_BYTES = def(8_571_00_0);
     public static final TransportVersion ML_INFERENCE_REQUEST_INPUT_TYPE_ADDED = def(8_572_00_0);
     public static final TransportVersion ESQL_ENRICH_POLICY_CCQ_MODE = def(8_573_00_0);
+    public static final TransportVersion DATE_HISTOGRAM_SUPPORT_DOWNSAMPLED_TZ = def(8_574_00_0);
 
     /*
      * STOP! READ THIS FIRST! No, really,

+ 12 - 9
server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetadata.java

@@ -26,7 +26,6 @@ import org.elasticsearch.cluster.routing.allocation.DataTier;
 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<IndexMetadata>, 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 class IndexMetadata implements Diffable<IndexMetadata>, ToXContentFragmen
     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<String> INDEX_DOWNSAMPLE_SOURCE_UUID = Setting.simpleString(
         INDEX_DOWNSAMPLE_SOURCE_UUID_KEY,
         Property.IndexScope,
@@ -1277,6 +1272,14 @@ public class IndexMetadata implements Diffable<IndexMetadata>, ToXContentFragmen
         Property.InternalIndex
     );
 
+    public static final Setting<String> 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";

+ 1 - 0
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,

+ 7 - 0
server/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java

@@ -285,6 +285,13 @@ public class QueryRewriteContext {
         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.

+ 42 - 16
server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java

@@ -14,6 +14,7 @@ import org.elasticsearch.common.Rounding;
 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.time.ZoneId;
 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 @@ public class DateHistogramAggregationBuilder extends ValuesSourceAggregationBuil
     ) 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 @@ public class DateHistogramAggregationBuilder extends ValuesSourceAggregationBuil
             order,
             keyed,
             minDocCount,
+            downsampledResultsOffset,
             rounding,
             roundedBounds,
             roundedHardBounds,

+ 1 - 0
server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationSupplier.java

@@ -29,6 +29,7 @@ public interface DateHistogramAggregationSupplier {
         BucketOrder order,
         boolean keyed,
         long minDocCount,
+        boolean downsampledResultsOffset,
         @Nullable LongBounds extendedBounds,
         @Nullable LongBounds hardBounds,
         ValuesSourceConfig valuesSourceConfig,

+ 14 - 0
server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java

@@ -79,6 +79,7 @@ class DateHistogramAggregator extends BucketsAggregator implements SizedBucketAg
         BucketOrder order,
         boolean keyed,
         long minDocCount,
+        boolean downsampledResultsOffset,
         @Nullable LongBounds extendedBounds,
         @Nullable LongBounds hardBounds,
         ValuesSourceConfig valuesSourceConfig,
@@ -96,6 +97,7 @@ class DateHistogramAggregator extends BucketsAggregator implements SizedBucketAg
             order,
             keyed,
             minDocCount,
+            downsampledResultsOffset,
             extendedBounds,
             hardBounds,
             valuesSourceConfig,
@@ -115,6 +117,7 @@ class DateHistogramAggregator extends BucketsAggregator implements SizedBucketAg
             order,
             keyed,
             minDocCount,
+            downsampledResultsOffset,
             extendedBounds,
             hardBounds,
             valuesSourceConfig,
@@ -133,6 +136,7 @@ class DateHistogramAggregator extends BucketsAggregator implements SizedBucketAg
         BucketOrder order,
         boolean keyed,
         long minDocCount,
+        boolean downsampledResultsOffset,
         @Nullable LongBounds extendedBounds,
         @Nullable LongBounds hardBounds,
         ValuesSourceConfig valuesSourceConfig,
@@ -191,6 +195,7 @@ class DateHistogramAggregator extends BucketsAggregator implements SizedBucketAg
             minDocCount,
             extendedBounds,
             keyed,
+            downsampledResultsOffset,
             fixedRoundingPoints
         );
     }
@@ -227,6 +232,7 @@ class DateHistogramAggregator extends BucketsAggregator implements SizedBucketAg
     private final boolean keyed;
 
     private final long minDocCount;
+    private final boolean downsampledResultsOffset;
     private final LongBounds extendedBounds;
     private final LongBounds hardBounds;
 
@@ -240,6 +246,7 @@ class DateHistogramAggregator extends BucketsAggregator implements SizedBucketAg
         BucketOrder order,
         boolean keyed,
         long minDocCount,
+        boolean downsampledResultsOffset,
         @Nullable LongBounds extendedBounds,
         @Nullable LongBounds hardBounds,
         ValuesSourceConfig valuesSourceConfig,
@@ -255,6 +262,7 @@ class DateHistogramAggregator extends BucketsAggregator implements SizedBucketAg
         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 @@ class DateHistogramAggregator extends BucketsAggregator implements SizedBucketAg
                 emptyBucketInfo,
                 formatter,
                 keyed,
+                downsampledResultsOffset,
                 metadata()
             );
         });
@@ -347,6 +356,7 @@ class DateHistogramAggregator extends BucketsAggregator implements SizedBucketAg
             emptyBucketInfo,
             formatter,
             keyed,
+            downsampledResultsOffset,
             metadata()
         );
     }
@@ -392,6 +402,7 @@ class DateHistogramAggregator extends BucketsAggregator implements SizedBucketAg
         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 @@ class DateHistogramAggregator extends BucketsAggregator implements SizedBucketAg
             long minDocCount,
             LongBounds extendedBounds,
             boolean keyed,
+            boolean downsampledResultsOffset,
             long[] fixedRoundingPoints
         ) throws IOException {
             super(parent, subAggregators, delegate);
@@ -416,6 +428,7 @@ class DateHistogramAggregator extends BucketsAggregator implements SizedBucketAg
             this.minDocCount = minDocCount;
             this.extendedBounds = extendedBounds;
             this.keyed = keyed;
+            this.downsampledResultsOffset = downsampledResultsOffset;
             this.fixedRoundingPoints = fixedRoundingPoints;
         }
 
@@ -454,6 +467,7 @@ class DateHistogramAggregator extends BucketsAggregator implements SizedBucketAg
                 emptyBucketInfo,
                 format,
                 keyed,
+                downsampledResultsOffset,
                 range.getMetadata()
             );
         }

+ 7 - 0
server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorFactory.java

@@ -52,6 +52,7 @@ public final class DateHistogramAggregatorFactory extends ValuesSourceAggregator
                 order,
                 keyed,
                 minDocCount,
+                downsampledResultsOffset,
                 extendedBounds,
                 hardBounds,
                 valuesSourceConfig,
@@ -71,6 +72,7 @@ public final class DateHistogramAggregatorFactory extends ValuesSourceAggregator
                     order,
                     keyed,
                     minDocCount,
+                    downsampledResultsOffset,
                     extendedBounds,
                     hardBounds,
                     valuesSourceConfig,
@@ -88,6 +90,7 @@ public final class DateHistogramAggregatorFactory extends ValuesSourceAggregator
     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 final class DateHistogramAggregatorFactory extends ValuesSourceAggregator
         BucketOrder order,
         boolean keyed,
         long minDocCount,
+        boolean downsampledResultsOffset,
         Rounding rounding,
         LongBounds extendedBounds,
         LongBounds hardBounds,
@@ -111,6 +115,7 @@ public final class DateHistogramAggregatorFactory extends ValuesSourceAggregator
         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 @@ public final class DateHistogramAggregatorFactory extends ValuesSourceAggregator
             order,
             keyed,
             minDocCount,
+            downsampledResultsOffset,
             extendedBounds,
             hardBounds,
             config,
@@ -159,6 +165,7 @@ public final class DateHistogramAggregatorFactory extends ValuesSourceAggregator
             order,
             keyed,
             minDocCount,
+            downsampledResultsOffset,
             extendedBounds,
             hardBounds,
             config,

+ 5 - 0
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 @@ class DateRangeHistogramAggregator extends BucketsAggregator {
                     emptyBucketInfo,
                     formatter,
                     keyed,
+                    downsampledResultsOffset,
                     metadata()
                 );
             }
@@ -217,6 +221,7 @@ class DateRangeHistogramAggregator extends BucketsAggregator {
             emptyBucketInfo,
             formatter,
             keyed,
+            downsampledResultsOffset,
             metadata()
         );
     }

+ 44 - 2
server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java

@@ -9,6 +9,7 @@ package org.elasticsearch.search.aggregations.bucket.histogram;
 
 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 final class InternalDateHistogram extends InternalMultiBucketAggregation<
     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 final class InternalDateHistogram extends InternalMultiBucketAggregation<
         EmptyBucketInfo emptyBucketInfo,
         DocValueFormat formatter,
         boolean keyed,
+        boolean downsampledResultsOffset,
         Map<String, Object> metadata
     ) {
         super(name, metadata);
@@ -226,6 +229,7 @@ public final class InternalDateHistogram extends InternalMultiBucketAggregation<
         this.emptyBucketInfo = emptyBucketInfo;
         this.format = formatter;
         this.keyed = keyed;
+        this.downsampledResultsOffset = downsampledResultsOffset;
     }
 
     /**
@@ -243,6 +247,11 @@ public final class InternalDateHistogram extends InternalMultiBucketAggregation<
         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 @@ public final class InternalDateHistogram extends InternalMultiBucketAggregation<
         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 @@ public final class InternalDateHistogram extends InternalMultiBucketAggregation<
 
     @Override
     public InternalDateHistogram create(List<Bucket> 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 final class InternalDateHistogram extends InternalMultiBucketAggregation<
             emptyBucketInfo,
             format,
             keyed,
+            downsampledResultsOffset,
             getMetadata()
         );
     }
@@ -523,6 +547,7 @@ public final class InternalDateHistogram extends InternalMultiBucketAggregation<
             emptyBucketInfo,
             format,
             keyed,
+            downsampledResultsOffset,
             getMetadata()
         );
     }
@@ -542,6 +567,12 @@ public final class InternalDateHistogram extends InternalMultiBucketAggregation<
         } 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 final class InternalDateHistogram extends InternalMultiBucketAggregation<
             buckets2.add((Bucket) b);
         }
         buckets2 = Collections.unmodifiableList(buckets2);
-        return new InternalDateHistogram(name, buckets2, order, minDocCount, offset, emptyBucketInfo, format, keyed, getMetadata());
+        return new InternalDateHistogram(
+            name,
+            buckets2,
+            order,
+            minDocCount,
+            offset,
+            emptyBucketInfo,
+            format,
+            keyed,
+            downsampledResultsOffset,
+            getMetadata()
+        );
     }
 
     @Override

+ 14 - 0
server/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java

@@ -29,6 +29,7 @@ import org.elasticsearch.index.cache.bitset.BitsetFilterCache;
 import org.elasticsearch.index.fielddata.IndexFieldData;
 import org.elasticsearch.index.mapper.DocCountFieldMapper;
 import org.elasticsearch.index.mapper.MappedFieldType;
+import org.elasticsearch.index.mapper.MappingLookup;
 import org.elasticsearch.index.mapper.NestedLookup;
 import org.elasticsearch.index.query.QueryBuilder;
 import org.elasticsearch.index.query.Rewriteable;
@@ -307,6 +308,14 @@ public abstract class AggregationContext implements Releasable {
 
     public abstract Set<String> 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 abstract class AggregationContext implements Releasable {
             return context.sourcePath(fullName);
         }
 
+        @Override
+        public MappingLookup getMappingLookup() {
+            return context.getMappingLookup();
+        }
+
         @Override
         public void close() {
             /*

+ 3 - 2
server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogramTests.java

@@ -109,7 +109,7 @@ public class InternalDateHistogramTests extends InternalMultiBucketAggregationTe
             }
         }
         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 @@ public class InternalDateHistogramTests extends InternalMultiBucketAggregationTe
             }
             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 class InternalDateHistogramTests extends InternalMultiBucketAggregationTe
             ),
             DocValueFormat.RAW,
             false,
+            false,
             null
         );
         expectReduceUsesTooManyBuckets(largeHisto, 100000);

+ 490 - 26
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" }

+ 1 - 0
x-pack/plugin/downsample/src/internalClusterTest/java/org/elasticsearch/xpack/downsample/DataStreamLifecycleDownsampleDisruptionIT.java

@@ -139,6 +139,7 @@ public class DataStreamLifecycleDownsampleDisruptionIT extends ESIntegTestCase {
                 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);
             }

+ 3 - 0
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<String> FORBIDDEN_SETTINGS = Set.of(
         IndexSettings.DEFAULT_PIPELINE.getKey(),
@@ -284,6 +285,7 @@ public class TransportDownsampleAction extends AcknowledgedTransportMasterNodeAc
 
             // Validate downsampling interval
             validateDownsamplingInterval(mapperService, request.getDownsampleConfig());
+            downsamplingInterval = request.getDownsampleConfig().getInterval().toString();
 
             final List<String> dimensionFields = new ArrayList<>();
             final List<String> metricFields = new ArrayList<>();
@@ -888,6 +890,7 @@ public class TransportDownsampleAction extends AcknowledgedTransportMasterNodeAc
                             Settings.builder()
                                 .put(downsampleIndex.getSettings())
                                 .put(IndexMetadata.INDEX_DOWNSAMPLE_STATUS.getKey(), DownsampleTaskStatus.SUCCESS)
+                                .put(IndexMetadata.INDEX_DOWNSAMPLE_INTERVAL.getKey(), downsamplingInterval)
                                 .build(),
                             downsampleIndexName
                         );

+ 5 - 0
x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/DownsampleActionIT.java

@@ -213,6 +213,7 @@ public class DownsampleActionIT extends ESRestTestCase {
             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 class DownsampleActionIT extends ESRestTestCase {
             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 class DownsampleActionIT extends ESRestTestCase {
             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()));
         });
     }
 
@@ -478,6 +481,7 @@ public class DownsampleActionIT extends ESRestTestCase {
                 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)) {
@@ -559,6 +563,7 @@ public class DownsampleActionIT extends ESRestTestCase {
             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);