Browse Source

TSDB: Cleaner trigger for tsdb boundary check (#81263)

This replaces the hard check for "are we in time series mode" with a
check for whether we have declared a time range for the index. It makes
me feel a little better not to have `if (mode == TIME_SERIES)` tests. I
just think they make it harder to reason about what's going on. Instead,
this code says "if there are settings declaring a time range for this
index we should enforce that range".
Nik Everett 3 years ago
parent
commit
1924ed77d9

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

@@ -75,7 +75,6 @@ public enum IndexMode {
             // non time-series indices must not have a TimeSeriesIdFieldMapper
             return null;
         }
-
     },
     TIME_SERIES {
         @Override

+ 10 - 27
server/src/main/java/org/elasticsearch/index/IndexSettings.java

@@ -552,13 +552,10 @@ public final class IndexSettings {
      */
     private final IndexMode mode;
     /**
-     * Start time of the time_series index.
+     * The bounds for {@code @timestamp} on this index or
+     * {@code null} if there are no bounds.
      */
-    private final long timeSeriesStartTime;
-    /**
-     * End time of the time_series index.
-     */
-    private volatile long timeSeriesEndTime;
+    private final TimestampBounds timestampBounds;
 
     // volatile fields are updated via #updateIndexMetadata(IndexMetadata) under lock
     private volatile Settings settings;
@@ -701,8 +698,7 @@ public final class IndexSettings {
         this.indexMetadata = indexMetadata;
         numberOfShards = settings.getAsInt(IndexMetadata.SETTING_NUMBER_OF_SHARDS, null);
         mode = isTimeSeriesModeEnabled() ? scopedSettings.get(MODE) : IndexMode.STANDARD;
-        timeSeriesStartTime = TIME_SERIES_START_TIME.get(settings).toEpochMilli();
-        timeSeriesEndTime = TIME_SERIES_END_TIME.get(settings).toEpochMilli();
+        this.timestampBounds = TIME_SERIES_START_TIME.exists(settings) ? new TimestampBounds(scopedSettings) : null;
         this.searchThrottled = INDEX_SEARCH_THROTTLED.get(settings);
         this.queryStringLenient = QUERY_STRING_LENIENT_SETTING.get(settings);
         this.queryStringAnalyzeWildcard = QUERY_STRING_ANALYZE_WILDCARD.get(nodeSettings);
@@ -813,9 +809,6 @@ public final class IndexSettings {
         scopedSettings.addSettingsUpdateConsumer(INDEX_MAPPING_DEPTH_LIMIT_SETTING, this::setMappingDepthLimit);
         scopedSettings.addSettingsUpdateConsumer(INDEX_MAPPING_FIELD_NAME_LENGTH_LIMIT_SETTING, this::setMappingFieldNameLengthLimit);
         scopedSettings.addSettingsUpdateConsumer(INDEX_MAPPING_DIMENSION_FIELDS_LIMIT_SETTING, this::setMappingDimensionFieldsLimit);
-        if (IndexSettings.isTimeSeriesModeEnabled()) {
-            scopedSettings.addSettingsUpdateConsumer(TIME_SERIES_END_TIME, this::updateTimeSeriesEndTime);
-        }
     }
 
     private void setSearchIdleAfter(TimeValue searchIdleAfter) {
@@ -1330,21 +1323,11 @@ public final class IndexSettings {
         this.mappingDimensionFieldsLimit = value;
     }
 
-    public long getTimeSeriesStartTime() {
-        return timeSeriesStartTime;
-    }
-
-    public long getTimeSeriesEndTime() {
-        return timeSeriesEndTime;
-    }
-
-    public void updateTimeSeriesEndTime(Instant endTimeInstant) {
-        long endTime = endTimeInstant.toEpochMilli();
-        if (this.timeSeriesEndTime > endTime) {
-            throw new IllegalArgumentException(
-                "index.time_series.end_time must be larger than current value [" + this.timeSeriesEndTime + "]"
-            );
-        }
-        this.timeSeriesEndTime = endTime;
+    /**
+     * The bounds for {@code @timestamp} on this index or
+     * {@code null} if there are no bounds.
+     */
+    public TimestampBounds getTimestampBounds() {
+        return timestampBounds;
     }
 }

+ 55 - 0
server/src/main/java/org/elasticsearch/index/TimestampBounds.java

@@ -0,0 +1,55 @@
+/*
+ * 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.index;
+
+import org.elasticsearch.common.settings.IndexScopedSettings;
+
+import java.time.Instant;
+
+/**
+ * Bounds for the {@code @timestamp} field on this index.
+ */
+public class TimestampBounds {
+    private final long startTime;
+    private volatile long endTime;
+
+    TimestampBounds(IndexScopedSettings scopedSettings) {
+        startTime = scopedSettings.get(IndexSettings.TIME_SERIES_START_TIME).toEpochMilli();
+        endTime = scopedSettings.get(IndexSettings.TIME_SERIES_END_TIME).toEpochMilli();
+        scopedSettings.addSettingsUpdateConsumer(IndexSettings.TIME_SERIES_END_TIME, this::updateEndTime);
+    }
+
+    /**
+     * The first valid {@code @timestamp} for the index.
+     */
+    public long startTime() {
+        return startTime;
+    }
+
+    /**
+     * The first invalid {@code @timestamp} for the index.
+     */
+    public long endTime() {
+        return endTime;
+    }
+
+    private void updateEndTime(Instant endTimeInstant) {
+        long newEndTime = endTimeInstant.toEpochMilli();
+        if (this.endTime > newEndTime) {
+            throw new IllegalArgumentException(
+                "index.time_series.end_time must be larger than current value [" + this.endTime + "] but was [" + newEndTime + "]"
+            );
+        }
+        this.endTime = newEndTime;
+    }
+
+    @Override
+    public String toString() {
+        return startTime + "-" + endTime;
+    }
+}

+ 10 - 10
server/src/main/java/org/elasticsearch/index/mapper/DataStreamTimestampFieldMapper.java

@@ -13,7 +13,7 @@ import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.search.Query;
 import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.xcontent.XContentHelper;
-import org.elasticsearch.index.IndexMode;
+import org.elasticsearch.index.TimestampBounds;
 import org.elasticsearch.index.mapper.DateFieldMapper.Resolution;
 import org.elasticsearch.index.query.SearchExecutionContext;
 import org.elasticsearch.xcontent.XContentBuilder;
@@ -209,26 +209,27 @@ public class DataStreamTimestampFieldMapper extends MetadataFieldMapper {
             throw new IllegalArgumentException("data stream timestamp field [" + DEFAULT_PATH + "] encountered multiple values");
         }
 
-        validateTimestamp(fields[0], context);
-    }
-
-    private void validateTimestamp(IndexableField field, DocumentParserContext context) {
-        if (context.indexSettings().getMode() == null || context.indexSettings().getMode() != IndexMode.TIME_SERIES) {
-            return;
+        TimestampBounds bounds = context.indexSettings().getTimestampBounds();
+        if (bounds != null) {
+            validateTimestamp(bounds.startTime(), bounds.endTime(), fields[0], context);
         }
+    }
 
+    private void validateTimestamp(long startTime, long endTime, IndexableField field, DocumentParserContext context) {
         long originValue = field.numericValue().longValue();
         long value = originValue;
 
         Resolution resolution;
-        if (context.mappingLookup().getMapper(DEFAULT_PATH).typeName().equals(DateFieldMapper.DATE_NANOS_CONTENT_TYPE)) {
+        if (context.mappingLookup()
+            .getMapper(DataStreamTimestampFieldMapper.DEFAULT_PATH)
+            .typeName()
+            .equals(DateFieldMapper.DATE_NANOS_CONTENT_TYPE)) {
             resolution = Resolution.NANOSECONDS;
             value /= NSEC_PER_MSEC;
         } else {
             resolution = Resolution.MILLISECONDS;
         }
 
-        long startTime = context.indexSettings().getTimeSeriesStartTime();
         if (value < startTime) {
             throw new IllegalArgumentException(
                 "time series index @timestamp value ["
@@ -238,7 +239,6 @@ public class DataStreamTimestampFieldMapper extends MetadataFieldMapper {
             );
         }
 
-        long endTime = context.indexSettings().getTimeSeriesEndTime();
         if (value >= endTime) {
             throw new IllegalArgumentException(
                 "time series index @timestamp value ["

+ 20 - 10
server/src/test/java/org/elasticsearch/index/IndexSettingsTests.java

@@ -22,7 +22,6 @@ import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.test.VersionUtils;
 import org.hamcrest.Matchers;
 
-import java.time.Instant;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashSet;
@@ -683,30 +682,41 @@ public class IndexSettingsTests extends ESTestCase {
         assertSettingDeprecationsAndWarnings(new Setting<?>[] { IndexMetadata.INDEX_DATA_PATH_SETTING });
     }
 
+    public void testNoTimeRange() {
+        final Settings originalSettings = Settings.builder().build();
+        IndexSettings indexSettings = new IndexSettings(newIndexMeta("test", originalSettings), Settings.EMPTY);
+        assertNull(indexSettings.getTimestampBounds());
+    }
+
     public void testUpdateTimeSeriesTimeRange() {
         long endTime = System.currentTimeMillis();
         long startTime = endTime - TimeUnit.DAYS.toMillis(1);
-        final Settings settings = Settings.builder()
+        final Settings originalSettings = Settings.builder()
             .put(IndexSettings.MODE.getKey(), IndexMode.TIME_SERIES)
             .put(IndexMetadata.INDEX_ROUTING_PATH.getKey(), "foo")
             .put(TIME_SERIES_START_TIME.getKey(), startTime)
             .put(TIME_SERIES_END_TIME.getKey(), endTime)
             .build();
-        IndexMetadata metadata = newIndexMeta("test", settings);
-        IndexSettings indexSettings = new IndexSettings(metadata, Settings.EMPTY);
+        IndexSettings indexSettings = new IndexSettings(newIndexMeta("test", originalSettings), Settings.EMPTY);
+        assertEquals(startTime, indexSettings.getTimestampBounds().startTime());
+        assertEquals(endTime, indexSettings.getTimestampBounds().endTime());
 
-        // test update end_time
-        // smaller
+        Settings endTimeBackwards = Settings.builder()
+            .put(originalSettings)
+            .put(TIME_SERIES_END_TIME.getKey(), endTime - randomLongBetween(1, 1000))
+            .build();
         IllegalArgumentException e = expectThrows(
             IllegalArgumentException.class,
-            () -> indexSettings.updateTimeSeriesEndTime(Instant.ofEpochMilli(endTime - randomLongBetween(1, 1000)))
+            () -> indexSettings.updateIndexMetadata(newIndexMeta("test", endTimeBackwards))
         );
         assertThat(e.getMessage(), Matchers.containsString("index.time_series.end_time must be larger than current value"));
 
-        // success
         long newEndTime = endTime + randomLongBetween(1, 1000);
-        indexSettings.updateTimeSeriesEndTime(Instant.ofEpochMilli(newEndTime));
-        assertEquals(newEndTime, indexSettings.getTimeSeriesEndTime());
+        Settings endTimeForwards = Settings.builder().put(originalSettings).put(TIME_SERIES_END_TIME.getKey(), newEndTime).build();
+        indexSettings.updateIndexMetadata(newIndexMeta("test", endTimeForwards));
+
+        assertEquals(startTime, indexSettings.getTimestampBounds().startTime());
+        assertEquals(newEndTime, indexSettings.getTimestampBounds().endTime());
     }
 
     public void testTimeSeriesTimeBoundary() {