1
0
Эх сурвалжийг харах

Revert "Search coordinator uses event.ingested in cluster state to do rewrites (#110352)" (#110881)

This reverts commit d45d164afe0a210ad2757dad7df9d6c5ff35550d.
Michael Peterson 1 жил өмнө
parent
commit
5c98098b0d

+ 0 - 5
docs/changelog/110352.yaml

@@ -1,5 +0,0 @@
-pr: 110352
-summary: Search coordinator uses `event.ingested` in cluster state to do rewrites
-area: Search
-type: enhancement
-issues: []

+ 2 - 2
modules/data-streams/src/test/java/org/elasticsearch/datastreams/TimestampFieldMapperServiceTests.java

@@ -61,7 +61,7 @@ public class TimestampFieldMapperServiceTests extends ESSingleNodeTestCase {
         DocWriteResponse indexResponse = indexDoc();
 
         var indicesService = getInstanceFromNode(IndicesService.class);
-        var result = indicesService.getTimestampFieldTypeInfo(indexResponse.getShardId().getIndex());
+        var result = indicesService.getTimestampFieldType(indexResponse.getShardId().getIndex());
         assertThat(result, notNullValue());
     }
 
@@ -70,7 +70,7 @@ public class TimestampFieldMapperServiceTests extends ESSingleNodeTestCase {
         DocWriteResponse indexResponse = indexDoc();
 
         var indicesService = getInstanceFromNode(IndicesService.class);
-        var result = indicesService.getTimestampFieldTypeInfo(indexResponse.getShardId().getIndex());
+        var result = indicesService.getTimestampFieldType(indexResponse.getShardId().getIndex());
         assertThat(result, nullValue());
     }
 

+ 18 - 95
server/src/main/java/org/elasticsearch/index/query/CoordinatorRewriteContext.java

@@ -9,14 +9,11 @@
 package org.elasticsearch.index.query;
 
 import org.elasticsearch.client.internal.Client;
-import org.elasticsearch.cluster.metadata.DataStream;
-import org.elasticsearch.cluster.metadata.IndexMetadata;
-import org.elasticsearch.common.Strings;
 import org.elasticsearch.core.Nullable;
+import org.elasticsearch.index.mapper.DateFieldMapper;
 import org.elasticsearch.index.mapper.MappedFieldType;
 import org.elasticsearch.index.mapper.MappingLookup;
 import org.elasticsearch.index.shard.IndexLongFieldRange;
-import org.elasticsearch.indices.DateFieldRangeInfo;
 import org.elasticsearch.xcontent.XContentParserConfiguration;
 
 import java.util.Collections;
@@ -26,24 +23,19 @@ import java.util.function.LongSupplier;
  * Context object used to rewrite {@link QueryBuilder} instances into simplified version in the coordinator.
  * Instances of this object rely on information stored in the {@code IndexMetadata} for certain indices.
  * Right now this context object is able to rewrite range queries that include a known timestamp field
- * (i.e. the timestamp field for DataStreams or the 'event.ingested' field in ECS) into a MatchNoneQueryBuilder
- * and skip the shards that don't hold queried data. See IndexMetadata for more details.
+ * (i.e. the timestamp field for DataStreams) into a MatchNoneQueryBuilder and skip the shards that
+ * don't hold queried data. See IndexMetadata#getTimestampRange() for more details
  */
 public class CoordinatorRewriteContext extends QueryRewriteContext {
-    private final DateFieldRangeInfo dateFieldRangeInfo;
+    private final IndexLongFieldRange indexLongFieldRange;
+    private final DateFieldMapper.DateFieldType timestampFieldType;
 
-    /**
-     * Context for coordinator search rewrites based on time ranges for the @timestamp field and/or 'event.ingested' field
-     * @param parserConfig
-     * @param client
-     * @param nowInMillis
-     * @param dateFieldRangeInfo range and field type info for @timestamp and 'event.ingested'
-     */
     public CoordinatorRewriteContext(
         XContentParserConfiguration parserConfig,
         Client client,
         LongSupplier nowInMillis,
-        DateFieldRangeInfo dateFieldRangeInfo
+        IndexLongFieldRange indexLongFieldRange,
+        DateFieldMapper.DateFieldType timestampFieldType
     ) {
         super(
             parserConfig,
@@ -61,98 +53,29 @@ public class CoordinatorRewriteContext extends QueryRewriteContext {
             null,
             null
         );
-        this.dateFieldRangeInfo = dateFieldRangeInfo;
+        this.indexLongFieldRange = indexLongFieldRange;
+        this.timestampFieldType = timestampFieldType;
     }
 
-    /**
-     * Get min timestamp for either '@timestamp' or 'event.ingested' fields. Any other field
-     * passed in will cause an {@link IllegalArgumentException} to be thrown, as these are the only
-     * two fields supported for coordinator rewrites (based on time range).
-     * @param fieldName Must be DataStream.TIMESTAMP_FIELD_NAME or IndexMetadata.EVENT_INGESTED_FIELD_NAME
-     * @return min timestamp for the field from IndexMetadata in cluster state.
-     */
-    long getMinTimestamp(String fieldName) {
-        if (DataStream.TIMESTAMP_FIELD_NAME.equals(fieldName)) {
-            return dateFieldRangeInfo.getTimestampRange().getMin();
-        } else if (IndexMetadata.EVENT_INGESTED_FIELD_NAME.equals(fieldName)) {
-            return dateFieldRangeInfo.getEventIngestedRange().getMin();
-        } else {
-            throw new IllegalArgumentException(
-                Strings.format(
-                    "Only [%s] or [%s] fields are supported for min timestamp coordinator rewrites, but got: [%s]",
-                    DataStream.TIMESTAMP_FIELD_NAME,
-                    IndexMetadata.EVENT_INGESTED_FIELD_NAME,
-                    fieldName
-                )
-            );
-        }
+    long getMinTimestamp() {
+        return indexLongFieldRange.getMin();
     }
 
-    /**
-     * Get max timestamp for either '@timestamp' or 'event.ingested' fields. Any other field
-     * passed in will cause an {@link IllegalArgumentException} to be thrown, as these are the only
-     * two fields supported for coordinator rewrites (based on time range).
-     * @param fieldName Must be DataStream.TIMESTAMP_FIELD_NAME or IndexMetadata.EVENT_INGESTED_FIELD_NAME
-     * @return max timestamp for the field from IndexMetadata in cluster state.
-     */
-    long getMaxTimestamp(String fieldName) {
-        if (DataStream.TIMESTAMP_FIELD_NAME.equals(fieldName)) {
-            return dateFieldRangeInfo.getTimestampRange().getMax();
-        } else if (IndexMetadata.EVENT_INGESTED_FIELD_NAME.equals(fieldName)) {
-            return dateFieldRangeInfo.getEventIngestedRange().getMax();
-        } else {
-            throw new IllegalArgumentException(
-                Strings.format(
-                    "Only [%s] or [%s] fields are supported for max timestamp coordinator rewrites, but got: [%s]",
-                    DataStream.TIMESTAMP_FIELD_NAME,
-                    IndexMetadata.EVENT_INGESTED_FIELD_NAME,
-                    fieldName
-                )
-            );
-        }
+    long getMaxTimestamp() {
+        return indexLongFieldRange.getMax();
     }
 
-    /**
-     * Determine whether either '@timestamp' or 'event.ingested' fields has useful timestamp ranges
-     * stored in cluster state for this context.
-     * Any other fieldname will cause an {@link IllegalArgumentException} to be thrown, as these are the only
-     * two fields supported for coordinator rewrites (based on time range).
-     * @param fieldName Must be DataStream.TIMESTAMP_FIELD_NAME or IndexMetadata.EVENT_INGESTED_FIELD_NAME
-     * @return min timestamp for the field from IndexMetadata in cluster state.
-     */
-    boolean hasTimestampData(String fieldName) {
-        if (DataStream.TIMESTAMP_FIELD_NAME.equals(fieldName)) {
-            return dateFieldRangeInfo.getTimestampRange().isComplete()
-                && dateFieldRangeInfo.getTimestampRange() != IndexLongFieldRange.EMPTY;
-        } else if (IndexMetadata.EVENT_INGESTED_FIELD_NAME.equals(fieldName)) {
-            return dateFieldRangeInfo.getEventIngestedRange().isComplete()
-                && dateFieldRangeInfo.getEventIngestedRange() != IndexLongFieldRange.EMPTY;
-        } else {
-            throw new IllegalArgumentException(
-                Strings.format(
-                    "Only [%s] or [%s] fields are supported for min/max timestamp coordinator rewrites, but got: [%s]",
-                    DataStream.TIMESTAMP_FIELD_NAME,
-                    IndexMetadata.EVENT_INGESTED_FIELD_NAME,
-                    fieldName
-                )
-            );
-        }
+    boolean hasTimestampData() {
+        return indexLongFieldRange.isComplete() && indexLongFieldRange != IndexLongFieldRange.EMPTY;
     }
 
-    /**
-     * @param fieldName Get MappedFieldType for either '@timestamp' or 'event.ingested' fields.
-     * @return min timestamp for the field from IndexMetadata in cluster state or null if fieldName was not
-     *         DataStream.TIMESTAMP_FIELD_NAME or IndexMetadata.EVENT_INGESTED_FIELD_NAME.
-     */
     @Nullable
     public MappedFieldType getFieldType(String fieldName) {
-        if (DataStream.TIMESTAMP_FIELD_NAME.equals(fieldName)) {
-            return dateFieldRangeInfo.getTimestampFieldType();
-        } else if (IndexMetadata.EVENT_INGESTED_FIELD_NAME.equals(fieldName)) {
-            return dateFieldRangeInfo.getEventIngestedFieldType();
-        } else {
+        if (fieldName.equals(timestampFieldType.name()) == false) {
             return null;
         }
+
+        return timestampFieldType;
     }
 
     @Override

+ 7 - 23
server/src/main/java/org/elasticsearch/index/query/CoordinatorRewriteContextProvider.java

@@ -14,7 +14,6 @@ import org.elasticsearch.core.Nullable;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.index.mapper.DateFieldMapper;
 import org.elasticsearch.index.shard.IndexLongFieldRange;
-import org.elasticsearch.indices.DateFieldRangeInfo;
 import org.elasticsearch.xcontent.XContentParserConfiguration;
 
 import java.util.function.Function;
@@ -26,14 +25,14 @@ public class CoordinatorRewriteContextProvider {
     private final Client client;
     private final LongSupplier nowInMillis;
     private final Supplier<ClusterState> clusterStateSupplier;
-    private final Function<Index, DateFieldRangeInfo> mappingSupplier;
+    private final Function<Index, DateFieldMapper.DateFieldType> mappingSupplier;
 
     public CoordinatorRewriteContextProvider(
         XContentParserConfiguration parserConfig,
         Client client,
         LongSupplier nowInMillis,
         Supplier<ClusterState> clusterStateSupplier,
-        Function<Index, DateFieldRangeInfo> mappingSupplier
+        Function<Index, DateFieldMapper.DateFieldType> mappingSupplier
     ) {
         this.parserConfig = parserConfig;
         this.client = client;
@@ -50,33 +49,18 @@ public class CoordinatorRewriteContextProvider {
         if (indexMetadata == null) {
             return null;
         }
-
-        DateFieldRangeInfo dateFieldRangeInfo = mappingSupplier.apply(index);
-        if (dateFieldRangeInfo == null) {
+        DateFieldMapper.DateFieldType dateFieldType = mappingSupplier.apply(index);
+        if (dateFieldType == null) {
             return null;
         }
-
-        DateFieldMapper.DateFieldType timestampFieldType = dateFieldRangeInfo.getTimestampFieldType();
-        DateFieldMapper.DateFieldType eventIngestedFieldType = dateFieldRangeInfo.getEventIngestedFieldType();
         IndexLongFieldRange timestampRange = indexMetadata.getTimestampRange();
-        IndexLongFieldRange eventIngestedRange = indexMetadata.getEventIngestedRange();
-
         if (timestampRange.containsAllShardRanges() == false) {
-            // if @timestamp range is not present or not ready in cluster state, fallback to using time series range (if present)
-            timestampRange = indexMetadata.getTimeSeriesTimestampRange(timestampFieldType);
-            // if timestampRange in the time series is null AND the eventIngestedRange is not ready for use, return null (no coord rewrite)
-            if (timestampRange == null && eventIngestedRange.containsAllShardRanges() == false) {
+            timestampRange = indexMetadata.getTimeSeriesTimestampRange(dateFieldType);
+            if (timestampRange == null) {
                 return null;
             }
         }
 
-        // the DateFieldRangeInfo from the mappingSupplier only has field types, but not ranges
-        // so create a new object with ranges pulled from cluster state
-        return new CoordinatorRewriteContext(
-            parserConfig,
-            client,
-            nowInMillis,
-            new DateFieldRangeInfo(timestampFieldType, timestampRange, eventIngestedFieldType, eventIngestedRange)
-        );
+        return new CoordinatorRewriteContext(parserConfig, client, nowInMillis, timestampRange, dateFieldType);
     }
 }

+ 3 - 3
server/src/main/java/org/elasticsearch/index/query/RangeQueryBuilder.java

@@ -436,11 +436,11 @@ public class RangeQueryBuilder extends AbstractQueryBuilder<RangeQueryBuilder> i
     protected MappedFieldType.Relation getRelation(final CoordinatorRewriteContext coordinatorRewriteContext) {
         final MappedFieldType fieldType = coordinatorRewriteContext.getFieldType(fieldName);
         if (fieldType instanceof final DateFieldMapper.DateFieldType dateFieldType) {
-            if (coordinatorRewriteContext.hasTimestampData(fieldName) == false) {
+            if (coordinatorRewriteContext.hasTimestampData() == false) {
                 return MappedFieldType.Relation.DISJOINT;
             }
-            long minTimestamp = coordinatorRewriteContext.getMinTimestamp(fieldName);
-            long maxTimestamp = coordinatorRewriteContext.getMaxTimestamp(fieldName);
+            long minTimestamp = coordinatorRewriteContext.getMinTimestamp();
+            long maxTimestamp = coordinatorRewriteContext.getMaxTimestamp();
             DateMathParser dateMathParser = getForceDateParser();
             return dateFieldType.isFieldWithinQuery(
                 minTimestamp,

+ 0 - 51
server/src/main/java/org/elasticsearch/indices/DateFieldRangeInfo.java

@@ -1,51 +0,0 @@
-/*
- * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
- * or more contributor license agreements. Licensed under the Elastic License
- * 2.0 and the Server Side Public License, v 1; you may not use this file except
- * in compliance with, at your election, the Elastic License 2.0 or the Server
- * Side Public License, v 1.
- */
-
-package org.elasticsearch.indices;
-
-import org.elasticsearch.index.mapper.DateFieldMapper;
-import org.elasticsearch.index.shard.IndexLongFieldRange;
-
-/**
- * Data holder of timestamp fields held in cluster state IndexMetadata.
- */
-public final class DateFieldRangeInfo {
-
-    private final DateFieldMapper.DateFieldType timestampFieldType;
-    private final IndexLongFieldRange timestampRange;
-    private final DateFieldMapper.DateFieldType eventIngestedFieldType;
-    private final IndexLongFieldRange eventIngestedRange;
-
-    public DateFieldRangeInfo(
-        DateFieldMapper.DateFieldType timestampFieldType,
-        IndexLongFieldRange timestampRange,
-        DateFieldMapper.DateFieldType eventIngestedFieldType,
-        IndexLongFieldRange eventIngestedRange
-    ) {
-        this.timestampFieldType = timestampFieldType;
-        this.timestampRange = timestampRange;
-        this.eventIngestedFieldType = eventIngestedFieldType;
-        this.eventIngestedRange = eventIngestedRange;
-    }
-
-    public DateFieldMapper.DateFieldType getTimestampFieldType() {
-        return timestampFieldType;
-    }
-
-    public IndexLongFieldRange getTimestampRange() {
-        return timestampRange;
-    }
-
-    public DateFieldMapper.DateFieldType getEventIngestedFieldType() {
-        return eventIngestedFieldType;
-    }
-
-    public IndexLongFieldRange getEventIngestedRange() {
-        return eventIngestedRange;
-    }
-}

+ 6 - 13
server/src/main/java/org/elasticsearch/indices/IndicesService.java

@@ -98,6 +98,7 @@ import org.elasticsearch.index.engine.NoOpEngine;
 import org.elasticsearch.index.fielddata.IndexFieldDataCache;
 import org.elasticsearch.index.flush.FlushStats;
 import org.elasticsearch.index.get.GetStats;
+import org.elasticsearch.index.mapper.DateFieldMapper;
 import org.elasticsearch.index.mapper.IdFieldMapper;
 import org.elasticsearch.index.mapper.MapperMetrics;
 import org.elasticsearch.index.mapper.MapperRegistry;
@@ -1763,13 +1764,7 @@ public class IndicesService extends AbstractLifecycleComponent
     }
 
     public CoordinatorRewriteContextProvider getCoordinatorRewriteContextProvider(LongSupplier nowInMillis) {
-        return new CoordinatorRewriteContextProvider(
-            parserConfig,
-            client,
-            nowInMillis,
-            clusterService::state,
-            this::getTimestampFieldTypeInfo
-        );
+        return new CoordinatorRewriteContextProvider(parserConfig, client, nowInMillis, clusterService::state, this::getTimestampFieldType);
     }
 
     /**
@@ -1859,16 +1854,14 @@ public class IndicesService extends AbstractLifecycleComponent
     }
 
     /**
-     * @return DateFieldRangeInfo holding the field types of the {@code @timestamp} and {@code event.ingested} fields of the index.
-     * or {@code null} if:
+     * @return the field type of the {@code @timestamp} field of the given index, or {@code null} if:
      * - the index is not found,
      * - the field is not found, or
-     * - the mapping is not known yet, or
-     * - the index does not have a useful timestamp field.
+     * - the field is not a timestamp field.
      */
     @Nullable
-    public DateFieldRangeInfo getTimestampFieldTypeInfo(Index index) {
-        return timestampFieldMapperService.getTimestampFieldTypeMap(index);
+    public DateFieldMapper.DateFieldType getTimestampFieldType(Index index) {
+        return timestampFieldMapperService.getTimestampFieldType(index);
     }
 
     public IndexScopedSettings getIndexScopedSettings() {

+ 19 - 37
server/src/main/java/org/elasticsearch/indices/TimestampFieldMapperService.java

@@ -42,9 +42,8 @@ import static org.elasticsearch.common.util.concurrent.EsExecutors.daemonThreadF
 import static org.elasticsearch.core.Strings.format;
 
 /**
- * Tracks the mapping of the '@timestamp' and 'event.ingested' fields of immutable indices that expose their timestamp range in their
- * index metadata. Coordinating nodes do not have (easy) access to mappings for all indices, so we extract the type of these two fields
- * from the mapping here, since timestamp fields can have millis or nanos level resolution.
+ * Tracks the mapping of the {@code @timestamp} field of immutable indices that expose their timestamp range in their index metadata.
+ * Coordinating nodes do not have (easy) access to mappings for all indices, so we extract the type of this one field from the mapping here.
  */
 public class TimestampFieldMapperService extends AbstractLifecycleComponent implements ClusterStateApplier {
 
@@ -54,12 +53,10 @@ public class TimestampFieldMapperService extends AbstractLifecycleComponent impl
     private final ExecutorService executor; // single thread to construct mapper services async as needed
 
     /**
-     * The type of the 'event.ingested' and/or '@timestamp' fields keyed by index.
-     * The inner map is keyed by field name ('@timestamp' or 'event.ingested').
-     * Futures may be completed with {@code null} to indicate that there is
-     * no usable timestamp field.
+     * The type of the {@code @timestamp} field keyed by index. Futures may be completed with {@code null} to indicate that there is
+     * no usable {@code @timestamp} field.
      */
-    private final Map<Index, PlainActionFuture<DateFieldRangeInfo>> fieldTypesByIndex = ConcurrentCollections.newConcurrentMap();
+    private final Map<Index, PlainActionFuture<DateFieldMapper.DateFieldType>> fieldTypesByIndex = ConcurrentCollections.newConcurrentMap();
 
     public TimestampFieldMapperService(Settings settings, ThreadPool threadPool, IndicesService indicesService) {
         this.indicesService = indicesService;
@@ -105,8 +102,8 @@ public class TimestampFieldMapperService extends AbstractLifecycleComponent impl
             final Index index = indexMetadata.getIndex();
 
             if (hasUsefulTimestampField(indexMetadata) && fieldTypesByIndex.containsKey(index) == false) {
-                logger.trace("computing timestamp mapping(s) for {}", index);
-                final PlainActionFuture<DateFieldRangeInfo> future = new PlainActionFuture<>();
+                logger.trace("computing timestamp mapping for {}", index);
+                final PlainActionFuture<DateFieldMapper.DateFieldType> future = new PlainActionFuture<>();
                 fieldTypesByIndex.put(index, future);
 
                 final IndexService indexService = indicesService.indexService(index);
@@ -151,45 +148,29 @@ public class TimestampFieldMapperService extends AbstractLifecycleComponent impl
             return true;
         }
 
-        IndexLongFieldRange timestampRange = indexMetadata.getTimestampRange();
-        if (timestampRange.isComplete() && timestampRange != IndexLongFieldRange.UNKNOWN) {
-            return true;
-        }
-
-        IndexLongFieldRange eventIngestedRange = indexMetadata.getEventIngestedRange();
-        return eventIngestedRange.isComplete() && eventIngestedRange != IndexLongFieldRange.UNKNOWN;
+        final IndexLongFieldRange timestampRange = indexMetadata.getTimestampRange();
+        return timestampRange.isComplete() && timestampRange != IndexLongFieldRange.UNKNOWN;
     }
 
-    private static DateFieldRangeInfo fromMapperService(MapperService mapperService) {
-        DateFieldMapper.DateFieldType timestampFieldType = null;
-        DateFieldMapper.DateFieldType eventIngestedFieldType = null;
-
-        MappedFieldType mappedFieldType = mapperService.fieldType(DataStream.TIMESTAMP_FIELD_NAME);
-        if (mappedFieldType instanceof DateFieldMapper.DateFieldType dateFieldType) {
-            timestampFieldType = dateFieldType;
-        }
-        mappedFieldType = mapperService.fieldType(IndexMetadata.EVENT_INGESTED_FIELD_NAME);
-        if (mappedFieldType instanceof DateFieldMapper.DateFieldType dateFieldType) {
-            eventIngestedFieldType = dateFieldType;
-        }
-        if (timestampFieldType == null && eventIngestedFieldType == null) {
+    private static DateFieldMapper.DateFieldType fromMapperService(MapperService mapperService) {
+        final MappedFieldType mappedFieldType = mapperService.fieldType(DataStream.TIMESTAMP_FIELD_NAME);
+        if (mappedFieldType instanceof DateFieldMapper.DateFieldType) {
+            return (DateFieldMapper.DateFieldType) mappedFieldType;
+        } else {
             return null;
         }
-        // the mapper only fills in the field types, not the actual range values
-        return new DateFieldRangeInfo(timestampFieldType, null, eventIngestedFieldType, null);
     }
 
     /**
-     * @return DateFieldRangeInfo holding the field types of the {@code @timestamp} and {@code event.ingested} fields of the index.
-     * or {@code null} if:
+     * @return the field type of the {@code @timestamp} field of the given index, or {@code null} if:
      * - the index is not found,
      * - the field is not found,
      * - the mapping is not known yet, or
-     * - the index does not have a useful timestamp field.
+     * - the field is not a timestamp field.
      */
     @Nullable
-    public DateFieldRangeInfo getTimestampFieldTypeMap(Index index) {
-        final PlainActionFuture<DateFieldRangeInfo> future = fieldTypesByIndex.get(index);
+    public DateFieldMapper.DateFieldType getTimestampFieldType(Index index) {
+        final PlainActionFuture<DateFieldMapper.DateFieldType> future = fieldTypesByIndex.get(index);
         if (future == null || future.isDone() == false) {
             return null;
         }
@@ -200,4 +181,5 @@ public class TimestampFieldMapperService extends AbstractLifecycleComponent impl
             throw new UncategorizedExecutionException("An error occurred fetching timestamp field type for " + index, e);
         }
     }
+
 }

+ 59 - 281
server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java

@@ -9,7 +9,6 @@
 package org.elasticsearch.action.search;
 
 import org.apache.lucene.util.BytesRef;
-import org.elasticsearch.TransportVersion;
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.action.OriginalIndices;
 import org.elasticsearch.action.search.CanMatchNodeResponse.ResponseOrFailure;
@@ -27,6 +26,8 @@ import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.UUIDs;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.index.Index;
+import org.elasticsearch.index.IndexMode;
+import org.elasticsearch.index.IndexSettings;
 import org.elasticsearch.index.IndexVersion;
 import org.elasticsearch.index.mapper.DateFieldMapper;
 import org.elasticsearch.index.query.BoolQueryBuilder;
@@ -37,7 +38,6 @@ import org.elasticsearch.index.query.TermQueryBuilder;
 import org.elasticsearch.index.shard.IndexLongFieldRange;
 import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.index.shard.ShardLongFieldRange;
-import org.elasticsearch.indices.DateFieldRangeInfo;
 import org.elasticsearch.search.CanMatchShardResponse;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.bucket.terms.SignificantTermsAggregationBuilder;
@@ -72,7 +72,6 @@ import java.util.stream.IntStream;
 import static org.elasticsearch.action.search.SearchAsyncActionTests.getShardsIter;
 import static org.elasticsearch.core.Types.forciblyCast;
 import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.greaterThan;
 import static org.hamcrest.Matchers.lessThanOrEqualTo;
 import static org.mockito.Mockito.mock;
 
@@ -465,17 +464,7 @@ public class CanMatchPreFilterSearchPhaseTests extends ESTestCase {
         }
     }
 
-    // test using @timestamp
-    public void testCanMatchFilteringOnCoordinatorThatCanBeSkippedUsingTimestamp() throws Exception {
-        doCanMatchFilteringOnCoordinatorThatCanBeSkipped(DataStream.TIMESTAMP_FIELD_NAME);
-    }
-
-    // test using event.ingested
-    public void testCanMatchFilteringOnCoordinatorThatCanBeSkippedUsingEventIngested() throws Exception {
-        doCanMatchFilteringOnCoordinatorThatCanBeSkipped(IndexMetadata.EVENT_INGESTED_FIELD_NAME);
-    }
-
-    public void doCanMatchFilteringOnCoordinatorThatCanBeSkipped(String timestampField) throws Exception {
+    public void testCanMatchFilteringOnCoordinatorThatCanBeSkipped() throws Exception {
         Index dataStreamIndex1 = new Index(".ds-mydata0001", UUIDs.base64UUID());
         Index dataStreamIndex2 = new Index(".ds-mydata0002", UUIDs.base64UUID());
         DataStream dataStream = DataStreamTestHelper.newInstance("mydata", List.of(dataStreamIndex1, dataStreamIndex2));
@@ -486,10 +475,15 @@ public class CanMatchPreFilterSearchPhaseTests extends ESTestCase {
         long indexMaxTimestamp = randomLongBetween(indexMinTimestamp, 5000 * 2);
         StaticCoordinatorRewriteContextProviderBuilder contextProviderBuilder = new StaticCoordinatorRewriteContextProviderBuilder();
         for (Index dataStreamIndex : dataStream.getIndices()) {
-            contextProviderBuilder.addIndexMinMaxTimestamps(dataStreamIndex, timestampField, indexMinTimestamp, indexMaxTimestamp);
+            contextProviderBuilder.addIndexMinMaxTimestamps(
+                dataStreamIndex,
+                DataStream.TIMESTAMP_FIELD_NAME,
+                indexMinTimestamp,
+                indexMaxTimestamp
+            );
         }
 
-        RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder(timestampField);
+        RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder(DataStream.TIMESTAMP_FIELD_NAME);
         // We query a range outside of the timestamp range covered by both datastream indices
         rangeQueryBuilder.from(indexMaxTimestamp + 1).to(indexMaxTimestamp + 2);
 
@@ -541,90 +535,6 @@ public class CanMatchPreFilterSearchPhaseTests extends ESTestCase {
         );
     }
 
-    public void testCoordinatorCanMatchFilteringThatCanBeSkippedUsingBothTimestamps() throws Exception {
-        Index dataStreamIndex1 = new Index(".ds-twoTimestamps0001", UUIDs.base64UUID());
-        Index dataStreamIndex2 = new Index(".ds-twoTimestamps0002", UUIDs.base64UUID());
-        DataStream dataStream = DataStreamTestHelper.newInstance("mydata", List.of(dataStreamIndex1, dataStreamIndex2));
-
-        List<Index> regularIndices = randomList(1, 2, () -> new Index(randomAlphaOfLength(10), UUIDs.base64UUID()));
-
-        long indexMinTimestamp = randomLongBetween(0, 5000);
-        long indexMaxTimestamp = randomLongBetween(indexMinTimestamp, 5000 * 2);
-        StaticCoordinatorRewriteContextProviderBuilder contextProviderBuilder = new StaticCoordinatorRewriteContextProviderBuilder();
-        for (Index dataStreamIndex : dataStream.getIndices()) {
-            // use same range for both @timestamp and event.ingested
-            contextProviderBuilder.addIndexMinMaxForTimestampAndEventIngested(
-                dataStreamIndex,
-                indexMinTimestamp,
-                indexMaxTimestamp,
-                indexMinTimestamp,
-                indexMaxTimestamp
-            );
-        }
-
-        /**
-         * Expected behavior: if either @timestamp or 'event.ingested' filters in the query are "out of range" (do not
-         * overlap the range in cluster state), then all shards in the datastream should be skipped.
-         * Only if both @timestamp or 'event.ingested' filters are "in range" should the data stream shards be searched
-         */
-        boolean timestampQueryOutOfRange = randomBoolean();
-        boolean eventIngestedQueryOutOfRange = randomBoolean();
-        int timestampOffset = timestampQueryOutOfRange ? 1 : -500;
-        int eventIngestedOffset = eventIngestedQueryOutOfRange ? 1 : -500;
-
-        RangeQueryBuilder tsRangeQueryBuilder = new RangeQueryBuilder(DataStream.TIMESTAMP_FIELD_NAME);
-        tsRangeQueryBuilder.from(indexMaxTimestamp + timestampOffset).to(indexMaxTimestamp + 2);
-
-        RangeQueryBuilder eventIngestedRangeQueryBuilder = new RangeQueryBuilder(IndexMetadata.EVENT_INGESTED_FIELD_NAME);
-        eventIngestedRangeQueryBuilder.from(indexMaxTimestamp + eventIngestedOffset).to(indexMaxTimestamp + 2);
-
-        BoolQueryBuilder queryBuilder = new BoolQueryBuilder().filter(tsRangeQueryBuilder).filter(eventIngestedRangeQueryBuilder);
-
-        if (randomBoolean()) {
-            // Add an additional filter that cannot be evaluated in the coordinator but shouldn't
-            // affect the end result as we're filtering
-            queryBuilder.filter(new TermQueryBuilder("fake", "value"));
-        }
-
-        assignShardsAndExecuteCanMatchPhase(
-            List.of(dataStream),
-            regularIndices,
-            contextProviderBuilder.build(),
-            queryBuilder,
-            List.of(),
-            null,
-            (updatedSearchShardIterators, requests) -> {
-                List<SearchShardIterator> skippedShards = updatedSearchShardIterators.stream().filter(SearchShardIterator::skip).toList();
-                List<SearchShardIterator> nonSkippedShards = updatedSearchShardIterators.stream()
-                    .filter(searchShardIterator -> searchShardIterator.skip() == false)
-                    .toList();
-
-                if (timestampQueryOutOfRange || eventIngestedQueryOutOfRange) {
-                    // data stream shards should have been skipped
-                    assertThat(skippedShards.size(), greaterThan(0));
-                    boolean allSkippedShardAreFromDataStream = skippedShards.stream()
-                        .allMatch(shardIterator -> dataStream.getIndices().contains(shardIterator.shardId().getIndex()));
-                    assertThat(allSkippedShardAreFromDataStream, equalTo(true));
-
-                    boolean allNonSkippedShardsAreFromRegularIndices = nonSkippedShards.stream()
-                        .allMatch(shardIterator -> regularIndices.contains(shardIterator.shardId().getIndex()));
-                    assertThat(allNonSkippedShardsAreFromRegularIndices, equalTo(true));
-
-                    boolean allRequestsWereTriggeredAgainstRegularIndices = requests.stream()
-                        .allMatch(request -> regularIndices.contains(request.shardId().getIndex()));
-                    assertThat(allRequestsWereTriggeredAgainstRegularIndices, equalTo(true));
-
-                } else {
-                    assertThat(skippedShards.size(), equalTo(0));
-                    long countSkippedShardsFromDatastream = nonSkippedShards.stream()
-                        .filter(iter -> dataStream.getIndices().contains(iter.shardId().getIndex()))
-                        .count();
-                    assertThat(countSkippedShardsFromDatastream, greaterThan(0L));
-                }
-            }
-        );
-    }
-
     public void testCanMatchFilteringOnCoordinatorParsingFails() throws Exception {
         Index dataStreamIndex1 = new Index(".ds-mydata0001", UUIDs.base64UUID());
         Index dataStreamIndex2 = new Index(".ds-mydata0002", UUIDs.base64UUID());
@@ -632,16 +542,19 @@ public class CanMatchPreFilterSearchPhaseTests extends ESTestCase {
 
         List<Index> regularIndices = randomList(0, 2, () -> new Index(randomAlphaOfLength(10), UUIDs.base64UUID()));
 
-        String timeField = randomFrom(DataStream.TIMESTAMP_FIELD_NAME, IndexMetadata.EVENT_INGESTED_FIELD_NAME);
-
         long indexMinTimestamp = randomLongBetween(0, 5000);
         long indexMaxTimestamp = randomLongBetween(indexMinTimestamp, 5000 * 2);
         StaticCoordinatorRewriteContextProviderBuilder contextProviderBuilder = new StaticCoordinatorRewriteContextProviderBuilder();
         for (Index dataStreamIndex : dataStream.getIndices()) {
-            contextProviderBuilder.addIndexMinMaxTimestamps(dataStreamIndex, timeField, indexMinTimestamp, indexMaxTimestamp);
+            contextProviderBuilder.addIndexMinMaxTimestamps(
+                dataStreamIndex,
+                DataStream.TIMESTAMP_FIELD_NAME,
+                indexMinTimestamp,
+                indexMaxTimestamp
+            );
         }
 
-        RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder(timeField);
+        RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder(DataStream.TIMESTAMP_FIELD_NAME);
         // Query with a non default date format
         rangeQueryBuilder.from("2020-1-01").to("2021-1-01");
 
@@ -672,20 +585,23 @@ public class CanMatchPreFilterSearchPhaseTests extends ESTestCase {
 
         List<Index> regularIndices = randomList(0, 2, () -> new Index(randomAlphaOfLength(10), UUIDs.base64UUID()));
 
-        String timeField = randomFrom(DataStream.TIMESTAMP_FIELD_NAME, IndexMetadata.EVENT_INGESTED_FIELD_NAME);
-
         long indexMinTimestamp = 10;
         long indexMaxTimestamp = 20;
         StaticCoordinatorRewriteContextProviderBuilder contextProviderBuilder = new StaticCoordinatorRewriteContextProviderBuilder();
         for (Index dataStreamIndex : dataStream.getIndices()) {
-            contextProviderBuilder.addIndexMinMaxTimestamps(dataStreamIndex, timeField, indexMinTimestamp, indexMaxTimestamp);
+            contextProviderBuilder.addIndexMinMaxTimestamps(
+                dataStreamIndex,
+                DataStream.TIMESTAMP_FIELD_NAME,
+                indexMinTimestamp,
+                indexMaxTimestamp
+            );
         }
 
         BoolQueryBuilder queryBuilder = new BoolQueryBuilder();
         // Query inside of the data stream index range
         if (randomBoolean()) {
             // Query generation
-            RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder(timeField);
+            RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder(DataStream.TIMESTAMP_FIELD_NAME);
             // We query a range within the timestamp range covered by both datastream indices
             rangeQueryBuilder.from(indexMinTimestamp).to(indexMaxTimestamp);
 
@@ -698,7 +614,8 @@ public class CanMatchPreFilterSearchPhaseTests extends ESTestCase {
             }
         } else {
             // We query a range outside of the timestamp range covered by both datastream indices
-            RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder(timeField).from(indexMaxTimestamp + 1).to(indexMaxTimestamp + 2);
+            RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder(DataStream.TIMESTAMP_FIELD_NAME).from(indexMaxTimestamp + 1)
+                .to(indexMaxTimestamp + 2);
 
             TermQueryBuilder termQueryBuilder = new TermQueryBuilder("fake", "value");
 
@@ -718,86 +635,17 @@ public class CanMatchPreFilterSearchPhaseTests extends ESTestCase {
         );
     }
 
-    public void testCanMatchFilteringOnCoordinatorWithTimestampAndEventIngestedThatCanNotBeSkipped() throws Exception {
-        // Generate indices
-        Index dataStreamIndex1 = new Index(".ds-mydata0001", UUIDs.base64UUID());
-        Index dataStreamIndex2 = new Index(".ds-mydata0002", UUIDs.base64UUID());
-        DataStream dataStream = DataStreamTestHelper.newInstance("mydata", List.of(dataStreamIndex1, dataStreamIndex2));
-
-        List<Index> regularIndices = randomList(0, 2, () -> new Index(randomAlphaOfLength(10), UUIDs.base64UUID()));
-
-        long indexMinTimestampForTs = 10;
-        long indexMaxTimestampForTs = 20;
-        long indexMinTimestampForEventIngested = 10;
-        long indexMaxTimestampForEventIngested = 20;
-        StaticCoordinatorRewriteContextProviderBuilder contextProviderBuilder = new StaticCoordinatorRewriteContextProviderBuilder();
-        for (Index dataStreamIndex : dataStream.getIndices()) {
-            contextProviderBuilder.addIndexMinMaxForTimestampAndEventIngested(
-                dataStreamIndex,
-                indexMinTimestampForTs,
-                indexMaxTimestampForTs,
-                indexMinTimestampForEventIngested,
-                indexMaxTimestampForEventIngested
-            );
-        }
-
-        BoolQueryBuilder queryBuilder = new BoolQueryBuilder();
-        // Query inside of the data stream index range
-        if (randomBoolean()) {
-            // Query generation
-            // We query a range within both timestamp ranges covered by both datastream indices
-            RangeQueryBuilder tsRangeQueryBuilder = new RangeQueryBuilder(DataStream.TIMESTAMP_FIELD_NAME);
-            tsRangeQueryBuilder.from(indexMinTimestampForTs).to(indexMaxTimestampForTs);
-
-            RangeQueryBuilder eventIngestedRangeQueryBuilder = new RangeQueryBuilder(IndexMetadata.EVENT_INGESTED_FIELD_NAME);
-            eventIngestedRangeQueryBuilder.from(indexMinTimestampForEventIngested).to(indexMaxTimestampForEventIngested);
-
-            queryBuilder.filter(tsRangeQueryBuilder).filter(eventIngestedRangeQueryBuilder);
-
-            if (randomBoolean()) {
-                // Add an additional filter that cannot be evaluated in the coordinator but shouldn't
-                // affect the end result as we're filtering
-                queryBuilder.filter(new TermQueryBuilder("fake", "value"));
-            }
-        } else {
-            // We query a range outside of the both ranges covered by both datastream indices
-            RangeQueryBuilder tsRangeQueryBuilder = new RangeQueryBuilder(DataStream.TIMESTAMP_FIELD_NAME).from(indexMaxTimestampForTs + 1)
-                .to(indexMaxTimestampForTs + 2);
-            RangeQueryBuilder eventIngestedRangeQueryBuilder = new RangeQueryBuilder(IndexMetadata.EVENT_INGESTED_FIELD_NAME).from(
-                indexMaxTimestampForEventIngested + 1
-            ).to(indexMaxTimestampForEventIngested + 2);
-
-            TermQueryBuilder termQueryBuilder = new TermQueryBuilder("fake", "value");
-
-            // This is always evaluated as true in the coordinator as we cannot determine there if
-            // the term query clause is false.
-            queryBuilder.should(tsRangeQueryBuilder).should(eventIngestedRangeQueryBuilder).should(termQueryBuilder);
-        }
-
-        assignShardsAndExecuteCanMatchPhase(
-            List.of(dataStream),
-            regularIndices,
-            contextProviderBuilder.build(),
-            queryBuilder,
-            List.of(),
-            null,
-            this::assertAllShardsAreQueried
-        );
-    }
-
     public void testCanMatchFilteringOnCoordinator_withSignificantTermsAggregation_withDefaultBackgroundFilter() throws Exception {
         Index index1 = new Index("index1", UUIDs.base64UUID());
         Index index2 = new Index("index2", UUIDs.base64UUID());
         Index index3 = new Index("index3", UUIDs.base64UUID());
 
-        String timeField = randomFrom(DataStream.TIMESTAMP_FIELD_NAME, IndexMetadata.EVENT_INGESTED_FIELD_NAME);
-
         StaticCoordinatorRewriteContextProviderBuilder contextProviderBuilder = new StaticCoordinatorRewriteContextProviderBuilder();
-        contextProviderBuilder.addIndexMinMaxTimestamps(index1, timeField, 0, 999);
-        contextProviderBuilder.addIndexMinMaxTimestamps(index2, timeField, 1000, 1999);
-        contextProviderBuilder.addIndexMinMaxTimestamps(index3, timeField, 2000, 2999);
+        contextProviderBuilder.addIndexMinMaxTimestamps(index1, DataStream.TIMESTAMP_FIELD_NAME, 0, 999);
+        contextProviderBuilder.addIndexMinMaxTimestamps(index2, DataStream.TIMESTAMP_FIELD_NAME, 1000, 1999);
+        contextProviderBuilder.addIndexMinMaxTimestamps(index3, DataStream.TIMESTAMP_FIELD_NAME, 2000, 2999);
 
-        QueryBuilder query = new BoolQueryBuilder().filter(new RangeQueryBuilder(timeField).from(2100).to(2200));
+        QueryBuilder query = new BoolQueryBuilder().filter(new RangeQueryBuilder(DataStream.TIMESTAMP_FIELD_NAME).from(2100).to(2200));
         AggregationBuilder aggregation = new SignificantTermsAggregationBuilder("significant_terms");
 
         assignShardsAndExecuteCanMatchPhase(
@@ -813,22 +661,20 @@ public class CanMatchPreFilterSearchPhaseTests extends ESTestCase {
     }
 
     public void testCanMatchFilteringOnCoordinator_withSignificantTermsAggregation_withBackgroundFilter() throws Exception {
-        String timestampField = randomFrom(IndexMetadata.EVENT_INGESTED_FIELD_NAME, DataStream.TIMESTAMP_FIELD_NAME);
-
         Index index1 = new Index("index1", UUIDs.base64UUID());
         Index index2 = new Index("index2", UUIDs.base64UUID());
         Index index3 = new Index("index3", UUIDs.base64UUID());
         Index index4 = new Index("index4", UUIDs.base64UUID());
 
         StaticCoordinatorRewriteContextProviderBuilder contextProviderBuilder = new StaticCoordinatorRewriteContextProviderBuilder();
-        contextProviderBuilder.addIndexMinMaxTimestamps(index1, timestampField, 0, 999);
-        contextProviderBuilder.addIndexMinMaxTimestamps(index2, timestampField, 1000, 1999);
-        contextProviderBuilder.addIndexMinMaxTimestamps(index3, timestampField, 2000, 2999);
-        contextProviderBuilder.addIndexMinMaxTimestamps(index4, timestampField, 3000, 3999);
+        contextProviderBuilder.addIndexMinMaxTimestamps(index1, DataStream.TIMESTAMP_FIELD_NAME, 0, 999);
+        contextProviderBuilder.addIndexMinMaxTimestamps(index2, DataStream.TIMESTAMP_FIELD_NAME, 1000, 1999);
+        contextProviderBuilder.addIndexMinMaxTimestamps(index3, DataStream.TIMESTAMP_FIELD_NAME, 2000, 2999);
+        contextProviderBuilder.addIndexMinMaxTimestamps(index4, DataStream.TIMESTAMP_FIELD_NAME, 3000, 3999);
 
-        QueryBuilder query = new BoolQueryBuilder().filter(new RangeQueryBuilder(timestampField).from(3100).to(3200));
+        QueryBuilder query = new BoolQueryBuilder().filter(new RangeQueryBuilder(DataStream.TIMESTAMP_FIELD_NAME).from(3100).to(3200));
         AggregationBuilder aggregation = new SignificantTermsAggregationBuilder("significant_terms").backgroundFilter(
-            new RangeQueryBuilder(timestampField).from(0).to(1999)
+            new RangeQueryBuilder(DataStream.TIMESTAMP_FIELD_NAME).from(0).to(1999)
         );
 
         assignShardsAndExecuteCanMatchPhase(
@@ -857,53 +703,14 @@ public class CanMatchPreFilterSearchPhaseTests extends ESTestCase {
         Index index2 = new Index("index2", UUIDs.base64UUID());
         Index index3 = new Index("index3", UUIDs.base64UUID());
 
-        String timestampField = randomFrom(IndexMetadata.EVENT_INGESTED_FIELD_NAME, DataStream.TIMESTAMP_FIELD_NAME);
-
         StaticCoordinatorRewriteContextProviderBuilder contextProviderBuilder = new StaticCoordinatorRewriteContextProviderBuilder();
-        contextProviderBuilder.addIndexMinMaxTimestamps(index1, timestampField, 0, 999);
-        contextProviderBuilder.addIndexMinMaxTimestamps(index2, timestampField, 1000, 1999);
-        contextProviderBuilder.addIndexMinMaxTimestamps(index3, timestampField, 2000, 2999);
+        contextProviderBuilder.addIndexMinMaxTimestamps(index1, DataStream.TIMESTAMP_FIELD_NAME, 0, 999);
+        contextProviderBuilder.addIndexMinMaxTimestamps(index2, DataStream.TIMESTAMP_FIELD_NAME, 1000, 1999);
+        contextProviderBuilder.addIndexMinMaxTimestamps(index3, DataStream.TIMESTAMP_FIELD_NAME, 2000, 2999);
 
-        QueryBuilder query = new BoolQueryBuilder().filter(new RangeQueryBuilder(timestampField).from(2100).to(2200));
+        QueryBuilder query = new BoolQueryBuilder().filter(new RangeQueryBuilder(DataStream.TIMESTAMP_FIELD_NAME).from(2100).to(2200));
         AggregationBuilder aggregation = new SignificantTermsAggregationBuilder("significant_terms").backgroundFilter(
-            new RangeQueryBuilder(timestampField).from(2000).to(2300)
-        );
-        SuggestBuilder suggest = new SuggestBuilder().setGlobalText("test");
-
-        assignShardsAndExecuteCanMatchPhase(
-            List.of(),
-            List.of(index1, index2, index3),
-            contextProviderBuilder.build(),
-            query,
-            List.of(aggregation),
-            suggest,
-            // The query and aggregation and match only index3, but suggest should match everything.
-            this::assertAllShardsAreQueried
-        );
-    }
-
-    public void testCanMatchFilteringOnCoordinator_withSignificantTermsAggregation_withSuggest_withTwoTimestamps() throws Exception {
-        Index index1 = new Index("index1", UUIDs.base64UUID());
-        Index index2 = new Index("index2", UUIDs.base64UUID());
-        Index index3 = new Index("index3", UUIDs.base64UUID());
-
-        StaticCoordinatorRewriteContextProviderBuilder contextProviderBuilder = new StaticCoordinatorRewriteContextProviderBuilder();
-        contextProviderBuilder.addIndexMinMaxForTimestampAndEventIngested(index1, 0, 999, 0, 999);
-        contextProviderBuilder.addIndexMinMaxForTimestampAndEventIngested(index2, 1000, 1999, 1000, 1999);
-        contextProviderBuilder.addIndexMinMaxForTimestampAndEventIngested(index3, 2000, 2999, 2000, 2999);
-
-        String fieldInRange = IndexMetadata.EVENT_INGESTED_FIELD_NAME;
-        String fieldOutOfRange = DataStream.TIMESTAMP_FIELD_NAME;
-
-        if (randomBoolean()) {
-            fieldInRange = DataStream.TIMESTAMP_FIELD_NAME;
-            fieldOutOfRange = IndexMetadata.EVENT_INGESTED_FIELD_NAME;
-        }
-
-        QueryBuilder query = new BoolQueryBuilder().filter(new RangeQueryBuilder(fieldInRange).from(2100).to(2200))
-            .filter(new RangeQueryBuilder(fieldOutOfRange).from(8888).to(9999));
-        AggregationBuilder aggregation = new SignificantTermsAggregationBuilder("significant_terms").backgroundFilter(
-            new RangeQueryBuilder(fieldInRange).from(2000).to(2300)
+            new RangeQueryBuilder(DataStream.TIMESTAMP_FIELD_NAME).from(2000).to(2300)
         );
         SuggestBuilder suggest = new SuggestBuilder().setGlobalText("test");
 
@@ -937,13 +744,13 @@ public class CanMatchPreFilterSearchPhaseTests extends ESTestCase {
         long indexMaxTimestamp = randomLongBetween(indexMinTimestamp, 5000 * 2);
         StaticCoordinatorRewriteContextProviderBuilder contextProviderBuilder = new StaticCoordinatorRewriteContextProviderBuilder();
         for (Index index : dataStream1.getIndices()) {
-            contextProviderBuilder.addIndexMinMaxTimestamps(index, DataStream.TIMESTAMP_FIELD_NAME, indexMinTimestamp, indexMaxTimestamp);
+            contextProviderBuilder.addIndexMinMaxTimestamps(index, indexMinTimestamp, indexMaxTimestamp);
         }
         for (Index index : dataStream2.getIndices()) {
             contextProviderBuilder.addIndex(index);
         }
 
-        RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder(DataStream.TIMESTAMP_FIELD_NAME);
+        RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder("@timestamp");
         // We query a range outside of the timestamp range covered by both datastream indices
         rangeQueryBuilder.from(indexMaxTimestamp + 1).to(indexMaxTimestamp + 2);
 
@@ -1147,9 +954,9 @@ public class CanMatchPreFilterSearchPhaseTests extends ESTestCase {
         canMatchResultsConsumer.accept(updatedSearchShardIterators, requests);
     }
 
-    static class StaticCoordinatorRewriteContextProviderBuilder {
+    private static class StaticCoordinatorRewriteContextProviderBuilder {
         private ClusterState clusterState = ClusterState.EMPTY_STATE;
-        private final Map<Index, DateFieldRangeInfo> fields = new HashMap<>();
+        private final Map<Index, DateFieldMapper.DateFieldType> fields = new HashMap<>();
 
         private void addIndexMinMaxTimestamps(Index index, String fieldName, long minTimeStamp, long maxTimestamp) {
             if (clusterState.metadata().index(index) != null) {
@@ -1167,64 +974,35 @@ public class CanMatchPreFilterSearchPhaseTests extends ESTestCase {
             IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(index.getName())
                 .settings(indexSettings)
                 .numberOfShards(1)
-                .numberOfReplicas(0);
-            if (fieldName.equals(DataStream.TIMESTAMP_FIELD_NAME)) {
-                indexMetadataBuilder.timestampRange(timestampRange);
-                fields.put(index, new DateFieldRangeInfo(new DateFieldMapper.DateFieldType(fieldName), null, null, null));
-            } else if (fieldName.equals(IndexMetadata.EVENT_INGESTED_FIELD_NAME)) {
-                indexMetadataBuilder.eventIngestedRange(timestampRange, TransportVersion.current());
-                fields.put(index, new DateFieldRangeInfo(null, null, new DateFieldMapper.DateFieldType(fieldName), null));
-            }
+                .numberOfReplicas(0)
+                .timestampRange(timestampRange);
 
             Metadata.Builder metadataBuilder = Metadata.builder(clusterState.metadata()).put(indexMetadataBuilder);
+
             clusterState = ClusterState.builder(clusterState).metadata(metadataBuilder).build();
+
+            fields.put(index, new DateFieldMapper.DateFieldType(fieldName));
         }
 
-        /**
-         * Add min/max timestamps to IndexMetadata for the specified index for both @timestamp and 'event.ingested'
-         */
-        private void addIndexMinMaxForTimestampAndEventIngested(
-            Index index,
-            long minTimestampForTs,
-            long maxTimestampForTs,
-            long minTimestampForEventIngested,
-            long maxTimestampForEventIngested
-        ) {
+        private void addIndexMinMaxTimestamps(Index index, long minTimestamp, long maxTimestamp) {
             if (clusterState.metadata().index(index) != null) {
                 throw new IllegalArgumentException("Min/Max timestamps for " + index + " were already defined");
             }
 
-            IndexLongFieldRange tsTimestampRange = IndexLongFieldRange.NO_SHARDS.extendWithShardRange(
-                0,
-                1,
-                ShardLongFieldRange.of(minTimestampForTs, maxTimestampForTs)
-            );
-            IndexLongFieldRange eventIngestedTimestampRange = IndexLongFieldRange.NO_SHARDS.extendWithShardRange(
-                0,
-                1,
-                ShardLongFieldRange.of(minTimestampForEventIngested, maxTimestampForEventIngested)
-            );
-
-            Settings.Builder indexSettings = settings(IndexVersion.current()).put(IndexMetadata.SETTING_INDEX_UUID, index.getUUID());
+            Settings.Builder indexSettings = settings(IndexVersion.current()).put(IndexMetadata.SETTING_INDEX_UUID, index.getUUID())
+                .put(IndexSettings.MODE.getKey(), IndexMode.TIME_SERIES)
+                .put(IndexMetadata.INDEX_ROUTING_PATH.getKey(), "a_field")
+                .put(IndexSettings.TIME_SERIES_START_TIME.getKey(), DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.formatMillis(minTimestamp))
+                .put(IndexSettings.TIME_SERIES_END_TIME.getKey(), DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.formatMillis(maxTimestamp));
 
             IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(index.getName())
                 .settings(indexSettings)
                 .numberOfShards(1)
-                .numberOfReplicas(0)
-                .timestampRange(tsTimestampRange)
-                .eventIngestedRange(eventIngestedTimestampRange, TransportVersion.current());
+                .numberOfReplicas(0);
 
             Metadata.Builder metadataBuilder = Metadata.builder(clusterState.metadata()).put(indexMetadataBuilder);
             clusterState = ClusterState.builder(clusterState).metadata(metadataBuilder).build();
-            fields.put(
-                index,
-                new DateFieldRangeInfo(
-                    new DateFieldMapper.DateFieldType(DataStream.TIMESTAMP_FIELD_NAME),
-                    null,
-                    new DateFieldMapper.DateFieldType(IndexMetadata.EVENT_INGESTED_FIELD_NAME),
-                    null
-                )
-            );
+            fields.put(index, new DateFieldMapper.DateFieldType("@timestamp"));
         }
 
         private void addIndex(Index index) {
@@ -1240,7 +1018,7 @@ public class CanMatchPreFilterSearchPhaseTests extends ESTestCase {
 
             Metadata.Builder metadataBuilder = Metadata.builder(clusterState.metadata()).put(indexMetadataBuilder);
             clusterState = ClusterState.builder(clusterState).metadata(metadataBuilder).build();
-            fields.put(index, new DateFieldRangeInfo(new DateFieldMapper.DateFieldType(DataStream.TIMESTAMP_FIELD_NAME), null, null, null));
+            fields.put(index, new DateFieldMapper.DateFieldType("@timestamp"));
         }
 
         public CoordinatorRewriteContextProvider build() {

+ 5 - 6
test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java

@@ -59,7 +59,6 @@ import org.elasticsearch.index.shard.IndexLongFieldRange;
 import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.index.shard.ShardLongFieldRange;
 import org.elasticsearch.index.similarity.SimilarityService;
-import org.elasticsearch.indices.DateFieldRangeInfo;
 import org.elasticsearch.indices.IndicesModule;
 import org.elasticsearch.indices.analysis.AnalysisModule;
 import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
@@ -623,13 +622,13 @@ public abstract class AbstractBuilderTestCase extends ESTestCase {
         }
 
         CoordinatorRewriteContext createCoordinatorContext(DateFieldMapper.DateFieldType dateFieldType, long min, long max) {
-            DateFieldRangeInfo timestampFieldInfo = new DateFieldRangeInfo(
-                dateFieldType,
+            return new CoordinatorRewriteContext(
+                parserConfiguration,
+                this.client,
+                () -> nowInMillis,
                 IndexLongFieldRange.NO_SHARDS.extendWithShardRange(0, 1, ShardLongFieldRange.of(min, max)),
-                dateFieldType,
-                IndexLongFieldRange.NO_SHARDS.extendWithShardRange(0, 1, ShardLongFieldRange.of(min, max))
+                dateFieldType
             );
-            return new CoordinatorRewriteContext(parserConfiguration, this.client, () -> nowInMillis, timestampFieldInfo);
         }
 
         DataRewriteContext createDataContext() {

+ 15 - 148
x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexIT.java

@@ -30,7 +30,6 @@ import org.elasticsearch.index.Index;
 import org.elasticsearch.index.mapper.DateFieldMapper;
 import org.elasticsearch.index.query.RangeQueryBuilder;
 import org.elasticsearch.index.shard.IndexLongFieldRange;
-import org.elasticsearch.indices.DateFieldRangeInfo;
 import org.elasticsearch.indices.IndicesService;
 import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.protocol.xpack.frozen.FreezeRequest;
@@ -45,7 +44,6 @@ import java.io.IOException;
 import java.time.Instant;
 import java.util.Collection;
 import java.util.List;
-import java.util.Map;
 import java.util.stream.Collectors;
 
 import static org.elasticsearch.cluster.metadata.IndexMetadata.INDEX_ROUTING_EXCLUDE_GROUP_SETTING;
@@ -78,15 +76,8 @@ public class FrozenIndexIT extends ESIntegTestCase {
 
         createIndex("index", 1, 1);
 
-        String timestampVal = "2010-01-06T02:03:04.567Z";
-        String eventIngestedVal = "2010-01-06T02:03:05.567Z";  // one second later
-
-        final DocWriteResponse indexResponse = prepareIndex("index").setSource(
-            DataStream.TIMESTAMP_FIELD_NAME,
-            timestampVal,
-            IndexMetadata.EVENT_INGESTED_FIELD_NAME,
-            eventIngestedVal
-        ).get();
+        final DocWriteResponse indexResponse = prepareIndex("index").setSource(DataStream.TIMESTAMP_FIELD_NAME, "2010-01-06T02:03:04.567Z")
+            .get();
 
         ensureGreen("index");
 
@@ -126,23 +117,13 @@ public class FrozenIndexIT extends ESIntegTestCase {
         assertThat(timestampFieldRange, not(sameInstance(IndexLongFieldRange.UNKNOWN)));
         assertThat(timestampFieldRange, not(sameInstance(IndexLongFieldRange.EMPTY)));
         assertTrue(timestampFieldRange.isComplete());
-        assertThat(timestampFieldRange.getMin(), equalTo(Instant.parse(timestampVal).toEpochMilli()));
-        assertThat(timestampFieldRange.getMax(), equalTo(Instant.parse(timestampVal).toEpochMilli()));
+        assertThat(timestampFieldRange.getMin(), equalTo(Instant.parse("2010-01-06T02:03:04.567Z").toEpochMilli()));
+        assertThat(timestampFieldRange.getMax(), equalTo(Instant.parse("2010-01-06T02:03:04.567Z").toEpochMilli()));
 
-        IndexLongFieldRange eventIngestedFieldRange = clusterAdmin().prepareState()
-            .get()
-            .getState()
-            .metadata()
-            .index("index")
-            .getEventIngestedRange();
-        assertThat(eventIngestedFieldRange, not(sameInstance(IndexLongFieldRange.UNKNOWN)));
-        assertThat(eventIngestedFieldRange, not(sameInstance(IndexLongFieldRange.EMPTY)));
-        assertTrue(eventIngestedFieldRange.isComplete());
-        assertThat(eventIngestedFieldRange.getMin(), equalTo(Instant.parse(eventIngestedVal).toEpochMilli()));
-        assertThat(eventIngestedFieldRange.getMax(), equalTo(Instant.parse(eventIngestedVal).toEpochMilli()));
+        assertThat(indexMetadata.getEventIngestedRange(), sameInstance(IndexLongFieldRange.UNKNOWN));
     }
 
-    public void testTimestampAndEventIngestedFieldTypeExposedByAllIndicesServices() throws Exception {
+    public void testTimestampFieldTypeExposedByAllIndicesServices() throws Exception {
         internalCluster().startNodes(between(2, 4));
 
         final String locale;
@@ -200,11 +181,11 @@ public class FrozenIndexIT extends ESIntegTestCase {
 
         ensureGreen("index");
         if (randomBoolean()) {
-            prepareIndex("index").setSource(DataStream.TIMESTAMP_FIELD_NAME, date, IndexMetadata.EVENT_INGESTED_FIELD_NAME, date).get();
+            prepareIndex("index").setSource(DataStream.TIMESTAMP_FIELD_NAME, date).get();
         }
 
         for (final IndicesService indicesService : internalCluster().getInstances(IndicesService.class)) {
-            assertNull(indicesService.getTimestampFieldTypeInfo(index));
+            assertNull(indicesService.getTimestampFieldType(index));
         }
 
         assertAcked(
@@ -212,129 +193,15 @@ public class FrozenIndexIT extends ESIntegTestCase {
         );
         ensureGreen("index");
         for (final IndicesService indicesService : internalCluster().getInstances(IndicesService.class)) {
-            final PlainActionFuture<Map<String, DateFieldMapper.DateFieldType>> future = new PlainActionFuture<>();
+            final PlainActionFuture<DateFieldMapper.DateFieldType> timestampFieldTypeFuture = new PlainActionFuture<>();
             assertBusy(() -> {
-                DateFieldRangeInfo timestampsFieldTypeInfo = indicesService.getTimestampFieldTypeInfo(index);
-                DateFieldMapper.DateFieldType timestampFieldType = timestampsFieldTypeInfo.getTimestampFieldType();
-                DateFieldMapper.DateFieldType eventIngestedFieldType = timestampsFieldTypeInfo.getEventIngestedFieldType();
-                assertNotNull(eventIngestedFieldType);
+                final DateFieldMapper.DateFieldType timestampFieldType = indicesService.getTimestampFieldType(index);
                 assertNotNull(timestampFieldType);
-                future.onResponse(
-                    Map.of(
-                        DataStream.TIMESTAMP_FIELD_NAME,
-                        timestampFieldType,
-                        IndexMetadata.EVENT_INGESTED_FIELD_NAME,
-                        eventIngestedFieldType
-                    )
-                );
-            });
-            assertTrue(future.isDone());
-            assertThat(future.get().get(DataStream.TIMESTAMP_FIELD_NAME).dateTimeFormatter().locale().toString(), equalTo(locale));
-            assertThat(future.get().get(DataStream.TIMESTAMP_FIELD_NAME).dateTimeFormatter().parseMillis(date), equalTo(1580817683000L));
-            assertThat(future.get().get(IndexMetadata.EVENT_INGESTED_FIELD_NAME).dateTimeFormatter().locale().toString(), equalTo(locale));
-            assertThat(
-                future.get().get(IndexMetadata.EVENT_INGESTED_FIELD_NAME).dateTimeFormatter().parseMillis(date),
-                equalTo(1580817683000L)
-            );
-        }
-
-        assertAcked(
-            client().execute(
-                FreezeIndexAction.INSTANCE,
-                new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, "index").setFreeze(false)
-            ).actionGet()
-        );
-        ensureGreen("index");
-        for (final IndicesService indicesService : internalCluster().getInstances(IndicesService.class)) {
-            assertNull(indicesService.getTimestampFieldTypeInfo(index));
-        }
-    }
-
-    public void testTimestampOrEventIngestedFieldTypeExposedByAllIndicesServices() throws Exception {
-        internalCluster().startNodes(between(2, 4));
-
-        final String locale;
-        final String date;
-
-        switch (between(1, 3)) {
-            case 1 -> {
-                locale = "";
-                date = "04 Feb 2020 12:01:23Z";
-            }
-            case 2 -> {
-                locale = "en_GB";
-                date = "04 Feb 2020 12:01:23Z";
-            }
-            case 3 -> {
-                locale = "fr_FR";
-                date = "04 févr. 2020 12:01:23Z";
-            }
-            default -> throw new AssertionError("impossible");
-        }
-
-        String timeField = randomFrom(IndexMetadata.EVENT_INGESTED_FIELD_NAME, DataStream.TIMESTAMP_FIELD_NAME);
-        assertAcked(
-            prepareCreate("index").setSettings(
-                Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)
-            )
-                .setMapping(
-                    jsonBuilder().startObject()
-                        .startObject("_doc")
-                        .startObject("properties")
-                        .startObject(timeField)
-                        .field("type", "date")
-                        .field("format", "dd LLL yyyy HH:mm:ssX")
-                        .field("locale", locale)
-                        .endObject()
-                        .endObject()
-                        .endObject()
-                        .endObject()
-                )
-        );
-
-        final Index index = clusterAdmin().prepareState()
-            .clear()
-            .setIndices("index")
-            .setMetadata(true)
-            .get()
-            .getState()
-            .metadata()
-            .index("index")
-            .getIndex();
-
-        ensureGreen("index");
-        if (randomBoolean()) {
-            prepareIndex("index").setSource(timeField, date).get();
-        }
-
-        for (final IndicesService indicesService : internalCluster().getInstances(IndicesService.class)) {
-            assertNull(indicesService.getTimestampFieldTypeInfo(index));
-        }
-
-        assertAcked(
-            client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, "index")).actionGet()
-        );
-        ensureGreen("index");
-        for (final IndicesService indicesService : internalCluster().getInstances(IndicesService.class)) {
-            // final PlainActionFuture<DateFieldMapper.DateFieldType> timestampFieldTypeFuture = new PlainActionFuture<>();
-            final PlainActionFuture<Map<String, DateFieldMapper.DateFieldType>> future = new PlainActionFuture<>();
-            assertBusy(() -> {
-                DateFieldRangeInfo timestampsFieldTypeInfo = indicesService.getTimestampFieldTypeInfo(index);
-                DateFieldMapper.DateFieldType timestampFieldType = timestampsFieldTypeInfo.getTimestampFieldType();
-                DateFieldMapper.DateFieldType eventIngestedFieldType = timestampsFieldTypeInfo.getEventIngestedFieldType();
-                if (timeField == DataStream.TIMESTAMP_FIELD_NAME) {
-                    assertNotNull(timestampFieldType);
-                    assertNull(eventIngestedFieldType);
-                    future.onResponse(Map.of(timeField, timestampFieldType));
-                } else {
-                    assertNull(timestampFieldType);
-                    assertNotNull(eventIngestedFieldType);
-                    future.onResponse(Map.of(timeField, eventIngestedFieldType));
-                }
+                timestampFieldTypeFuture.onResponse(timestampFieldType);
             });
-            assertTrue(future.isDone());
-            assertThat(future.get().get(timeField).dateTimeFormatter().locale().toString(), equalTo(locale));
-            assertThat(future.get().get(timeField).dateTimeFormatter().parseMillis(date), equalTo(1580817683000L));
+            assertTrue(timestampFieldTypeFuture.isDone());
+            assertThat(timestampFieldTypeFuture.get().dateTimeFormatter().locale().toString(), equalTo(locale));
+            assertThat(timestampFieldTypeFuture.get().dateTimeFormatter().parseMillis(date), equalTo(1580817683000L));
         }
 
         assertAcked(
@@ -345,7 +212,7 @@ public class FrozenIndexIT extends ESIntegTestCase {
         );
         ensureGreen("index");
         for (final IndicesService indicesService : internalCluster().getInstances(IndicesService.class)) {
-            assertNull(indicesService.getTimestampFieldTypeInfo(index));
+            assertNull(indicesService.getTimestampFieldType(index));
         }
     }
 

+ 39 - 370
x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsCanMatchOnCoordinatorIntegTests.java

@@ -29,7 +29,6 @@ import org.elasticsearch.index.mapper.DateFieldMapper;
 import org.elasticsearch.index.query.QueryBuilders;
 import org.elasticsearch.index.query.RangeQueryBuilder;
 import org.elasticsearch.index.shard.IndexLongFieldRange;
-import org.elasticsearch.indices.DateFieldRangeInfo;
 import org.elasticsearch.indices.IndicesService;
 import org.elasticsearch.indices.recovery.RecoveryState;
 import org.elasticsearch.plugins.Plugin;
@@ -101,11 +100,11 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz
 
         final String indexOutsideSearchRange = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
         final int indexOutsideSearchRangeShardCount = randomIntBetween(1, 3);
-        createIndexWithTimestampAndEventIngested(indexOutsideSearchRange, indexOutsideSearchRangeShardCount, Settings.EMPTY);
+        createIndexWithTimestamp(indexOutsideSearchRange, indexOutsideSearchRangeShardCount, Settings.EMPTY);
 
         final String indexWithinSearchRange = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
         final int indexWithinSearchRangeShardCount = randomIntBetween(1, 3);
-        createIndexWithTimestampAndEventIngested(
+        createIndexWithTimestamp(
             indexWithinSearchRange,
             indexWithinSearchRangeShardCount,
             Settings.builder()
@@ -118,10 +117,11 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz
         // Either add data outside of the range, or documents that don't have timestamp data
         final boolean indexDataWithTimestamp = randomBoolean();
         // Add enough documents to have non-metadata segment files in all shards,
-        // otherwise the mount operation might go through as the read won't be blocked
+        // otherwise the mount operation might go through as the read won't be
+        // blocked
         final int numberOfDocsInIndexOutsideSearchRange = between(350, 1000);
         if (indexDataWithTimestamp) {
-            indexDocumentsWithTimestampAndEventIngestedDates(
+            indexDocumentsWithTimestampWithinDate(
                 indexOutsideSearchRange,
                 numberOfDocsInIndexOutsideSearchRange,
                 TIMESTAMP_TEMPLATE_OUTSIDE_RANGE
@@ -132,7 +132,7 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz
 
         // Index enough documents to ensure that all shards have at least some documents
         int numDocsWithinRange = between(100, 1000);
-        indexDocumentsWithTimestampAndEventIngestedDates(indexWithinSearchRange, numDocsWithinRange, TIMESTAMP_TEMPLATE_WITHIN_RANGE);
+        indexDocumentsWithTimestampWithinDate(indexWithinSearchRange, numDocsWithinRange, TIMESTAMP_TEMPLATE_WITHIN_RANGE);
 
         final String repositoryName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
         createRepository(repositoryName, "mock");
@@ -166,10 +166,9 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz
 
         final IndexMetadata indexMetadata = getIndexMetadata(searchableSnapshotIndexOutsideSearchRange);
         assertThat(indexMetadata.getTimestampRange(), equalTo(IndexLongFieldRange.NO_SHARDS));
-        assertThat(indexMetadata.getEventIngestedRange(), equalTo(IndexLongFieldRange.NO_SHARDS));
 
-        DateFieldRangeInfo timestampFieldTypeInfo = indicesService.getTimestampFieldTypeInfo(indexMetadata.getIndex());
-        assertThat(timestampFieldTypeInfo, nullValue());
+        DateFieldMapper.DateFieldType timestampFieldType = indicesService.getTimestampFieldType(indexMetadata.getIndex());
+        assertThat(timestampFieldType, nullValue());
 
         final boolean includeIndexCoveringSearchRangeInSearchRequest = randomBoolean();
         List<String> indicesToSearch = new ArrayList<>();
@@ -177,9 +176,7 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz
             indicesToSearch.add(indexWithinSearchRange);
         }
         indicesToSearch.add(searchableSnapshotIndexOutsideSearchRange);
-
-        String timeField = randomFrom(IndexMetadata.EVENT_INGESTED_FIELD_NAME, DataStream.TIMESTAMP_FIELD_NAME);
-        RangeQueryBuilder rangeQuery = QueryBuilders.rangeQuery(timeField)
+        RangeQueryBuilder rangeQuery = QueryBuilders.rangeQuery(DataStream.TIMESTAMP_FIELD_NAME)
             .from("2020-11-28T00:00:00.000000000Z", true)
             .to("2020-11-29T00:00:00.000000000Z");
 
@@ -253,44 +250,20 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz
         ensureGreen(searchableSnapshotIndexOutsideSearchRange);
 
         final IndexMetadata updatedIndexMetadata = getIndexMetadata(searchableSnapshotIndexOutsideSearchRange);
-
-        // check that @timestamp and 'event.ingested' are now in cluster state
         final IndexLongFieldRange updatedTimestampMillisRange = updatedIndexMetadata.getTimestampRange();
+        final DateFieldMapper.DateFieldType dateFieldType = indicesService.getTimestampFieldType(updatedIndexMetadata.getIndex());
+        assertThat(dateFieldType, notNullValue());
+        final DateFieldMapper.Resolution resolution = dateFieldType.resolution();
         assertThat(updatedTimestampMillisRange.isComplete(), equalTo(true));
-        final IndexLongFieldRange updatedEventIngestedRange = updatedIndexMetadata.getEventIngestedRange();
-        assertThat(updatedEventIngestedRange.isComplete(), equalTo(true));
-
-        timestampFieldTypeInfo = indicesService.getTimestampFieldTypeInfo(updatedIndexMetadata.getIndex());
-        final DateFieldMapper.DateFieldType timestampDataFieldType = timestampFieldTypeInfo.getTimestampFieldType();
-        assertThat(timestampDataFieldType, notNullValue());
-        final DateFieldMapper.DateFieldType eventIngestedDataFieldType = timestampFieldTypeInfo.getEventIngestedFieldType();
-        assertThat(eventIngestedDataFieldType, notNullValue());
-
-        final DateFieldMapper.Resolution timestampResolution = timestampDataFieldType.resolution();
-        final DateFieldMapper.Resolution eventIngestedResolution = eventIngestedDataFieldType.resolution();
         if (indexDataWithTimestamp) {
             assertThat(updatedTimestampMillisRange, not(sameInstance(IndexLongFieldRange.EMPTY)));
             assertThat(
                 updatedTimestampMillisRange.getMin(),
-                greaterThanOrEqualTo(timestampResolution.convert(Instant.parse("2020-11-26T00:00:00Z")))
-            );
-            assertThat(
-                updatedTimestampMillisRange.getMax(),
-                lessThanOrEqualTo(timestampResolution.convert(Instant.parse("2020-11-27T00:00:00Z")))
-            );
-
-            assertThat(updatedEventIngestedRange, not(sameInstance(IndexLongFieldRange.EMPTY)));
-            assertThat(
-                updatedEventIngestedRange.getMin(),
-                greaterThanOrEqualTo(eventIngestedResolution.convert(Instant.parse("2020-11-26T00:00:00Z")))
-            );
-            assertThat(
-                updatedEventIngestedRange.getMax(),
-                lessThanOrEqualTo(eventIngestedResolution.convert(Instant.parse("2020-11-27T00:00:00Z")))
+                greaterThanOrEqualTo(resolution.convert(Instant.parse("2020-11-26T00:00:00Z")))
             );
+            assertThat(updatedTimestampMillisRange.getMax(), lessThanOrEqualTo(resolution.convert(Instant.parse("2020-11-27T00:00:00Z"))));
         } else {
             assertThat(updatedTimestampMillisRange, sameInstance(IndexLongFieldRange.EMPTY));
-            assertThat(updatedEventIngestedRange, sameInstance(IndexLongFieldRange.EMPTY));
         }
 
         // Stop the node holding the searchable snapshots, and since we defined
@@ -410,171 +383,6 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz
         }
     }
 
-    /**
-     * Test shard skipping when only 'event.ingested' is in the index and cluster state.
-     */
-    public void testEventIngestedRangeInSearchAgainstSearchableSnapshotShards() throws Exception {
-        internalCluster().startMasterOnlyNode();
-        internalCluster().startCoordinatingOnlyNode(Settings.EMPTY);
-        final String dataNodeHoldingRegularIndex = internalCluster().startDataOnlyNode();
-        final String dataNodeHoldingSearchableSnapshot = internalCluster().startDataOnlyNode();
-        final IndicesService indicesService = internalCluster().getInstance(IndicesService.class, dataNodeHoldingSearchableSnapshot);
-
-        final String indexOutsideSearchRange = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
-        final int indexOutsideSearchRangeShardCount = randomIntBetween(1, 3);
-
-        final String timestampField = IndexMetadata.EVENT_INGESTED_FIELD_NAME;
-
-        createIndexWithOnlyOneTimestampField(timestampField, indexOutsideSearchRange, indexOutsideSearchRangeShardCount, Settings.EMPTY);
-
-        final String indexWithinSearchRange = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
-        final int indexWithinSearchRangeShardCount = randomIntBetween(1, 3);
-        createIndexWithOnlyOneTimestampField(
-            timestampField,
-            indexWithinSearchRange,
-            indexWithinSearchRangeShardCount,
-            Settings.builder()
-                .put(INDEX_ROUTING_REQUIRE_GROUP_SETTING.getConcreteSettingForNamespace("_name").getKey(), dataNodeHoldingRegularIndex)
-                .build()
-        );
-
-        final int totalShards = indexOutsideSearchRangeShardCount + indexWithinSearchRangeShardCount;
-
-        // Add enough documents to have non-metadata segment files in all shards,
-        // otherwise the mount operation might go through as the read won't be blocked
-        final int numberOfDocsInIndexOutsideSearchRange = between(350, 1000);
-
-        indexDocumentsWithOnlyOneTimestampField(
-            timestampField,
-            indexOutsideSearchRange,
-            numberOfDocsInIndexOutsideSearchRange,
-            TIMESTAMP_TEMPLATE_OUTSIDE_RANGE
-        );
-
-        // Index enough documents to ensure that all shards have at least some documents
-        int numDocsWithinRange = between(100, 1000);
-        indexDocumentsWithOnlyOneTimestampField(
-            timestampField,
-            indexWithinSearchRange,
-            numDocsWithinRange,
-            TIMESTAMP_TEMPLATE_WITHIN_RANGE
-        );
-
-        final String repositoryName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
-        createRepository(repositoryName, "mock");
-
-        final SnapshotId snapshotId = createSnapshot(repositoryName, "snapshot-1", List.of(indexOutsideSearchRange)).snapshotId();
-        assertAcked(indicesAdmin().prepareDelete(indexOutsideSearchRange));
-
-        final String searchableSnapshotIndexOutsideSearchRange = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
-
-        // Block the repository for the node holding the searchable snapshot shards
-        // to delay its restore
-        blockDataNode(repositoryName, dataNodeHoldingSearchableSnapshot);
-
-        // Force the searchable snapshot to be allocated in a particular node
-        Settings restoredIndexSettings = Settings.builder()
-            .put(INDEX_ROUTING_REQUIRE_GROUP_SETTING.getConcreteSettingForNamespace("_name").getKey(), dataNodeHoldingSearchableSnapshot)
-            .build();
-
-        final MountSearchableSnapshotRequest mountRequest = new MountSearchableSnapshotRequest(
-            TEST_REQUEST_TIMEOUT,
-            searchableSnapshotIndexOutsideSearchRange,
-            repositoryName,
-            snapshotId.getName(),
-            indexOutsideSearchRange,
-            restoredIndexSettings,
-            Strings.EMPTY_ARRAY,
-            false,
-            randomFrom(MountSearchableSnapshotRequest.Storage.values())
-        );
-        client().execute(MountSearchableSnapshotAction.INSTANCE, mountRequest).actionGet();
-
-        final IndexMetadata indexMetadata = getIndexMetadata(searchableSnapshotIndexOutsideSearchRange);
-        assertThat(indexMetadata.getTimestampRange(), equalTo(IndexLongFieldRange.NO_SHARDS));
-        assertThat(indexMetadata.getEventIngestedRange(), equalTo(IndexLongFieldRange.NO_SHARDS));
-
-        // Allow the searchable snapshots to be finally mounted
-        unblockNode(repositoryName, dataNodeHoldingSearchableSnapshot);
-        waitUntilRecoveryIsDone(searchableSnapshotIndexOutsideSearchRange);
-        ensureGreen(searchableSnapshotIndexOutsideSearchRange);
-
-        IndexMetadata updatedIndexMetadata = getIndexMetadata(searchableSnapshotIndexOutsideSearchRange);
-        IndexLongFieldRange updatedTimestampMillisRange = updatedIndexMetadata.getTimestampRange();
-        IndexLongFieldRange updatedEventIngestedMillisRange = updatedIndexMetadata.getEventIngestedRange();
-
-        // @timestamp range should be null since it was not included in the index or indexed docs
-        assertThat(updatedTimestampMillisRange, equalTo(IndexLongFieldRange.UNKNOWN));
-        assertThat(updatedEventIngestedMillisRange, not(equalTo(IndexLongFieldRange.UNKNOWN)));
-
-        DateFieldRangeInfo timestampFieldTypeInfo = indicesService.getTimestampFieldTypeInfo(updatedIndexMetadata.getIndex());
-
-        DateFieldMapper.DateFieldType timestampDataFieldType = timestampFieldTypeInfo.getTimestampFieldType();
-        assertThat(timestampDataFieldType, nullValue());
-
-        DateFieldMapper.DateFieldType eventIngestedFieldType = timestampFieldTypeInfo.getEventIngestedFieldType();
-        assertThat(eventIngestedFieldType, notNullValue());
-
-        DateFieldMapper.Resolution eventIngestedResolution = eventIngestedFieldType.resolution();
-        assertThat(updatedEventIngestedMillisRange.isComplete(), equalTo(true));
-        assertThat(
-            updatedEventIngestedMillisRange.getMin(),
-            greaterThanOrEqualTo(eventIngestedResolution.convert(Instant.parse("2020-11-26T00:00:00Z")))
-        );
-        assertThat(
-            updatedEventIngestedMillisRange.getMax(),
-            lessThanOrEqualTo(eventIngestedResolution.convert(Instant.parse("2020-11-27T00:00:00Z")))
-        );
-
-        // now do a search against event.ingested
-        List<String> indicesToSearch = new ArrayList<>();
-        indicesToSearch.add(indexWithinSearchRange);
-        indicesToSearch.add(searchableSnapshotIndexOutsideSearchRange);
-
-        {
-            RangeQueryBuilder rangeQuery = QueryBuilders.rangeQuery(timestampField)
-                .from("2020-11-28T00:00:00.000000000Z", true)
-                .to("2020-11-29T00:00:00.000000000Z");
-
-            SearchRequest request = new SearchRequest().indices(indicesToSearch.toArray(new String[0]))
-                .source(new SearchSourceBuilder().query(rangeQuery));
-
-            assertResponse(client().search(request), searchResponse -> {
-                // All the regular index searches succeeded
-                assertThat(searchResponse.getSuccessfulShards(), equalTo(totalShards));
-                assertThat(searchResponse.getFailedShards(), equalTo(0));
-                // All the searchable snapshots shards were skipped
-                assertThat(searchResponse.getSkippedShards(), equalTo(indexOutsideSearchRangeShardCount));
-                assertThat(searchResponse.getTotalShards(), equalTo(totalShards));
-            });
-
-            SearchShardAPIResult searchShardResult = doSearchShardAPIQuery(indicesToSearch, rangeQuery, true, totalShards);
-            assertThat(searchShardResult.skipped().size(), equalTo(indexOutsideSearchRangeShardCount));
-            assertThat(searchShardResult.notSkipped().size(), equalTo(indexWithinSearchRangeShardCount));
-        }
-
-        // query a range that covers both indexes - all shards should be searched, none skipped
-        {
-            RangeQueryBuilder rangeQuery = QueryBuilders.rangeQuery(timestampField)
-                .from("2019-11-28T00:00:00.000000000Z", true)
-                .to("2021-11-29T00:00:00.000000000Z");
-
-            SearchRequest request = new SearchRequest().indices(indicesToSearch.toArray(new String[0]))
-                .source(new SearchSourceBuilder().query(rangeQuery));
-
-            assertResponse(client().search(request), searchResponse -> {
-                assertThat(searchResponse.getSuccessfulShards(), equalTo(totalShards));
-                assertThat(searchResponse.getFailedShards(), equalTo(0));
-                assertThat(searchResponse.getSkippedShards(), equalTo(0));
-                assertThat(searchResponse.getTotalShards(), equalTo(totalShards));
-            });
-
-            SearchShardAPIResult searchShardResult = doSearchShardAPIQuery(indicesToSearch, rangeQuery, true, totalShards);
-            assertThat(searchShardResult.skipped().size(), equalTo(0));
-            assertThat(searchShardResult.notSkipped().size(), equalTo(totalShards));
-        }
-    }
-
     /**
      * Can match against searchable snapshots is tested via both the Search API and the SearchShards (transport-only) API.
      * The latter is a way to do only a can-match rather than all search phases.
@@ -588,7 +396,7 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz
 
         final String indexOutsideSearchRange = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
         final int indexOutsideSearchRangeShardCount = randomIntBetween(1, 3);
-        createIndexWithTimestampAndEventIngested(
+        createIndexWithTimestamp(
             indexOutsideSearchRange,
             indexOutsideSearchRangeShardCount,
             Settings.builder()
@@ -596,7 +404,7 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz
                 .build()
         );
 
-        indexDocumentsWithTimestampAndEventIngestedDates(indexOutsideSearchRange, between(1, 1000), TIMESTAMP_TEMPLATE_OUTSIDE_RANGE);
+        indexDocumentsWithTimestampWithinDate(indexOutsideSearchRange, between(1, 1000), TIMESTAMP_TEMPLATE_OUTSIDE_RANGE);
 
         final String repositoryName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
         createRepository(repositoryName, "mock");
@@ -630,14 +438,11 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz
 
         final IndexMetadata indexMetadata = getIndexMetadata(searchableSnapshotIndexOutsideSearchRange);
         assertThat(indexMetadata.getTimestampRange(), equalTo(IndexLongFieldRange.NO_SHARDS));
-        assertThat(indexMetadata.getEventIngestedRange(), equalTo(IndexLongFieldRange.NO_SHARDS));
 
-        DateFieldRangeInfo timestampFieldTypeInfo = indicesService.getTimestampFieldTypeInfo(indexMetadata.getIndex());
-        assertThat(timestampFieldTypeInfo, nullValue());
+        DateFieldMapper.DateFieldType timestampFieldType = indicesService.getTimestampFieldType(indexMetadata.getIndex());
+        assertThat(timestampFieldType, nullValue());
 
-        final String timestampField = randomFrom(DataStream.TIMESTAMP_FIELD_NAME, IndexMetadata.EVENT_INGESTED_FIELD_NAME);
-
-        RangeQueryBuilder rangeQuery = QueryBuilders.rangeQuery(timestampField)
+        RangeQueryBuilder rangeQuery = QueryBuilders.rangeQuery(DataStream.TIMESTAMP_FIELD_NAME)
             .from("2020-11-28T00:00:00.000000000Z", true)
             .to("2020-11-29T00:00:00.000000000Z");
 
@@ -695,29 +500,14 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz
         ensureGreen(searchableSnapshotIndexOutsideSearchRange);
 
         final IndexMetadata updatedIndexMetadata = getIndexMetadata(searchableSnapshotIndexOutsideSearchRange);
-        timestampFieldTypeInfo = indicesService.getTimestampFieldTypeInfo(updatedIndexMetadata.getIndex());
-        assertThat(timestampFieldTypeInfo, notNullValue());
-
-        final IndexLongFieldRange updatedTimestampRange = updatedIndexMetadata.getTimestampRange();
-        DateFieldMapper.Resolution tsResolution = timestampFieldTypeInfo.getTimestampFieldType().resolution();
-        ;
-        assertThat(updatedTimestampRange.isComplete(), equalTo(true));
-        assertThat(updatedTimestampRange, not(sameInstance(IndexLongFieldRange.EMPTY)));
-        assertThat(updatedTimestampRange.getMin(), greaterThanOrEqualTo(tsResolution.convert(Instant.parse("2020-11-26T00:00:00Z"))));
-        assertThat(updatedTimestampRange.getMax(), lessThanOrEqualTo(tsResolution.convert(Instant.parse("2020-11-27T00:00:00Z"))));
-
-        final IndexLongFieldRange updatedEventIngestedRange = updatedIndexMetadata.getEventIngestedRange();
-        DateFieldMapper.Resolution eventIngestedResolution = timestampFieldTypeInfo.getEventIngestedFieldType().resolution();
-        assertThat(updatedEventIngestedRange.isComplete(), equalTo(true));
-        assertThat(updatedEventIngestedRange, not(sameInstance(IndexLongFieldRange.EMPTY)));
-        assertThat(
-            updatedEventIngestedRange.getMin(),
-            greaterThanOrEqualTo(eventIngestedResolution.convert(Instant.parse("2020-11-26T00:00:00Z")))
-        );
-        assertThat(
-            updatedEventIngestedRange.getMax(),
-            lessThanOrEqualTo(eventIngestedResolution.convert(Instant.parse("2020-11-27T00:00:00Z")))
-        );
+        final IndexLongFieldRange updatedTimestampMillisRange = updatedIndexMetadata.getTimestampRange();
+        final DateFieldMapper.DateFieldType dateFieldType = indicesService.getTimestampFieldType(updatedIndexMetadata.getIndex());
+        assertThat(dateFieldType, notNullValue());
+        final DateFieldMapper.Resolution resolution = dateFieldType.resolution();
+        assertThat(updatedTimestampMillisRange.isComplete(), equalTo(true));
+        assertThat(updatedTimestampMillisRange, not(sameInstance(IndexLongFieldRange.EMPTY)));
+        assertThat(updatedTimestampMillisRange.getMin(), greaterThanOrEqualTo(resolution.convert(Instant.parse("2020-11-26T00:00:00Z"))));
+        assertThat(updatedTimestampMillisRange.getMax(), lessThanOrEqualTo(resolution.convert(Instant.parse("2020-11-27T00:00:00Z"))));
 
         // Stop the node holding the searchable snapshots, and since we defined
         // the index allocation criteria to require the searchable snapshot
@@ -789,7 +579,7 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz
 
         final String indexWithinSearchRange = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
         final int indexWithinSearchRangeShardCount = randomIntBetween(1, 3);
-        createIndexWithTimestampAndEventIngested(
+        createIndexWithTimestamp(
             indexWithinSearchRange,
             indexWithinSearchRangeShardCount,
             Settings.builder()
@@ -797,7 +587,7 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz
                 .build()
         );
 
-        indexDocumentsWithTimestampAndEventIngestedDates(indexWithinSearchRange, between(1, 1000), TIMESTAMP_TEMPLATE_WITHIN_RANGE);
+        indexDocumentsWithTimestampWithinDate(indexWithinSearchRange, between(1, 1000), TIMESTAMP_TEMPLATE_WITHIN_RANGE);
 
         final String repositoryName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
         createRepository(repositoryName, "mock");
@@ -831,13 +621,11 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz
 
         final IndexMetadata indexMetadata = getIndexMetadata(searchableSnapshotIndexWithinSearchRange);
         assertThat(indexMetadata.getTimestampRange(), equalTo(IndexLongFieldRange.NO_SHARDS));
-        assertThat(indexMetadata.getEventIngestedRange(), equalTo(IndexLongFieldRange.NO_SHARDS));
 
-        DateFieldRangeInfo timestampFieldTypeInfo = indicesService.getTimestampFieldTypeInfo(indexMetadata.getIndex());
-        assertThat(timestampFieldTypeInfo, nullValue());
+        DateFieldMapper.DateFieldType timestampFieldType = indicesService.getTimestampFieldType(indexMetadata.getIndex());
+        assertThat(timestampFieldType, nullValue());
 
-        String timeField = randomFrom(IndexMetadata.EVENT_INGESTED_FIELD_NAME, DataStream.TIMESTAMP_FIELD_NAME);
-        RangeQueryBuilder rangeQuery = QueryBuilders.rangeQuery(timeField)
+        RangeQueryBuilder rangeQuery = QueryBuilders.rangeQuery(DataStream.TIMESTAMP_FIELD_NAME)
             .from("2020-11-28T00:00:00.000000000Z", true)
             .to("2020-11-29T00:00:00.000000000Z");
 
@@ -892,32 +680,13 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz
 
         final IndexMetadata updatedIndexMetadata = getIndexMetadata(searchableSnapshotIndexWithinSearchRange);
         final IndexLongFieldRange updatedTimestampMillisRange = updatedIndexMetadata.getTimestampRange();
-        timestampFieldTypeInfo = indicesService.getTimestampFieldTypeInfo(updatedIndexMetadata.getIndex());
-        assertThat(timestampFieldTypeInfo, notNullValue());
-        final DateFieldMapper.Resolution timestampResolution = timestampFieldTypeInfo.getTimestampFieldType().resolution();
+        final DateFieldMapper.DateFieldType dateFieldType = indicesService.getTimestampFieldType(updatedIndexMetadata.getIndex());
+        assertThat(dateFieldType, notNullValue());
+        final DateFieldMapper.Resolution resolution = dateFieldType.resolution();
         assertThat(updatedTimestampMillisRange.isComplete(), equalTo(true));
         assertThat(updatedTimestampMillisRange, not(sameInstance(IndexLongFieldRange.EMPTY)));
-        assertThat(
-            updatedTimestampMillisRange.getMin(),
-            greaterThanOrEqualTo(timestampResolution.convert(Instant.parse("2020-11-28T00:00:00Z")))
-        );
-        assertThat(
-            updatedTimestampMillisRange.getMax(),
-            lessThanOrEqualTo(timestampResolution.convert(Instant.parse("2020-11-29T00:00:00Z")))
-        );
-
-        final IndexLongFieldRange updatedEventIngestedMillisRange = updatedIndexMetadata.getEventIngestedRange();
-        final DateFieldMapper.Resolution eventIngestedResolution = timestampFieldTypeInfo.getEventIngestedFieldType().resolution();
-        assertThat(updatedEventIngestedMillisRange.isComplete(), equalTo(true));
-        assertThat(updatedEventIngestedMillisRange, not(sameInstance(IndexLongFieldRange.EMPTY)));
-        assertThat(
-            updatedEventIngestedMillisRange.getMin(),
-            greaterThanOrEqualTo(eventIngestedResolution.convert(Instant.parse("2020-11-28T00:00:00Z")))
-        );
-        assertThat(
-            updatedEventIngestedMillisRange.getMax(),
-            lessThanOrEqualTo(eventIngestedResolution.convert(Instant.parse("2020-11-29T00:00:00Z")))
-        );
+        assertThat(updatedTimestampMillisRange.getMin(), greaterThanOrEqualTo(resolution.convert(Instant.parse("2020-11-28T00:00:00Z"))));
+        assertThat(updatedTimestampMillisRange.getMax(), lessThanOrEqualTo(resolution.convert(Instant.parse("2020-11-29T00:00:00Z"))));
 
         // Stop the node holding the searchable snapshots, and since we defined
         // the index allocation criteria to require the searchable snapshot
@@ -955,24 +724,17 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz
         }
     }
 
-    private void createIndexWithTimestampAndEventIngested(String indexName, int numShards, Settings extraSettings) throws IOException {
+    private void createIndexWithTimestamp(String indexName, int numShards, Settings extraSettings) throws IOException {
         assertAcked(
             indicesAdmin().prepareCreate(indexName)
                 .setMapping(
                     XContentFactory.jsonBuilder()
                         .startObject()
                         .startObject("properties")
-
                         .startObject(DataStream.TIMESTAMP_FIELD_NAME)
                         .field("type", randomFrom("date", "date_nanos"))
                         .field("format", "strict_date_optional_time_nanos")
                         .endObject()
-
-                        .startObject(IndexMetadata.EVENT_INGESTED_FIELD_NAME)
-                        .field("type", randomFrom("date", "date_nanos"))
-                        .field("format", "strict_date_optional_time_nanos")
-                        .endObject()
-
                         .endObject()
                         .endObject()
                 )
@@ -981,70 +743,12 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz
         ensureGreen(indexName);
     }
 
-    private void createIndexWithOnlyOneTimestampField(String timestampField, String index, int numShards, Settings extraSettings)
-        throws IOException {
-        assertAcked(
-            indicesAdmin().prepareCreate(index)
-                .setMapping(
-                    XContentFactory.jsonBuilder()
-                        .startObject()
-                        .startObject("properties")
-
-                        .startObject(timestampField)
-                        .field("type", randomFrom("date", "date_nanos"))
-                        .field("format", "strict_date_optional_time_nanos")
-                        .endObject()
-
-                        .endObject()
-                        .endObject()
-                )
-                .setSettings(indexSettingsNoReplicas(numShards).put(INDEX_SOFT_DELETES_SETTING.getKey(), true).put(extraSettings))
-        );
-        ensureGreen(index);
-    }
-
-    private void indexDocumentsWithOnlyOneTimestampField(String timestampField, String index, int docCount, String timestampTemplate)
-        throws Exception {
-        final List<IndexRequestBuilder> indexRequestBuilders = new ArrayList<>();
-        for (int i = 0; i < docCount; i++) {
-            indexRequestBuilders.add(
-                prepareIndex(index).setSource(
-                    timestampField,
-                    String.format(
-                        Locale.ROOT,
-                        timestampTemplate,
-                        between(0, 23),
-                        between(0, 59),
-                        between(0, 59),
-                        randomLongBetween(0, 999999999L)
-                    )
-                )
-            );
-        }
-        indexRandom(true, false, indexRequestBuilders);
-
-        assertThat(indicesAdmin().prepareForceMerge(index).setOnlyExpungeDeletes(true).setFlush(true).get().getFailedShards(), equalTo(0));
-        refresh(index);
-        forceMerge();
-    }
-
-    private void indexDocumentsWithTimestampAndEventIngestedDates(String indexName, int docCount, String timestampTemplate)
-        throws Exception {
-
+    private void indexDocumentsWithTimestampWithinDate(String indexName, int docCount, String timestampTemplate) throws Exception {
         final List<IndexRequestBuilder> indexRequestBuilders = new ArrayList<>();
         for (int i = 0; i < docCount; i++) {
             indexRequestBuilders.add(
                 prepareIndex(indexName).setSource(
                     DataStream.TIMESTAMP_FIELD_NAME,
-                    String.format(
-                        Locale.ROOT,
-                        timestampTemplate,
-                        between(0, 23),
-                        between(0, 59),
-                        between(0, 59),
-                        randomLongBetween(0, 999999999L)
-                    ),
-                    IndexMetadata.EVENT_INGESTED_FIELD_NAME,
                     String.format(
                         Locale.ROOT,
                         timestampTemplate,
@@ -1085,39 +789,4 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz
     private void waitUntilAllShardsAreUnassigned(Index index) throws Exception {
         awaitClusterState(state -> state.getRoutingTable().index(index).allPrimaryShardsUnassigned());
     }
-
-    record SearchShardAPIResult(List<SearchShardsGroup> skipped, List<SearchShardsGroup> notSkipped) {}
-
-    private static SearchShardAPIResult doSearchShardAPIQuery(
-        List<String> indicesToSearch,
-        RangeQueryBuilder rangeQuery,
-        boolean allowPartialSearchResults,
-        int expectedTotalShards
-    ) {
-        SearchShardsRequest searchShardsRequest = new SearchShardsRequest(
-            indicesToSearch.toArray(new String[0]),
-            SearchRequest.DEFAULT_INDICES_OPTIONS,
-            rangeQuery,
-            null,
-            null,
-            allowPartialSearchResults,
-            null
-        );
-
-        SearchShardsResponse searchShardsResponse = client().execute(TransportSearchShardsAction.TYPE, searchShardsRequest).actionGet();
-        assertThat(searchShardsResponse.getGroups().size(), equalTo(expectedTotalShards));
-        List<List<SearchShardsGroup>> partitionedBySkipped = searchShardsResponse.getGroups()
-            .stream()
-            .collect(
-                Collectors.teeing(
-                    Collectors.filtering(g -> g.skipped(), Collectors.toList()),
-                    Collectors.filtering(g -> g.skipped() == false, Collectors.toList()),
-                    List::of
-                )
-            );
-
-        List<SearchShardsGroup> skipped = partitionedBySkipped.get(0);
-        List<SearchShardsGroup> notSkipped = partitionedBySkipped.get(1);
-        return new SearchShardAPIResult(skipped, notSkipped);
-    }
 }