Browse Source

ILM: parse origination date from index name (#46755)

* ILM: parse origination date from index name

Introduce the `index.lifecycle.parse_origination_date` setting that
indicates if the origination date should be parsed from the index name.
If set to true an index which doesn't match the expected format (namely
`indexName-{dateFormat}-optional_digits` will fail before being created.
The origination date will be parsed when initialising a lifecycle for an
index and it will be set as the `index.lifecycle.origination_date` for
that index.

A user set value for `index.lifecycle.origination_date` will always
override a possible parsable date from the index name.
Andrei Dan 6 years ago
parent
commit
c363d27f02

+ 8 - 0
docs/reference/settings/ilm-settings.asciidoc

@@ -27,6 +27,14 @@ information about rollover, see <<using-policies-rollover>>.
 (<<time-units, time units>>) How often {ilm} checks for indices that meet policy
 criteria. Defaults to `10m`.
 
+`index.lifecycle.parse_origination_date`::
+When configured to `true` the origination date will be parsed from the index
+name. The index format must match the pattern `^.*-{date_format}-\\d+`, where
+the `date_format` is `yyyy.MM.dd` and the trailing digits are optional (an
+index that was rolled over would normally match the full format eg.
+`logs-2016.10.31-000002`). If the index name doesn't match the pattern
+the index creation will fail.
+
 `index.lifecycle.origination_date`::
 The timestamp that will be used to calculate the index age for its phase
 transitions. This allows the users to create an index containing old data and

+ 50 - 0
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/IndexLifecycleOriginationDateParser.java

@@ -0,0 +1,50 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License;
+ * you may not use this file except in compliance with the Elastic License.
+ */
+package org.elasticsearch.xpack.core.ilm;
+
+import org.elasticsearch.ElasticsearchParseException;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.time.DateFormatter;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.elasticsearch.xpack.core.ilm.LifecycleSettings.LIFECYCLE_ORIGINATION_DATE;
+import static org.elasticsearch.xpack.core.ilm.LifecycleSettings.LIFECYCLE_PARSE_ORIGINATION_DATE;
+
+public class IndexLifecycleOriginationDateParser {
+
+    private static final DateFormatter DATE_FORMATTER = DateFormatter.forPattern("yyyy.MM.dd");
+    private static final String INDEX_NAME_REGEX = "^.*-(\\d{4}.\\d{2}.\\d{2})(-[\\d]+)?$";
+    private static final Pattern INDEX_NAME_PATTERN = Pattern.compile(INDEX_NAME_REGEX);
+
+    /**
+     * Determines if the origination date needs to be parsed from the index name.
+     */
+    public static boolean shouldParseIndexName(Settings indexSettings) {
+        return indexSettings.getAsLong(LIFECYCLE_ORIGINATION_DATE, -1L) == -1L &&
+            indexSettings.getAsBoolean(LIFECYCLE_PARSE_ORIGINATION_DATE, false);
+    }
+
+    /**
+     * Parses the index according to the supported format and extracts the origination date. If the index does not match the expected
+     * format or the date in the index name doesn't match the `yyyy.MM.dd` format it throws an {@link IllegalArgumentException}
+     */
+    public static long parseIndexNameAndExtractDate(String indexName) {
+        Matcher matcher = INDEX_NAME_PATTERN.matcher(indexName);
+        if (matcher.matches()) {
+            String dateAsString = matcher.group(1);
+            try {
+                return DATE_FORMATTER.parseMillis(dateAsString);
+            } catch (ElasticsearchParseException | IllegalArgumentException e) {
+                throw new IllegalArgumentException("index name [" + indexName + "] contains date [" + dateAsString + "] which " +
+                    "couldn't be parsed using the 'yyyy.MM.dd' format", e);
+            }
+        }
+
+        throw new IllegalArgumentException("index name [" + indexName + "] does not match pattern '" + INDEX_NAME_REGEX + "'");
+    }
+}

+ 21 - 3
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/InitializePolicyContextStep.java

@@ -10,8 +10,11 @@ import org.apache.logging.log4j.Logger;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.cluster.metadata.MetaData;
+import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.index.Index;
 
+import static org.elasticsearch.xpack.core.ilm.IndexLifecycleOriginationDateParser.parseIndexNameAndExtractDate;
+import static org.elasticsearch.xpack.core.ilm.IndexLifecycleOriginationDateParser.shouldParseIndexName;
 import static org.elasticsearch.xpack.core.ilm.LifecycleExecutionState.ILM_CUSTOM_METADATA_KEY;
 
 /**
@@ -34,19 +37,34 @@ public final class InitializePolicyContextStep extends ClusterStateActionStep {
             // Index must have been since deleted, ignore it
             return clusterState;
         }
+
         LifecycleExecutionState lifecycleState = LifecycleExecutionState
             .fromIndexMetadata(indexMetaData);
+
         if (lifecycleState.getLifecycleDate() != null) {
             return clusterState;
         }
 
+        IndexMetaData.Builder indexMetadataBuilder = IndexMetaData.builder(indexMetaData);
+        if (shouldParseIndexName(indexMetaData.getSettings())) {
+            long parsedOriginationDate = parseIndexNameAndExtractDate(index.getName());
+            indexMetadataBuilder.settingsVersion(indexMetaData.getSettingsVersion() + 1)
+                .settings(Settings.builder()
+                    .put(indexMetaData.getSettings())
+                    .put(LifecycleSettings.LIFECYCLE_ORIGINATION_DATE, parsedOriginationDate)
+                    .build()
+                );
+        }
+
         ClusterState.Builder newClusterStateBuilder = ClusterState.builder(clusterState);
 
         LifecycleExecutionState.Builder newCustomData = LifecycleExecutionState.builder(lifecycleState);
         newCustomData.setIndexCreationDate(indexMetaData.getCreationDate());
-        newClusterStateBuilder.metaData(MetaData.builder(clusterState.getMetaData()).put(IndexMetaData
-            .builder(indexMetaData)
-            .putCustom(ILM_CUSTOM_METADATA_KEY, newCustomData.build().asMap())));
+        indexMetadataBuilder.putCustom(ILM_CUSTOM_METADATA_KEY, newCustomData.build().asMap());
+
+        newClusterStateBuilder.metaData(
+            MetaData.builder(clusterState.getMetaData()).put(indexMetadataBuilder)
+        );
         return newClusterStateBuilder.build();
     }
 }

+ 3 - 0
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/LifecycleSettings.java

@@ -18,6 +18,7 @@ public class LifecycleSettings {
     public static final String LIFECYCLE_NAME = "index.lifecycle.name";
     public static final String LIFECYCLE_INDEXING_COMPLETE = "index.lifecycle.indexing_complete";
     public static final String LIFECYCLE_ORIGINATION_DATE = "index.lifecycle.origination_date";
+    public static final String LIFECYCLE_PARSE_ORIGINATION_DATE = "index.lifecycle.parse_origination_date";
 
     public static final String SLM_HISTORY_INDEX_ENABLED = "slm.history_index_enabled";
     public static final String SLM_RETENTION_SCHEDULE = "slm.retention_schedule";
@@ -32,6 +33,8 @@ public class LifecycleSettings {
         Setting.Property.Dynamic, Setting.Property.IndexScope);
     public static final Setting<Long> LIFECYCLE_ORIGINATION_DATE_SETTING =
         Setting.longSetting(LIFECYCLE_ORIGINATION_DATE, -1, -1, Setting.Property.Dynamic, Setting.Property.IndexScope);
+    public static final Setting<Boolean> LIFECYCLE_PARSE_ORIGINATION_DATE_SETTING = Setting.boolSetting(LIFECYCLE_PARSE_ORIGINATION_DATE,
+        false, Setting.Property.Dynamic, Setting.Property.IndexScope);
 
     public static final Setting<Boolean> SLM_HISTORY_INDEX_ENABLED_SETTING = Setting.boolSetting(SLM_HISTORY_INDEX_ENABLED, true,
         Setting.Property.NodeScope);

+ 112 - 0
x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/IndexLifecycleOriginationDateParserTests.java

@@ -0,0 +1,112 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License;
+ * you may not use this file except in compliance with the Elastic License.
+ */
+package org.elasticsearch.xpack.core.ilm;
+
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.test.ESTestCase;
+
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Locale;
+import java.util.TimeZone;
+
+import static org.elasticsearch.xpack.core.ilm.IndexLifecycleOriginationDateParser.parseIndexNameAndExtractDate;
+import static org.elasticsearch.xpack.core.ilm.IndexLifecycleOriginationDateParser.shouldParseIndexName;
+import static org.hamcrest.Matchers.is;
+
+public class IndexLifecycleOriginationDateParserTests extends ESTestCase {
+
+    public void testShouldParseIndexNameReturnsFalseWhenOriginationDateIsSet() {
+        Settings settings = Settings.builder()
+            .put(LifecycleSettings.LIFECYCLE_ORIGINATION_DATE, 1L)
+            .build();
+        assertThat(shouldParseIndexName(settings), is(false));
+    }
+
+    public void testShouldParseIndexNameReturnsFalseIfParseOriginationDateIsDisabled() {
+        Settings settings = Settings.builder()
+            .put(LifecycleSettings.LIFECYCLE_PARSE_ORIGINATION_DATE, false)
+            .build();
+        assertThat(shouldParseIndexName(settings), is(false));
+    }
+
+    public void testShouldParseIndexNameReturnsTrueIfParseOriginationDateIsTrueAndOriginationDateIsNotSet() {
+        Settings settings = Settings.builder()
+            .put(LifecycleSettings.LIFECYCLE_PARSE_ORIGINATION_DATE, true)
+            .build();
+        assertThat(shouldParseIndexName(settings), is(true));
+    }
+
+    public void testParseIndexNameThatMatchesExpectedFormat() throws ParseException {
+        SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy.MM.dd", Locale.getDefault());
+        dateFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
+        long expectedDate = dateFormat.parse("2019.09.04").getTime();
+
+        {
+            long parsedDate = parseIndexNameAndExtractDate("indexName-2019.09.04");
+            assertThat("indexName-yyyy.MM.dd is a valid index format", parsedDate, is(expectedDate));
+        }
+
+        {
+            long parsedDate = parseIndexNameAndExtractDate("indexName-2019.09.04-0000001");
+            assertThat("indexName-yyyy.MM.dd-\\d+$ is a valid index format", parsedDate, is(expectedDate));
+        }
+
+        {
+            long parsedDate = parseIndexNameAndExtractDate("indexName-2019.09.04-2019.09.24");
+            long secondDateInIndexName = dateFormat.parse("2019.09.24").getTime();
+            assertThat("indexName-yyyy.MM.dd-yyyy.MM.dd is a valid index format and the second date should be parsed",
+                parsedDate, is(secondDateInIndexName));
+        }
+
+        {
+            long parsedDate = parseIndexNameAndExtractDate("index-2019.09.04-2019.09.24-00002");
+            long secondDateInIndexName = dateFormat.parse("2019.09.24").getTime();
+            assertThat("indexName-yyyy.MM.dd-yyyy.MM.dd-digits is a valid index format and the second date should be parsed",
+                parsedDate, is(secondDateInIndexName));
+        }
+    }
+
+    public void testParseIndexNameThrowsIllegalArgumentExceptionForInvalidIndexFormat() {
+        expectThrows(
+            IllegalArgumentException.class,
+            "plainIndexName does not match the expected pattern",
+            () -> parseIndexNameAndExtractDate("plainIndexName")
+        );
+
+        expectThrows(
+            IllegalArgumentException.class,
+            "indexName--00001 does not match the expected pattern as the origination date is missing",
+            () -> parseIndexNameAndExtractDate("indexName--00001")
+        );
+
+        expectThrows(
+            IllegalArgumentException.class,
+            "indexName-00001 does not match the expected pattern as the origination date is missing",
+            () -> parseIndexNameAndExtractDate("indexName-00001")
+        );
+
+        expectThrows(
+            IllegalArgumentException.class,
+            "indexName_2019.09.04_00001 does not match the expected pattern as _ is not the expected delimiter",
+            () -> parseIndexNameAndExtractDate("indexName_2019.09.04_00001")
+        );
+    }
+
+    public void testParseIndexNameThrowsIllegalArgumentExceptionForInvalidDateFormat() {
+        expectThrows(
+            IllegalArgumentException.class,
+            "indexName-2019.04-00001 does not match the expected pattern as the date does not conform with the yyyy.MM.dd pattern",
+            () -> parseIndexNameAndExtractDate("indexName-2019.04-00001")
+        );
+
+        expectThrows(
+            IllegalArgumentException.class,
+            "java.lang.IllegalArgumentException: failed to parse date field [2019.09.44] with format [yyyy.MM.dd]",
+            () -> parseIndexNameAndExtractDate("index-2019.09.44")
+        );
+    }
+}

+ 10 - 0
x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IndexLifecycle.java

@@ -27,6 +27,7 @@ import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.core.internal.io.IOUtils;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.env.NodeEnvironment;
+import org.elasticsearch.index.IndexModule;
 import org.elasticsearch.plugins.ActionPlugin;
 import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.rest.RestController;
@@ -141,6 +142,7 @@ public class IndexLifecycle extends Plugin implements ActionPlugin {
             LifecycleSettings.LIFECYCLE_POLL_INTERVAL_SETTING,
             LifecycleSettings.LIFECYCLE_NAME_SETTING,
             LifecycleSettings.LIFECYCLE_ORIGINATION_DATE_SETTING,
+            LifecycleSettings.LIFECYCLE_PARSE_ORIGINATION_DATE_SETTING,
             LifecycleSettings.LIFECYCLE_INDEXING_COMPLETE_SETTING,
             RolloverAction.LIFECYCLE_ROLLOVER_ALIAS_SETTING,
             LifecycleSettings.SLM_HISTORY_INDEX_ENABLED_SETTING,
@@ -268,6 +270,14 @@ public class IndexLifecycle extends Plugin implements ActionPlugin {
         return actions;
     }
 
+    @Override
+    public void onIndexModule(IndexModule indexModule) {
+        if (ilmEnabled) {
+            assert indexLifecycleInitialisationService.get() != null;
+            indexModule.addIndexEventListener(indexLifecycleInitialisationService.get());
+        }
+    }
+
     @Override
     public void close() {
         try {

+ 13 - 1
x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IndexLifecycleService.java

@@ -22,6 +22,8 @@ import org.elasticsearch.common.component.Lifecycle.State;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
+import org.elasticsearch.index.Index;
+import org.elasticsearch.index.shard.IndexEventListener;
 import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.xpack.core.XPackField;
 import org.elasticsearch.xpack.core.ilm.IndexLifecycleMetadata;
@@ -39,11 +41,14 @@ import java.util.Collections;
 import java.util.Set;
 import java.util.function.LongSupplier;
 
+import static org.elasticsearch.xpack.core.ilm.IndexLifecycleOriginationDateParser.parseIndexNameAndExtractDate;
+import static org.elasticsearch.xpack.core.ilm.IndexLifecycleOriginationDateParser.shouldParseIndexName;
+
 /**
  * A service which runs the {@link LifecyclePolicy}s associated with indexes.
  */
 public class IndexLifecycleService
-    implements ClusterStateListener, ClusterStateApplier, SchedulerEngine.Listener, Closeable, LocalNodeMasterListener {
+    implements ClusterStateListener, ClusterStateApplier, SchedulerEngine.Listener, Closeable, LocalNodeMasterListener, IndexEventListener {
     private static final Logger logger = LogManager.getLogger(IndexLifecycleService.class);
     private static final Set<String> IGNORE_STEPS_MAINTENANCE_REQUESTED = Collections.singleton(ShrinkStep.NAME);
     private volatile boolean isMaster = false;
@@ -148,6 +153,13 @@ public class IndexLifecycleService
         return ThreadPool.Names.MANAGEMENT;
     }
 
+    @Override
+    public void beforeIndexAddedToCluster(Index index, Settings indexSettings) {
+        if (shouldParseIndexName(indexSettings)) {
+            parseIndexNameAndExtractDate(index.getName());
+        }
+    }
+
     private void updatePollInterval(TimeValue newInterval) {
         this.pollInterval = newInterval;
         maybeScheduleJob();

+ 79 - 6
x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/IndexLifecycleInitialisationTests.java

@@ -51,13 +51,16 @@ import org.elasticsearch.xpack.core.ilm.action.StopILMAction;
 import org.junit.Before;
 
 import java.io.IOException;
+import java.text.SimpleDateFormat;
 import java.time.Instant;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
+import java.util.TimeZone;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.stream.Collectors;
@@ -72,6 +75,7 @@ import static org.elasticsearch.xpack.core.ilm.LifecyclePolicyTestsUtils.newLock
 import static org.hamcrest.CoreMatchers.not;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThan;
 import static org.hamcrest.Matchers.greaterThanOrEqualTo;
 import static org.hamcrest.Matchers.lessThanOrEqualTo;
 import static org.hamcrest.core.CombinableMatcher.both;
@@ -211,7 +215,7 @@ public class IndexLifecycleInitialisationTests extends ESIntegTestCase {
         {
             PhaseExecutionInfo expectedExecutionInfo = new PhaseExecutionInfo(lifecyclePolicy.getName(), mockPhase, 1L, actualModifiedDate);
             assertBusy(() -> {
-                IndexLifecycleExplainResponse indexResponse = executeExplainRequestAndGetTestIndexResponse();
+                IndexLifecycleExplainResponse indexResponse = executeExplainRequestAndGetTestIndexResponse("test");
                 assertThat(indexResponse.getStep(), equalTo("observable_cluster_state_action"));
                 assertThat(indexResponse.getPhaseExecutionInfo(), equalTo(expectedExecutionInfo));
                 originalLifecycleDate.set(indexResponse.getLifecycleDate());
@@ -224,7 +228,7 @@ public class IndexLifecycleInitialisationTests extends ESIntegTestCase {
 
         {
             assertBusy(() -> {
-                IndexLifecycleExplainResponse indexResponse = executeExplainRequestAndGetTestIndexResponse();
+                IndexLifecycleExplainResponse indexResponse = executeExplainRequestAndGetTestIndexResponse("test");
                 assertThat("The configured origination date dictates the lifecycle date",
                     indexResponse.getLifecycleDate(), equalTo(1000L));
             });
@@ -236,7 +240,7 @@ public class IndexLifecycleInitialisationTests extends ESIntegTestCase {
 
         {
             assertBusy(() -> {
-                IndexLifecycleExplainResponse indexResponse = executeExplainRequestAndGetTestIndexResponse();
+                IndexLifecycleExplainResponse indexResponse = executeExplainRequestAndGetTestIndexResponse("test");
                 assertThat("Without the origination date, the index create date should dictate the lifecycle date",
                     indexResponse.getLifecycleDate(), equalTo(originalLifecycleDate.get()));
             });
@@ -249,7 +253,7 @@ public class IndexLifecycleInitialisationTests extends ESIntegTestCase {
         {
             PhaseExecutionInfo expectedExecutionInfo = new PhaseExecutionInfo(lifecyclePolicy.getName(), null, 1L, actualModifiedDate);
             assertBusy(() -> {
-                IndexLifecycleExplainResponse indexResponse = executeExplainRequestAndGetTestIndexResponse();
+                IndexLifecycleExplainResponse indexResponse = executeExplainRequestAndGetTestIndexResponse("test");
                 assertThat(indexResponse.getPhase(), equalTo(TerminalPolicyStep.COMPLETED_PHASE));
                 assertThat(indexResponse.getStep(), equalTo(TerminalPolicyStep.KEY.getName()));
                 assertThat(indexResponse.getPhaseExecutionInfo(), equalTo(expectedExecutionInfo));
@@ -257,11 +261,80 @@ public class IndexLifecycleInitialisationTests extends ESIntegTestCase {
         }
     }
 
-    private IndexLifecycleExplainResponse executeExplainRequestAndGetTestIndexResponse() throws ExecutionException, InterruptedException {
+    public void testExplainParseOriginationDate() throws Exception {
+        // start node
+        logger.info("Starting server1");
+        internalCluster().startNode();
+        logger.info("Starting server2");
+        internalCluster().startNode();
+        logger.info("Creating lifecycle [test_lifecycle]");
+        PutLifecycleAction.Request putLifecycleRequest = new PutLifecycleAction.Request(lifecyclePolicy);
+        PutLifecycleAction.Response putLifecycleResponse = client().execute(PutLifecycleAction.INSTANCE, putLifecycleRequest).get();
+        assertAcked(putLifecycleResponse);
+
+        GetLifecycleAction.Response getLifecycleResponse = client().execute(GetLifecycleAction.INSTANCE,
+            new GetLifecycleAction.Request()).get();
+        assertThat(getLifecycleResponse.getPolicies().size(), equalTo(1));
+        GetLifecycleAction.LifecyclePolicyResponseItem responseItem = getLifecycleResponse.getPolicies().get(0);
+        assertThat(responseItem.getLifecyclePolicy(), equalTo(lifecyclePolicy));
+        assertThat(responseItem.getVersion(), equalTo(1L));
+        long actualModifiedDate = Instant.parse(responseItem.getModifiedDate()).toEpochMilli();
+
+        String indexName = "test-2019.09.14";
+        logger.info("Creating index [{}]", indexName);
+        CreateIndexResponse createIndexResponse =
+            client().admin().indices().create(createIndexRequest(indexName)
+                .settings(Settings.builder().put(settings).put(LifecycleSettings.LIFECYCLE_PARSE_ORIGINATION_DATE, true))
+            ).actionGet();
+        assertAcked(createIndexResponse);
+
+        SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy.MM.dd", Locale.getDefault());
+        dateFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
+        long parsedLifecycleDate = dateFormat.parse("2019.09.14").getTime();
+        assertBusy(() -> {
+            IndexLifecycleExplainResponse indexResponse = executeExplainRequestAndGetTestIndexResponse(indexName);
+            assertThat(indexResponse.getLifecycleDate(), is(parsedLifecycleDate));
+        });
+
+        // disabling the lifecycle parsing would maintain the parsed value as that was set as the origination date
+        client().admin().indices().prepareUpdateSettings(indexName)
+            .setSettings(Collections.singletonMap(LifecycleSettings.LIFECYCLE_PARSE_ORIGINATION_DATE, false)).get();
+
+        assertBusy(() -> {
+            IndexLifecycleExplainResponse indexResponse = executeExplainRequestAndGetTestIndexResponse(indexName);
+            assertThat(indexResponse.getLifecycleDate(), is(parsedLifecycleDate));
+        });
+
+        // setting the lifecycle origination date setting to null should make the lifecyle date fallback on the index creation date
+        client().admin().indices().prepareUpdateSettings(indexName)
+            .setSettings(Collections.singletonMap(LifecycleSettings.LIFECYCLE_ORIGINATION_DATE, null)).get();
+
+        assertBusy(() -> {
+            IndexLifecycleExplainResponse indexResponse = executeExplainRequestAndGetTestIndexResponse(indexName);
+            assertThat(indexResponse.getLifecycleDate(), is(greaterThan(parsedLifecycleDate)));
+        });
+
+        // setting the lifecycle origination date to an explicit value overrides the date parsing
+        long originationDate = 42L;
+        client().admin().indices().prepareUpdateSettings(indexName)
+            .setSettings(
+                Map.of(
+                    LifecycleSettings.LIFECYCLE_PARSE_ORIGINATION_DATE, true,
+                    LifecycleSettings.LIFECYCLE_ORIGINATION_DATE, originationDate)
+            ).get();
+
+        assertBusy(() -> {
+            IndexLifecycleExplainResponse indexResponse = executeExplainRequestAndGetTestIndexResponse(indexName);
+            assertThat(indexResponse.getLifecycleDate(), is(originationDate));
+        });
+    }
+
+    private IndexLifecycleExplainResponse executeExplainRequestAndGetTestIndexResponse(String indexName) throws ExecutionException,
+        InterruptedException {
         ExplainLifecycleRequest explainRequest = new ExplainLifecycleRequest();
         ExplainLifecycleResponse explainResponse = client().execute(ExplainLifecycleAction.INSTANCE, explainRequest).get();
         assertThat(explainResponse.getIndexResponses().size(), equalTo(1));
-        return explainResponse.getIndexResponses().get("test");
+        return explainResponse.getIndexResponses().get(indexName);
     }
 
     public void testMasterDedicatedDataDedicated() throws Exception {

+ 18 - 0
x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/IndexLifecycleServiceTests.java

@@ -50,6 +50,7 @@ import java.time.ZoneId;
 import java.util.Collections;
 import java.util.SortedMap;
 import java.util.TreeMap;
+import java.util.UUID;
 import java.util.concurrent.ExecutorService;
 
 import static org.elasticsearch.node.Node.NODE_MASTER_SETTING;
@@ -305,4 +306,21 @@ public class IndexLifecycleServiceTests extends ESTestCase {
         indexLifecycleService.triggered(schedulerEvent);
         Mockito.verifyZeroInteractions(indicesClient, clusterService);
     }
+
+    public void testParsingOriginationDateBeforeIndexCreation() {
+        Settings indexSettings = Settings.builder().put(LifecycleSettings.LIFECYCLE_PARSE_ORIGINATION_DATE, true).build();
+        Index index = new Index("invalid_index_name", UUID.randomUUID().toString());
+        expectThrows(IllegalArgumentException.class,
+            "The parse origination date setting was configured for index " + index.getName() +
+                " but the index name did not match the expected format",
+            () -> indexLifecycleService.beforeIndexAddedToCluster(index, indexSettings)
+        );
+
+        // disabling the parsing origination date setting should prevent the validation from throwing exception
+        try {
+            indexLifecycleService.beforeIndexAddedToCluster(index, Settings.EMPTY);
+        } catch (Exception e) {
+            fail("Did not expect the before index validation to throw an exception as the parse origination date setting was not set");
+        }
+    }
 }