diff --git a/server/src/main/java/org/elasticsearch/ElasticsearchException.java b/server/src/main/java/org/elasticsearch/ElasticsearchException.java index 77c2abac8304f..0c4a70cb4327f 100644 --- a/server/src/main/java/org/elasticsearch/ElasticsearchException.java +++ b/server/src/main/java/org/elasticsearch/ElasticsearchException.java @@ -23,6 +23,7 @@ import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.SearchException; import org.elasticsearch.search.aggregations.MultiBucketConsumerService; +import org.elasticsearch.search.aggregations.UnsupportedAggregationOnRollupIndex; import org.elasticsearch.transport.TcpTransport; import org.elasticsearch.xcontent.ParseField; import org.elasticsearch.xcontent.ToXContentFragment; @@ -1578,6 +1579,12 @@ private enum ElasticsearchExceptionHandle { HealthNodeNotDiscoveredException::new, 166, Version.V_8_5_0 + ), + UNSUPPORTED_AGGREGATION_ON_DOWNSAMPLED_FIELD_EXCEPTION( + UnsupportedAggregationOnRollupIndex.class, + UnsupportedAggregationOnRollupIndex::new, + 167, + Version.V_8_5_0 ); final Class exceptionClass; 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 412f4cc3ae475..2d0975faec9b3 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetadata.java @@ -22,6 +22,7 @@ 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.collect.MapBuilder; import org.elasticsearch.common.compress.CompressedXContent; @@ -127,6 +128,16 @@ public class IndexMetadata implements Diffable, ToXContentFragmen EnumSet.of(ClusterBlockLevel.WRITE) ); + // TODO: refactor this method after adding more rollup metadata + public boolean isRollupIndex() { + final String sourceIndex = settings.get(IndexMetadata.INDEX_ROLLUP_SOURCE_NAME_KEY); + final String indexRollupStatus = settings.get(IndexMetadata.INDEX_ROLLUP_STATUS_KEY); + final boolean rollupSuccess = IndexMetadata.RollupTaskStatus.SUCCESS.name() + .toLowerCase(Locale.ROOT) + .equals(indexRollupStatus != null ? indexRollupStatus.toLowerCase(Locale.ROOT) : IndexMetadata.RollupTaskStatus.UNKNOWN); + return Strings.isNullOrEmpty(sourceIndex) == false && rollupSuccess; + } + public enum State implements Writeable { OPEN((byte) 0), CLOSE((byte) 1); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/UnsupportedAggregationOnRollupIndex.java b/server/src/main/java/org/elasticsearch/search/aggregations/UnsupportedAggregationOnRollupIndex.java new file mode 100644 index 0000000000000..574cd64448483 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/UnsupportedAggregationOnRollupIndex.java @@ -0,0 +1,37 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.aggregations; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.rest.RestStatus; + +import java.io.IOException; + +// NOTE: the name of this class is part of a contract with Kibana which uses it to detect specific +// errors while running aggregations on rollup indices. +/** + * Thrown when executing an aggregation on a time series index field whose type is not supported. + * Downsampling uses specific types while aggregating some fields (like 'aggregate_metric_double'). + * Such field types do not support some aggregations. + */ +public class UnsupportedAggregationOnRollupIndex extends AggregationExecutionException { + + public UnsupportedAggregationOnRollupIndex(final String msg) { + super(msg); + } + + public UnsupportedAggregationOnRollupIndex(final StreamInput in) throws IOException { + super(in); + } + + @Override + public RestStatus status() { + return RestStatus.BAD_REQUEST; + } +} 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 c76ee3d1bfbaf..c83f6a0fb4007 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 @@ -417,9 +417,28 @@ protected ValuesSourceAggregatorFactory innerBuild( AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder ) throws IOException { - DateHistogramAggregationSupplier aggregatorSupplier = context.getValuesSourceRegistry().getAggregator(REGISTRY_KEY, config); + final DateIntervalWrapper.IntervalTypeEnum dateHistogramIntervalType = dateHistogramInterval.getIntervalType(); + + if (context.getIndexSettings().getIndexMetadata().isRollupIndex() + && DateIntervalWrapper.IntervalTypeEnum.CALENDAR.equals(dateHistogramIntervalType)) { + throw new IllegalArgumentException( + config.getDescription() + + " is not supported for aggregation [" + + getName() + + "] with interval type [" + + dateHistogramIntervalType.getPreferredName() + + "]" + ); + } final ZoneId tz = timeZone(); + if (context.getIndexSettings().getIndexMetadata().isRollupIndex() && tz != null && ZoneId.of("UTC").equals(tz) == false) { + throw new IllegalArgumentException( + config.getDescription() + " is not supported for aggregation [" + getName() + "] with timezone [" + tz + "]" + ); + } + + DateHistogramAggregationSupplier aggregatorSupplier = context.getValuesSourceRegistry().getAggregator(REGISTRY_KEY, config); final Rounding rounding = dateHistogramInterval.createRounding(tz, offset); LongBounds roundedBounds = null; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceRegistry.java b/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceRegistry.java index 3a43f147fefa3..827ed32aff09a 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceRegistry.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceRegistry.java @@ -169,13 +169,23 @@ public T getAggregator(RegistryKey registryKey, ValuesSourceConfig values @SuppressWarnings("unchecked") T supplier = (T) aggregatorRegistry.get(registryKey).get(valuesSourceConfig.valueSourceType()); if (supplier == null) { - throw new IllegalArgumentException( - valuesSourceConfig.getDescription() + " is not supported for aggregation [" + registryKey.getName() + "]" - ); + final RuntimeException unmappedException = valuesSourceConfig.valueSourceType() + .getUnregisteredException( + valuesSourceConfig.getDescription() + " is not supported for aggregation [" + registryKey.getName() + "]" + ); + assert unmappedException != null + : "Value source type [" + + valuesSourceConfig.valueSourceType() + + "] did not return a valid exception for aggregation [" + + registryKey.getName() + + "]"; + throw unmappedException; } return supplier; } - throw new AggregationExecutionException("Unregistered Aggregation [" + registryKey.getName() + "]"); + throw new AggregationExecutionException( + "Unregistered Aggregation [" + (registryKey != null ? registryKey.getName() : "unknown aggregation") + "]" + ); } public AggregationUsageService getUsageService() { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceType.java b/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceType.java index ed7e786b10b34..b13e85457a2be 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceType.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceType.java @@ -92,4 +92,15 @@ default DocValueFormat getFormatter(String format, ZoneId tz) { * @return the name of the Values Source Type */ String typeName(); + + /** + * Returns the exception to throw in case the registry (type, aggregator) entry + * is not registered. + * + * @param message the message for the exception + * @return the exception to throw + */ + default RuntimeException getUnregisteredException(String message) { + return new IllegalArgumentException(message); + } } diff --git a/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java b/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java index d59b18bf3b1f2..1cfa86e446002 100644 --- a/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java +++ b/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java @@ -73,6 +73,7 @@ import org.elasticsearch.search.SearchParseException; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.aggregations.MultiBucketConsumerService; +import org.elasticsearch.search.aggregations.UnsupportedAggregationOnRollupIndex; import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotException; @@ -831,6 +832,7 @@ public void testIds() { ids.put(164, VersionConflictException.class); ids.put(165, SnapshotNameAlreadyInUseException.class); ids.put(166, HealthNodeNotDiscoveredException.class); + ids.put(167, UnsupportedAggregationOnRollupIndex.class); Map, Integer> reverse = new HashMap<>(); for (Map.Entry> entry : ids.entrySet()) { diff --git a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/support/AggregateMetricsValuesSourceType.java b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/support/AggregateMetricsValuesSourceType.java index dd3f6d694664b..daf8990c5fb8f 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/support/AggregateMetricsValuesSourceType.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/support/AggregateMetricsValuesSourceType.java @@ -10,6 +10,7 @@ import org.elasticsearch.script.AggregationScript; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.UnsupportedAggregationOnRollupIndex; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.FieldContext; import org.elasticsearch.search.aggregations.support.ValueType; @@ -22,6 +23,12 @@ public enum AggregateMetricsValuesSourceType implements ValuesSourceType { AGGREGATE_METRIC() { + + @Override + public RuntimeException getUnregisteredException(String message) { + return new UnsupportedAggregationOnRollupIndex(message); + } + @Override public ValuesSource getEmpty() { throw new IllegalArgumentException("Can't deal with unmapped AggregateMetricsValuesSource type " + this.value()); diff --git a/x-pack/plugin/rollup/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/rollup/20_unsupported_aggs.yml b/x-pack/plugin/rollup/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/rollup/20_unsupported_aggs.yml new file mode 100644 index 0000000000000..4db6f6a925712 --- /dev/null +++ b/x-pack/plugin/rollup/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/rollup/20_unsupported_aggs.yml @@ -0,0 +1,174 @@ +setup: + - skip: + version: " - 8.4.99" + reason: "rollup: unsupported aggregations errors added in 8.5.0" + + - do: + indices.create: + index: test + body: + settings: + number_of_shards: 1 + number_of_replicas: 0 + index: + mode: time_series + routing_path: [ uid ] + time_series: + start_time: 2021-04-28T00:00:00Z + end_time: 2021-04-29T00:00:00Z + mappings: + properties: + "@timestamp": + type: date + uid: + type: keyword + time_series_dimension: true + total_memory_used: + type: aggregate_metric_double + metrics: [ min, max ] + default_metric: min + - do: + bulk: + refresh: true + index: test + body: + - '{ "index": {} }' + - '{ "@timestamp": "2021-04-28T18:50:00Z", "uid": "001", "total_memory_used": { "min": 99198, "max": 106780 } }' + - '{ "index": {} }' + - '{ "@timestamp": "2021-04-28T18:55:00Z", "uid": "002", "total_memory_used": { "min": 102334, "max": 110450 } }' + - '{ "index": {} }' + - '{ "@timestamp": "2021-04-28T18:50:00Z", "uid": "003", "total_memory_used": { "min": 98012, "max": 109009 } }' + - '{ "index": {} }' + - '{ "@timestamp": "2021-04-28T18:55:00Z", "uid": "004", "total_memory_used": { "min": 101990, "max": 120770 } }' + +--- +"Histogram aggregation on aggregate_metric_double field": + - skip: + version: " - 8.4.99" + reason: "rollup: unsupported aggregations errors added in 8.5.0" + + - do: + catch: bad_request + search: + index: test + body: + size: 0 + aggs: + total_memory_used_histogram: + histogram: + field: total_memory_used + interval: 100000 + + + - match: { status: 400 } + # NOTE: the type of error is part of a contract with Kibana which uses it to detect specific + # errors while running aggregations on rollup indices. + - match: { error.root_cause.0.type: unsupported_aggregation_on_rollup_index } + - match: { error.root_cause.0.reason: "Field [total_memory_used] of type [aggregate_metric_double] is not supported for aggregation [histogram]" } + +--- +"Range aggregation on aggregate_metric_double field": + - skip: + version: " - 8.4.99" + reason: "rollup: unsupported aggregations errors added in 8.5.0" + + - do: + catch: bad_request + search: + index: test + body: + size: 0 + aggs: + tx_range: + range: + field: total_memory_used + ranges: + - + from: 0.0 + to: 200.0 + - + from: 201.0 + to: 400.0 + - + from: 401.0 + + - match: { status: 400 } + # NOTE: the type of error is part of a contract with Kibana which uses it to detect specific + # errors while running aggregations on rollup indices. + - match: { error.root_cause.0.type: unsupported_aggregation_on_rollup_index } + - match: { error.root_cause.0.reason: "Field [total_memory_used] of type [aggregate_metric_double] is not supported for aggregation [range]" } + +--- +"Cardinality aggregation on aggregate_metric_double field": + - skip: + version: " - 8.4.99" + reason: "rollup: unsupported aggregations errors added in 8.5.0" + + - do: + catch: bad_request + search: + index: test + body: + size: 0 + aggs: + tx_cardinality: + cardinality: + field: total_memory_used + + - match: { status: 400 } + # NOTE: the type of error is part of a contract with Kibana which uses it to detect specific + # errors while running aggregations on rollup indices. + - match: { error.root_cause.0.type: unsupported_aggregation_on_rollup_index } + - match: { error.root_cause.0.reason: "Field [total_memory_used] of type [aggregate_metric_double] is not supported for aggregation [cardinality]" } + +--- +"Percentiles aggregation on aggregate_metric_double field": + - skip: + version: " - 8.4.99" + reason: "rollup: unsupported aggregations errors added in 8.5.0" + + - do: + catch: bad_request + search: + index: test + body: + size: 0 + aggs: + tx_percentile: + percentiles: + field: total_memory_used + percents: [90, 95, 99] + + - match: { status: 400 } + # NOTE: the type of error is part of a contract with Kibana which uses it to detect specific + # errors while running aggregations on rollup indices. + - match: { error.root_cause.0.type: unsupported_aggregation_on_rollup_index } + - match: { error.root_cause.0.reason: "Field [total_memory_used] of type [aggregate_metric_double] is not supported for aggregation [percentiles]" } + +--- +"Top-metrics aggregation on aggregate_metric_double field": + - skip: + version: " - 8.4.99" + reason: "rollup: unsupported aggregations errors added in 8.5.0" + + - do: + catch: bad_request + search: + index: test + body: + size: 0 + aggs: + top_total_memory_used: + top_metrics: + metrics: + field: total_memory_used + sort: + uid: desc + size: 2 + + + - match: { status: 400 } + # NOTE: the type of error is part of a contract with Kibana which uses it to detect specific + # errors while running aggregations on rollup indices. + - match: { error.root_cause.0.type: unsupported_aggregation_on_rollup_index } + - match: { error.root_cause.0.reason: "Field [total_memory_used] of type [aggregate_metric_double] is not supported for aggregation [top_metrics]" } diff --git a/x-pack/plugin/rollup/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/rollup/30_date_histogram.yml b/x-pack/plugin/rollup/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/rollup/30_date_histogram.yml new file mode 100644 index 0000000000000..87b882db3d020 --- /dev/null +++ b/x-pack/plugin/rollup/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/rollup/30_date_histogram.yml @@ -0,0 +1,250 @@ +setup: + - skip: + version: " - 8.4.99" + reason: "rollup: unsupported aggregations errors added in 8.5.0" + + - do: + indices.create: + index: test + body: + settings: + number_of_shards: 1 + number_of_replicas: 0 + index: + mode: time_series + routing_path: [ uid ] + time_series: + start_time: 2021-04-28T00:00:00Z + end_time: 2021-04-29T00:00:00Z + mappings: + properties: + "@timestamp": + type: date + uid: + type: keyword + time_series_dimension: true + total_memory_used: + type: integer + time_series_metric: gauge + - do: + bulk: + refresh: true + index: test + body: + - '{ "index": {} }' + - '{ "@timestamp": "2021-04-28T18:50:00Z", "uid": "001", "total_memory_used": 106780 }' + - '{ "index": {} }' + - '{ "@timestamp": "2021-04-28T18:55:00Z", "uid": "002", "total_memory_used": 110450 }' + - '{ "index": {} }' + - '{ "@timestamp": "2021-04-28T18:50:00Z", "uid": "003", "total_memory_used": 109009 }' + - '{ "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 indexs": + - skip: + version: " - 8.4.99" + reason: "rollup: unsupported aggregations errors added in 8.5.0" + features: close_to + + - do: + rollup.rollup: + index: test + rollup_index: rollup-test + body: > + { + "fixed_interval": "1h" + } + + - is_true: acknowledged + + - do: + search: + index: rollup-test + body: + sort: [ "_tsid", "@timestamp" ] + + - length: { hits.hits: 4 } + + - match: { hits.hits.0._index: "rollup-test" } + - match: { hits.hits.0._source._doc_count: 1 } + - match: { hits.hits.0._source.@timestamp: "2021-04-28T18:00:00.000Z" } + - match: { hits.hits.0._source.uid: "001" } + - close_to: { hits.hits.0._source.total_memory_used.min: { value: 106780.0, error: 0.00001 } } + - close_to: { hits.hits.0._source.total_memory_used.max: { value: 106780.0, error: 0.00001 } } + - close_to: { hits.hits.0._source.total_memory_used.sum: { value: 106780.0, error: 0.00001 } } + - match: { hits.hits.0._source.total_memory_used.value_count: 1 } + + - match: { hits.hits.1._index: "rollup-test" } + - match: { hits.hits.1._source._doc_count: 1 } + - match: { hits.hits.1._source.@timestamp: "2021-04-28T18:00:00.000Z" } + - match: { hits.hits.1._source.uid: "002" } + - close_to: { hits.hits.1._source.total_memory_used.min: { value: 110450.0, error: 0.00001 } } + - close_to: { hits.hits.1._source.total_memory_used.max: { value: 110450.0, error: 0.00001 } } + - close_to: { hits.hits.1._source.total_memory_used.sum: { value: 110450.0, error: 0.00001 } } + - match: { hits.hits.1._source.total_memory_used.value_count: 1 } + + - match: { hits.hits.2._index: "rollup-test" } + - match: { hits.hits.2._source._doc_count: 1 } + - match: { hits.hits.2._source.@timestamp: "2021-04-28T18:00:00.000Z" } + - match: { hits.hits.2._source.uid: "003" } + - close_to: { hits.hits.2._source.total_memory_used.min: { value: 109009.0, error: 0.00001 } } + - close_to: { hits.hits.2._source.total_memory_used.max: { value: 109009.0, error: 0.00001 } } + - close_to: { hits.hits.2._source.total_memory_used.sum: { value: 109009.0, error: 0.00001 } } + - match: { hits.hits.2._source.total_memory_used.value_count: 1 } + + - match: { hits.hits.3._index: "rollup-test" } + - match: { hits.hits.3._source._doc_count: 1 } + - match: { hits.hits.3._source.@timestamp: "2021-04-28T18:00:00.000Z" } + - match: { hits.hits.3._source.uid: "004" } + - close_to: { hits.hits.3._source.total_memory_used.min: { value: 120770.0, error: 0.00001 } } + - close_to: { hits.hits.3._source.total_memory_used.max: { value: 120770.0, error: 0.00001 } } + - close_to: { hits.hits.3._source.total_memory_used.sum: { value: 120770.0, error: 0.00001 } } + - match: { hits.hits.3._source.total_memory_used.value_count: 1 } + + # date histogram aggregation with calendar interval on rollup index not supported + - do: + catch: bad_request + search: + index: rollup-test + body: + size: 0 + aggs: + date_histogram: + date_histogram: + field: "@timestamp" + calendar_interval: hour + + - 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 interval type [calendar_interval]" } + + # date histogram aggregation with calendar interval on time series index supported + - do: + search: + index: test + body: + size: 0 + aggs: + date_histogram: + date_histogram: + field: "@timestamp" + calendar_interval: hour + + - 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-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: rollup-test + 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: + index: test + body: + size: 0 + aggs: + date_histogram: + date_histogram: + field: "@timestamp" + fixed_interval: 1h + time_zone: "America/New_York" + + - 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 } + + # date histogram aggregation with fixed interval and utc timezone on rollup index supported + - do: + search: + index: rollup-test + body: + size: 0 + aggs: + date_histogram: + date_histogram: + field: "@timestamp" + fixed_interval: 1h + min_doc_count: 1 + + - match: { hits.total.value: 4 } + - match: { hits.total.relation: "eq" } + - length: { aggregations.date_histogram.buckets: 1 } + - match: { aggregations.date_histogram.buckets.0.key_as_string: "2021-04-28T18:00:00.000Z" } + - match: { aggregations.date_histogram.buckets.0.key: 1619632800000 } + - match: { aggregations.date_histogram.buckets.0.doc_count: 4 } + + # date histogram aggregation with fixed interval and utc timezone on time series index supported + - do: + search: + index: test + body: + size: 0 + aggs: + date_histogram: + date_histogram: + field: "@timestamp" + fixed_interval: 1h + min_doc_count: 1 + + - match: { hits.total.value: 4 } + - match: { hits.total.relation: "eq" } + - length: { aggregations.date_histogram.buckets: 1 } + - match: { aggregations.date_histogram.buckets.0.key_as_string: "2021-04-28T18:00:00.000Z" } + - match: { aggregations.date_histogram.buckets.0.key: 1619632800000 } + - match: { aggregations.date_histogram.buckets.0.doc_count: 4 } + + # date histogram aggregation on both indices resulting in partial results and shard failures + - do: + search: + index: test,rollup-test + body: + size: 0 + aggs: + date_histogram: + date_histogram: + field: "@timestamp" + calendar_interval: hour + min_doc_count: 1 + + # NOTE: the rollup index is created with the same number of shards of the source index. + # As a result we have 1 shard for the 'test' index and 1 shard for the 'rollup-test' index. + - match: { _shards.total: 2 } + - match: { _shards.successful: 1 } + - match: { _shards.failed: 1 } + - match: { hits.total.value: 4 } + - match: { hits.total.relation: "eq" } + - length: { aggregations.date_histogram.buckets: 1 } + - match: { aggregations.date_histogram.buckets.0.key_as_string: "2021-04-28T18:00:00.000Z" } + - match: { aggregations.date_histogram.buckets.0.key: 1619632800000 } + - match: { aggregations.date_histogram.buckets.0.doc_count: 4 } + - length: { _shards.failures: 1 } + - match: { _shards.failures.0.index: "rollup-test" } + - 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]" }