Browse Source

Revert "Deprecate _source.mode in mappings (#117106)" (#117151)

This reverts #117106. Bwc tests fail, because older nodes are killed with the following error:

```
[2024-11-20T10:54:58,600][ERROR][o.e.b.ElasticsearchUncaughtExceptionHandler] [v8.17.0-0] fatal error in thread [elasticsearch[v8.17.0-0
][clusterApplierService#updateTask][T#1]], exiting java.lang.AssertionError: provided source [{"_doc":{"_data_stream_timestamp":{"enabled":true},"_source":{},"properties":{"@timestamp":{"type":"date"},"k8s":{"properties":{"pod":{"properties":{"ip":{"type":"ip"},"name":{"type":"keyword"},"network":{"properties":{"rx":{"type":"long"},"tx":{"type":"long"}}},"uid":{"type":"keyword","time_series_dimension":true}}}}},"metricset":{"type":"keyword","time_series_dimension":true}}}}] differs from mapping [{"_doc":{"_data_stream_timestamp":{"enabled":true},"_source":{"mode":"synthetic"},"properties":{"@timestamp":{"type":"date"},"k8s":{"properties":{"pod":{"properties":{"ip":{"type":"ip"},"name":{"type":"keyword"},"network":{"properties":{"rx":{"type":"long"},"tx":{"type":"long"}}},"uid":{"type":"keyword","time_series_dimension":true}}}}},"metricset":{"type":"keyword","time_series_dimension":true}}}}]
        at org.elasticsearch.server@9.0.0-SNAPSHOT/org.elasticsearch.index.mapper.DocumentMapper.<init>(DocumentMapper.java:66)
        at org.elasticsearch.server@9.0.0-SNAPSHOT/org.elasticsearch.index.mapper.MapperService.newDocumentMapper(MapperService.java:588)
        at org.elasticsearch.server@9.0.0-SNAPSHOT/org.elasticsearch.index.mapper.MapperService.updateMapping(MapperService.java:346)
        at org.elasticsearch.server@9.0.0-SNAPSHOT/org.elasticsearch.index.IndexService.updateMapping(IndexService.java:840)
        at org.elasticsearch.server@9.0.0-SNAPSHOT/org.elasticsearch.indices.cluster.IndicesClusterStateService.createIndicesAndUpdateShards(IndicesClusterStateService.java:583)
        at org.elasticsearch.server@9.0.0-SNAPSHOT/org.elasticsearch.indices.cluster.IndicesClusterStateService.doApplyClusterState(IndicesClusterStateService.java:306)
        at org.elasticsearch.server@9.0.0-SNAPSHOT/org.elasticsearch.indices.cluster.IndicesClusterStateService.applyClusterState(IndicesClusterStateService.java:260)
        at org.elasticsearch.server@9.0.0-SNAPSHOT/org.elasticsearch.cluster.service.ClusterApplierService.callClusterStateAppliers(ClusterApplierService.java:544)
        at org.elasticsearch.server@9.0.0-SNAPSHOT/org.elasticsearch.cluster.service.ClusterApplierService.callClusterStateAppliers(ClusterApplierService.java:530)
        at org.elasticsearch.server@9.0.0-SNAPSHOT/org.elasticsearch.cluster.service.ClusterApplierService.applyChanges(ClusterApplierService.java:503)
        at org.elasticsearch.server@9.0.0-SNAPSHOT/org.elasticsearch.cluster.service.ClusterApplierService.runTask(ClusterApplierService.java:432)
        at org.elasticsearch.server@9.0.0-SNAPSHOT/org.elasticsearch.cluster.service.ClusterApplierService$UpdateTask.run(ClusterApplierService.java:157)
        at org.elasticsearch.server@9.0.0-SNAPSHOT/org.elasticsearch.common.util.concurrent.ThreadContext$ContextPreservingRunnable.run(ThreadContext.java:956)
        at org.elasticsearch.server@9.0.0-SNAPSHOT/org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor$TieBreakingPrioritizedRunnable.runAndClean(PrioritizedEsThreadPoolExecutor.java:218)
        at org.elasticsearch.server@9.0.0-SNAPSHOT/org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor$TieBreakingPrioritizedRunnable.run(PrioritizedEsThreadPoolExecutor.java:184)
        at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1144)
        at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:642)
        at java.base/java.lang.Thread.run(Thread.java:1575)
```

The `mode` parameter no longer gets serialized for new indices. However on the older nodes still serialize the `mode` parameter, which caused the menioned assertion to fail. Reverting for now and see how best to address this bwc serialization issue.

We can only stop serializing mode, when all nodes are on the same version.  Unfortunately we can't invoke `c.clusterTransportVersion().get()` from parser or builder, because that calling thread isn't allowed to call `clusterService.state()`.
Martijn van Groningen 11 months ago
parent
commit
1bc60acdef
23 changed files with 133 additions and 170 deletions
  1. 2 2
      build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/test/rest/compat/compat/RestCompatTestTransformTask.java
  2. 0 10
      docs/changelog/116689.yaml
  3. 13 3
      qa/full-cluster-restart/src/javaRestTest/java/org/elasticsearch/upgrades/LogsIndexModeFullClusterRestartIT.java
  4. 15 5
      qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/LogsIndexModeRollingUpgradeIT.java
  5. 0 6
      rest-api-spec/build.gradle
  6. 5 0
      rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/logsdb/10_settings.yml
  7. 8 7
      rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/logsdb/20_source_mapping.yml
  8. 11 0
      rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/tsdb/20_mapping.yml
  9. 1 2
      server/src/main/java/org/elasticsearch/index/IndexSettingProvider.java
  10. 0 1
      server/src/main/java/org/elasticsearch/index/IndexVersions.java
  11. 31 61
      server/src/main/java/org/elasticsearch/index/mapper/SourceFieldMapper.java
  12. 1 1
      server/src/main/java/org/elasticsearch/node/NodeConstruction.java
  13. 1 1
      server/src/test/java/org/elasticsearch/index/mapper/DynamicFieldsBuilderTests.java
  14. 2 2
      server/src/test/java/org/elasticsearch/index/mapper/SourceFieldMapperTests.java
  15. 1 1
      server/src/test/java/org/elasticsearch/index/query/SearchExecutionContextTests.java
  16. 4 31
      test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java
  17. 0 3
      test/yaml-rest-runner/src/main/java/org/elasticsearch/test/rest/yaml/section/DoSection.java
  18. 3 5
      x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java
  19. 5 2
      x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/FieldExtractorTestCase.java
  20. 3 10
      x-pack/plugin/logsdb/src/main/java/org/elasticsearch/xpack/logsdb/LogsDBPlugin.java
  21. 2 6
      x-pack/plugin/logsdb/src/main/java/org/elasticsearch/xpack/logsdb/SyntheticSourceIndexSettingsProvider.java
  22. 2 4
      x-pack/plugin/logsdb/src/test/java/org/elasticsearch/xpack/logsdb/SyntheticSourceIndexSettingsProviderTests.java
  23. 23 7
      x-pack/plugin/logsdb/src/yamlRestTest/resources/rest-api-spec/test/40_source_mode_setting.yml

+ 2 - 2
build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/test/rest/compat/compat/RestCompatTestTransformTask.java

@@ -135,8 +135,8 @@ public abstract class RestCompatTestTransformTask extends DefaultTask {
         // For example: indices.get_mapping/20_missing_type/Non-existent type returns 404
         // However, the folder can be arbitrarily nest so, a == a1/a2/a3, and the test name can include forward slashes, so c == c1/c2/c3
         // So we also need to support a1/a2/a3/b/c1/c2/c3
-        boolean limitTo3Separators = fullTestName.equals("logsdb/20_source_mapping/include/exclude is supported with stored _source");
-        String[] testParts = limitTo3Separators ? fullTestName.split("/", 3) : fullTestName.split("/");
+
+        String[] testParts = fullTestName.split("/");
         if (testParts.length < 3) {
             throw new IllegalArgumentException(
                 "To skip tests, all 3 parts [folder/file/test name] must be defined. found [" + fullTestName + "]"

+ 0 - 10
docs/changelog/116689.yaml

@@ -1,10 +0,0 @@
-pr: 116689
-summary: Deprecate `_source.mode` in mappings
-area: Mapping
-type: deprecation
-issues: []
-deprecation:
-  title: Deprecate `_source.mode` in mappings
-  area: Mapping
-  details: Configuring `_source.mode` in mappings is deprecated and will be removed in future versions. Use `index.mapping.source.mode` index setting instead.
-  impact: Use `index.mapping.source.mode` index setting instead

+ 13 - 3
qa/full-cluster-restart/src/javaRestTest/java/org/elasticsearch/upgrades/LogsIndexModeFullClusterRestartIT.java

@@ -17,6 +17,7 @@ import org.elasticsearch.client.RestClient;
 import org.elasticsearch.common.network.InetAddresses;
 import org.elasticsearch.common.time.DateFormatter;
 import org.elasticsearch.common.time.FormatNames;
+import org.elasticsearch.test.MapMatcher;
 import org.elasticsearch.test.cluster.ElasticsearchCluster;
 import org.elasticsearch.test.cluster.local.distribution.DistributionType;
 import org.elasticsearch.test.rest.RestTestLegacyFeatures;
@@ -30,6 +31,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.function.Supplier;
 
+import static org.elasticsearch.test.MapMatcher.assertMap;
+import static org.elasticsearch.test.MapMatcher.matchesMap;
+
 public class LogsIndexModeFullClusterRestartIT extends ParameterizedFullClusterRestartTestCase {
 
     @ClassRule
@@ -168,16 +172,22 @@ public class LogsIndexModeFullClusterRestartIT extends ParameterizedFullClusterR
             assertOK(bulkIndexResponse);
             assertThat(entityAsMap(bulkIndexResponse).get("errors"), Matchers.is(false));
 
-            assertIndexSettings(0, Matchers.nullValue());
-            assertIndexSettings(1, Matchers.equalTo("logsdb"));
+            assertIndexMappingsAndSettings(0, Matchers.nullValue(), matchesMap().extraOk());
+            assertIndexMappingsAndSettings(
+                1,
+                Matchers.equalTo("logsdb"),
+                matchesMap().extraOk().entry("_source", Map.of("mode", "synthetic"))
+            );
         }
     }
 
-    private void assertIndexSettings(int backingIndex, final Matcher<Object> indexModeMatcher) throws IOException {
+    private void assertIndexMappingsAndSettings(int backingIndex, final Matcher<Object> indexModeMatcher, final MapMatcher mappingsMatcher)
+        throws IOException {
         assertThat(
             getSettings(client(), getWriteBackingIndex(client(), "logs-apache-production", backingIndex)).get("index.mode"),
             indexModeMatcher
         );
+        assertMap(getIndexMappingAsMap(getWriteBackingIndex(client(), "logs-apache-production", backingIndex)), mappingsMatcher);
     }
 
     private static Request createDataStream(final String dataStreamName) {

+ 15 - 5
qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/LogsIndexModeRollingUpgradeIT.java

@@ -17,6 +17,7 @@ import org.elasticsearch.client.RestClient;
 import org.elasticsearch.common.network.InetAddresses;
 import org.elasticsearch.common.time.DateFormatter;
 import org.elasticsearch.common.time.FormatNames;
+import org.elasticsearch.test.MapMatcher;
 import org.elasticsearch.test.cluster.ElasticsearchCluster;
 import org.elasticsearch.test.cluster.local.distribution.DistributionType;
 import org.hamcrest.Matcher;
@@ -29,6 +30,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.function.Supplier;
 
+import static org.elasticsearch.test.MapMatcher.assertMap;
+import static org.elasticsearch.test.MapMatcher.matchesMap;
+
 public class LogsIndexModeRollingUpgradeIT extends AbstractRollingUpgradeTestCase {
 
     @ClassRule()
@@ -156,10 +160,14 @@ public class LogsIndexModeRollingUpgradeIT extends AbstractRollingUpgradeTestCas
             assertOK(bulkIndexResponse);
             assertThat(entityAsMap(bulkIndexResponse).get("errors"), Matchers.is(false));
 
-            assertIndexSettings(0, Matchers.nullValue());
-            assertIndexSettings(1, Matchers.nullValue());
-            assertIndexSettings(2, Matchers.nullValue());
-            assertIndexSettings(3, Matchers.equalTo("logsdb"));
+            assertIndexMappingsAndSettings(0, Matchers.nullValue(), matchesMap().extraOk());
+            assertIndexMappingsAndSettings(1, Matchers.nullValue(), matchesMap().extraOk());
+            assertIndexMappingsAndSettings(2, Matchers.nullValue(), matchesMap().extraOk());
+            assertIndexMappingsAndSettings(
+                3,
+                Matchers.equalTo("logsdb"),
+                matchesMap().extraOk().entry("_source", Map.of("mode", "synthetic"))
+            );
         }
     }
 
@@ -175,11 +183,13 @@ public class LogsIndexModeRollingUpgradeIT extends AbstractRollingUpgradeTestCas
         assertOK(client().performRequest(request));
     }
 
-    private void assertIndexSettings(int backingIndex, final Matcher<Object> indexModeMatcher) throws IOException {
+    private void assertIndexMappingsAndSettings(int backingIndex, final Matcher<Object> indexModeMatcher, final MapMatcher mappingsMatcher)
+        throws IOException {
         assertThat(
             getSettings(client(), getWriteBackingIndex(client(), "logs-apache-production", backingIndex)).get("index.mode"),
             indexModeMatcher
         );
+        assertMap(getIndexMappingAsMap(getWriteBackingIndex(client(), "logs-apache-production", backingIndex)), mappingsMatcher);
     }
 
     private static Request createDataStream(final String dataStreamName) {

+ 0 - 6
rest-api-spec/build.gradle

@@ -247,10 +247,4 @@ tasks.named("precommit").configure {
 tasks.named("yamlRestTestV7CompatTransform").configure({ task ->
   task.skipTest("indices.sort/10_basic/Index Sort", "warning does not exist for compatibility")
   task.skipTest("search/330_fetch_fields/Test search rewrite", "warning does not exist for compatibility")
-  task.skipTest("tsdb/20_mapping/stored source is supported", "no longer serialize source_mode")
-  task.skipTest("tsdb/20_mapping/Synthetic source", "no longer serialize source_mode")
-  task.skipTest("logsdb/10_settings/create logs index", "no longer serialize source_mode")
-  task.skipTest("logsdb/20_source_mapping/stored _source mode is supported", "no longer serialize source_mode")
-  task.skipTest("logsdb/20_source_mapping/include/exclude is supported with stored _source", "no longer serialize source_mode")
-  task.skipTest("logsdb/20_source_mapping/synthetic _source is default", "no longer serialize source_mode")
 })

+ 5 - 0
rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/logsdb/10_settings.yml

@@ -77,6 +77,11 @@ create logs index:
   - is_true: test
   - match: { test.settings.index.mode: "logsdb" }
 
+  - do:
+      indices.get_mapping:
+        index: test
+  - match: { test.mappings._source.mode: synthetic }
+
 ---
 using default timestamp field mapping:
   - requires:

+ 8 - 7
rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/logsdb/20_source_mapping.yml

@@ -13,10 +13,10 @@ synthetic _source is default:
             index:
               mode: logsdb
   - do:
-      indices.get_settings:
+      indices.get:
         index: test-default-source
-  - match: { test-default-source.settings.index.mode: logsdb }
-  - match: { test-default-source.settings.index.mapping.source.mode: null }
+
+  - match: { test-default-source.mappings._source.mode: "synthetic" }
 
 ---
 stored _source mode is supported:
@@ -28,12 +28,11 @@ stored _source mode is supported:
             index:
               mode: logsdb
               mapping.source.mode: stored
-
   - do:
-      indices.get_settings:
+      indices.get:
         index: test-stored-source
-  - match: { test-stored-source.settings.index.mode: logsdb }
-  - match: { test-stored-source.settings.index.mapping.source.mode: stored }
+
+  - match: { test-stored-source.mappings._source.mode: "stored" }
 
 ---
 disabled _source is not supported:
@@ -111,6 +110,7 @@ include/exclude is supported with stored _source:
       indices.get:
         index: test-includes
 
+  - match: { test-includes.mappings._source.mode: "stored" }
   - match: { test-includes.mappings._source.includes: ["a"] }
 
   - do:
@@ -129,4 +129,5 @@ include/exclude is supported with stored _source:
       indices.get:
         index: test-excludes
 
+  - match: { test-excludes.mappings._source.mode: "stored" }
   - match: { test-excludes.mappings._source.excludes: ["b"] }

+ 11 - 0
rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/tsdb/20_mapping.yml

@@ -450,6 +450,11 @@ nested fields:
                 type: long
                 time_series_metric: gauge
 
+  - do:
+      indices.get_mapping: {}
+
+  - match: {tsdb-synthetic.mappings._source.mode: synthetic}
+
 ---
 stored source is supported:
   - requires:
@@ -481,6 +486,12 @@ stored source is supported:
                         type: keyword
                         time_series_dimension: true
 
+  - do:
+      indices.get:
+        index: tsdb_index
+
+  - match: { tsdb_index.mappings._source.mode: "stored" }
+
 ---
 disabled source is not supported:
   - requires:

+ 1 - 2
server/src/main/java/org/elasticsearch/index/IndexSettingProvider.java

@@ -11,7 +11,6 @@ package org.elasticsearch.index;
 
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.Metadata;
-import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.compress.CompressedXContent;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.core.CheckedFunction;
@@ -55,7 +54,7 @@ public interface IndexSettingProvider {
     /**
      * Infrastructure class that holds services that can be used by {@link IndexSettingProvider} instances.
      */
-    record Parameters(ClusterService clusterService, CheckedFunction<IndexMetadata, MapperService, IOException> mapperServiceFactory) {
+    record Parameters(CheckedFunction<IndexMetadata, MapperService, IOException> mapperServiceFactory) {
 
     }
 

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

@@ -121,7 +121,6 @@ public class IndexVersions {
     public static final IndexVersion ADD_ROLE_MAPPING_CLEANUP_MIGRATION = def(8_518_00_0, Version.LUCENE_9_12_0);
     public static final IndexVersion LOGSDB_DEFAULT_IGNORE_DYNAMIC_BEYOND_LIMIT_BACKPORT = def(8_519_00_0, Version.LUCENE_9_12_0);
     public static final IndexVersion TIME_BASED_K_ORDERED_DOC_ID_BACKPORT = def(8_520_00_0, Version.LUCENE_9_12_0);
-    public static final IndexVersion DEPRECATE_SOURCE_MODE_MAPPER = def(8_521_00_0, Version.LUCENE_9_12_0);
     /*
      * STOP! READ THIS FIRST! No, really,
      *        ____ _____ ___  ____  _        ____  _____    _    ____    _____ _   _ ___ ____    _____ ___ ____  ____ _____ _

+ 31 - 61
server/src/main/java/org/elasticsearch/index/mapper/SourceFieldMapper.java

@@ -18,7 +18,6 @@ import org.apache.lucene.util.BytesRef;
 import org.elasticsearch.common.Explicit;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.bytes.BytesReference;
-import org.elasticsearch.common.logging.DeprecationCategory;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.util.CollectionUtils;
@@ -39,7 +38,6 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.Locale;
-import java.util.Map;
 
 public class SourceFieldMapper extends MetadataFieldMapper {
     public static final NodeFeature SYNTHETIC_SOURCE_FALLBACK = new NodeFeature("mapper.source.synthetic_source_fallback");
@@ -70,9 +68,6 @@ public class SourceFieldMapper extends MetadataFieldMapper {
         return indexMode.defaultSourceMode().name();
     }, "index.mapping.source.mode", value -> {}, Setting.Property.Final, Setting.Property.IndexScope);
 
-    public static final String DEPRECATION_WARNING = "Configuring source mode in mappings is deprecated and will be removed "
-        + "in future versions. Use [index.mapping.source.mode] index setting instead.";
-
     /** The source mode */
     public enum Mode {
         DISABLED,
@@ -84,32 +79,28 @@ public class SourceFieldMapper extends MetadataFieldMapper {
         null,
         Explicit.IMPLICIT_TRUE,
         Strings.EMPTY_ARRAY,
-        Strings.EMPTY_ARRAY,
-        false
+        Strings.EMPTY_ARRAY
     );
 
     private static final SourceFieldMapper STORED = new SourceFieldMapper(
         Mode.STORED,
         Explicit.IMPLICIT_TRUE,
         Strings.EMPTY_ARRAY,
-        Strings.EMPTY_ARRAY,
-        false
+        Strings.EMPTY_ARRAY
     );
 
     private static final SourceFieldMapper SYNTHETIC = new SourceFieldMapper(
         Mode.SYNTHETIC,
         Explicit.IMPLICIT_TRUE,
         Strings.EMPTY_ARRAY,
-        Strings.EMPTY_ARRAY,
-        false
+        Strings.EMPTY_ARRAY
     );
 
     private static final SourceFieldMapper DISABLED = new SourceFieldMapper(
         Mode.DISABLED,
         Explicit.IMPLICIT_TRUE,
         Strings.EMPTY_ARRAY,
-        Strings.EMPTY_ARRAY,
-        false
+        Strings.EMPTY_ARRAY
     );
 
     public static class Defaults {
@@ -143,7 +134,16 @@ public class SourceFieldMapper extends MetadataFieldMapper {
          * The default mode for TimeSeries is left empty on purpose, so that mapping printings include the synthetic
          * source mode.
          */
-        private final Parameter<Mode> mode;
+        private final Parameter<Mode> mode = new Parameter<>(
+            "mode",
+            true,
+            () -> null,
+            (n, c, o) -> Mode.valueOf(o.toString().toUpperCase(Locale.ROOT)),
+            m -> toType(m).enabled.explicit() ? null : toType(m).mode,
+            (b, n, v) -> b.field(n, v.toString().toLowerCase(Locale.ROOT)),
+            v -> v.toString().toLowerCase(Locale.ROOT)
+        ).setMergeValidator((previous, current, conflicts) -> (previous == current) || current != Mode.STORED)
+            .setSerializerCheck((includeDefaults, isConfigured, value) -> value != null); // don't emit if `enabled` is configured
         private final Parameter<List<String>> includes = Parameter.stringArrayParam(
             "includes",
             false,
@@ -158,28 +158,15 @@ public class SourceFieldMapper extends MetadataFieldMapper {
         private final Settings settings;
 
         private final IndexMode indexMode;
-        private boolean serializeMode;
 
         private final boolean supportsNonDefaultParameterValues;
 
-        public Builder(IndexMode indexMode, final Settings settings, boolean supportsCheckForNonDefaultParams, boolean serializeMode) {
+        public Builder(IndexMode indexMode, final Settings settings, boolean supportsCheckForNonDefaultParams) {
             super(Defaults.NAME);
             this.settings = settings;
             this.indexMode = indexMode;
             this.supportsNonDefaultParameterValues = supportsCheckForNonDefaultParams == false
                 || settings.getAsBoolean(LOSSY_PARAMETERS_ALLOWED_SETTING_NAME, true);
-            this.serializeMode = serializeMode;
-            this.mode = new Parameter<>(
-                "mode",
-                true,
-                () -> null,
-                (n, c, o) -> Mode.valueOf(o.toString().toUpperCase(Locale.ROOT)),
-                m -> toType(m).enabled.explicit() ? null : toType(m).mode,
-                (b, n, v) -> b.field(n, v.toString().toLowerCase(Locale.ROOT)),
-                v -> v.toString().toLowerCase(Locale.ROOT)
-            ).setMergeValidator((previous, current, conflicts) -> (previous == current) || current != Mode.STORED)
-                // don't emit if `enabled` is configured
-                .setSerializerCheck((includeDefaults, isConfigured, value) -> serializeMode && value != null);
         }
 
         public Builder setSynthetic() {
@@ -232,22 +219,21 @@ public class SourceFieldMapper extends MetadataFieldMapper {
             if (sourceMode == Mode.SYNTHETIC && (includes.getValue().isEmpty() == false || excludes.getValue().isEmpty() == false)) {
                 throw new IllegalArgumentException("filtering the stored _source is incompatible with synthetic source");
             }
-            if (mode.isConfigured()) {
-                serializeMode = true;
-            }
-            final SourceFieldMapper sourceFieldMapper;
-            if (isDefault() && sourceMode == null) {
+
+            SourceFieldMapper sourceFieldMapper;
+            if (isDefault()) {
                 // Needed for bwc so that "mode" is not serialized in case of a standard index with stored source.
-                sourceFieldMapper = DEFAULT;
-            } else if (isDefault() && serializeMode == false && sourceMode != null) {
-                sourceFieldMapper = resolveStaticInstance(sourceMode);
+                if (sourceMode == null) {
+                    sourceFieldMapper = DEFAULT;
+                } else {
+                    sourceFieldMapper = resolveStaticInstance(sourceMode);
+                }
             } else {
                 sourceFieldMapper = new SourceFieldMapper(
                     sourceMode,
                     enabled.get(),
                     includes.getValue().toArray(Strings.EMPTY_ARRAY),
-                    excludes.getValue().toArray(Strings.EMPTY_ARRAY),
-                    serializeMode
+                    excludes.getValue().toArray(Strings.EMPTY_ARRAY)
                 );
             }
             if (indexMode != null) {
@@ -297,29 +283,15 @@ public class SourceFieldMapper extends MetadataFieldMapper {
         if (indexMode == IndexMode.STANDARD && settingSourceMode == Mode.STORED) {
             return DEFAULT;
         }
-        if (c.indexVersionCreated().onOrAfter(IndexVersions.DEPRECATE_SOURCE_MODE_MAPPER)) {
-            return resolveStaticInstance(settingSourceMode);
-        } else {
-            return new SourceFieldMapper(settingSourceMode, Explicit.IMPLICIT_TRUE, Strings.EMPTY_ARRAY, Strings.EMPTY_ARRAY, true);
-        }
+
+        return resolveStaticInstance(settingSourceMode);
     },
         c -> new Builder(
             c.getIndexSettings().getMode(),
             c.getSettings(),
-            c.indexVersionCreated().onOrAfter(IndexVersions.SOURCE_MAPPER_LOSSY_PARAMS_CHECK),
-            c.indexVersionCreated().before(IndexVersions.DEPRECATE_SOURCE_MODE_MAPPER)
+            c.indexVersionCreated().onOrAfter(IndexVersions.SOURCE_MAPPER_LOSSY_PARAMS_CHECK)
         )
-    ) {
-        @Override
-        public MetadataFieldMapper.Builder parse(String name, Map<String, Object> node, MappingParserContext parserContext)
-            throws MapperParsingException {
-            assert name.equals(SourceFieldMapper.NAME) : name;
-            if (parserContext.indexVersionCreated().after(IndexVersions.DEPRECATE_SOURCE_MODE_MAPPER) && node.containsKey("mode")) {
-                deprecationLogger.critical(DeprecationCategory.MAPPINGS, "mapping_source_mode", SourceFieldMapper.DEPRECATION_WARNING);
-            }
-            return super.parse(name, node, parserContext);
-        }
-    };
+    );
 
     static final class SourceFieldType extends MappedFieldType {
         private final boolean enabled;
@@ -358,9 +330,8 @@ public class SourceFieldMapper extends MetadataFieldMapper {
         }
     }
 
-    // nullable for bwc reasons - TODO: fold this into serializeMode
+    // nullable for bwc reasons
     private final @Nullable Mode mode;
-    private final boolean serializeMode;
     private final Explicit<Boolean> enabled;
 
     /** indicates whether the source will always exist and be complete, for use by features like the update API */
@@ -370,7 +341,7 @@ public class SourceFieldMapper extends MetadataFieldMapper {
     private final String[] excludes;
     private final SourceFilter sourceFilter;
 
-    private SourceFieldMapper(Mode mode, Explicit<Boolean> enabled, String[] includes, String[] excludes, boolean serializeMode) {
+    private SourceFieldMapper(Mode mode, Explicit<Boolean> enabled, String[] includes, String[] excludes) {
         super(new SourceFieldType((enabled.explicit() && enabled.value()) || (enabled.explicit() == false && mode != Mode.DISABLED)));
         this.mode = mode;
         this.enabled = enabled;
@@ -378,7 +349,6 @@ public class SourceFieldMapper extends MetadataFieldMapper {
         this.includes = includes;
         this.excludes = excludes;
         this.complete = stored() && sourceFilter == null;
-        this.serializeMode = serializeMode;
     }
 
     private static SourceFilter buildSourceFilter(String[] includes, String[] excludes) {
@@ -449,7 +419,7 @@ public class SourceFieldMapper extends MetadataFieldMapper {
 
     @Override
     public FieldMapper.Builder getMergeBuilder() {
-        return new Builder(null, Settings.EMPTY, false, serializeMode).init(this);
+        return new Builder(null, Settings.EMPTY, false).init(this);
     }
 
     /**

+ 1 - 1
server/src/main/java/org/elasticsearch/node/NodeConstruction.java

@@ -825,7 +825,7 @@ class NodeConstruction {
             .searchOperationListeners(searchOperationListeners)
             .build();
 
-        final var parameters = new IndexSettingProvider.Parameters(clusterService, indicesService::createIndexMapperServiceForValidation);
+        final var parameters = new IndexSettingProvider.Parameters(indicesService::createIndexMapperServiceForValidation);
         IndexSettingProviders indexSettingProviders = new IndexSettingProviders(
             Sets.union(
                 builtinIndexSettingProviders(),

+ 1 - 1
server/src/test/java/org/elasticsearch/index/mapper/DynamicFieldsBuilderTests.java

@@ -69,7 +69,7 @@ public class DynamicFieldsBuilderTests extends ESTestCase {
         XContentParser parser = createParser(JsonXContent.jsonXContent, source);
         SourceToParse sourceToParse = new SourceToParse("test", new BytesArray(source), XContentType.JSON);
 
-        SourceFieldMapper sourceMapper = new SourceFieldMapper.Builder(null, Settings.EMPTY, false, false).setSynthetic().build();
+        SourceFieldMapper sourceMapper = new SourceFieldMapper.Builder(null, Settings.EMPTY, false).setSynthetic().build();
         RootObjectMapper root = new RootObjectMapper.Builder("_doc", Optional.empty()).add(
             new PassThroughObjectMapper.Builder("labels").setPriority(0).setContainsDimensions().dynamic(ObjectMapper.Dynamic.TRUE)
         ).build(MapperBuilderContext.root(false, false));

+ 2 - 2
server/src/test/java/org/elasticsearch/index/mapper/SourceFieldMapperTests.java

@@ -247,14 +247,14 @@ public class SourceFieldMapperTests extends MetadataMapperTestCase {
         });
         DocumentMapper mapper = createTimeSeriesModeDocumentMapper(mapping);
         assertTrue(mapper.sourceMapper().isSynthetic());
-        assertEquals("{\"_source\":{}}", mapper.sourceMapper().toString());
+        assertEquals("{\"_source\":{\"mode\":\"synthetic\"}}", mapper.sourceMapper().toString());
     }
 
     public void testSyntheticSourceWithLogsIndexMode() throws IOException {
         XContentBuilder mapping = fieldMapping(b -> { b.field("type", "keyword"); });
         DocumentMapper mapper = createLogsModeDocumentMapper(mapping);
         assertTrue(mapper.sourceMapper().isSynthetic());
-        assertEquals("{\"_source\":{}}", mapper.sourceMapper().toString());
+        assertEquals("{\"_source\":{\"mode\":\"synthetic\"}}", mapper.sourceMapper().toString());
     }
 
     public void testSupportsNonDefaultParameterValues() throws IOException {

+ 1 - 1
server/src/test/java/org/elasticsearch/index/query/SearchExecutionContextTests.java

@@ -384,7 +384,7 @@ public class SearchExecutionContextTests extends ESTestCase {
 
     public void testSyntheticSourceSearchLookup() throws IOException {
         // Build a mapping using synthetic source
-        SourceFieldMapper sourceMapper = new SourceFieldMapper.Builder(null, Settings.EMPTY, false, false).setSynthetic().build();
+        SourceFieldMapper sourceMapper = new SourceFieldMapper.Builder(null, Settings.EMPTY, false).setSynthetic().build();
         RootObjectMapper root = new RootObjectMapper.Builder("_doc", Optional.empty()).add(
             new KeywordFieldMapper.Builder("cat", IndexVersion.current()).ignoreAbove(100)
         ).build(MapperBuilderContext.root(true, false));

+ 4 - 31
test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java

@@ -68,7 +68,6 @@ import org.elasticsearch.health.node.selection.HealthNode;
 import org.elasticsearch.index.IndexSettings;
 import org.elasticsearch.index.IndexVersion;
 import org.elasticsearch.index.IndexVersions;
-import org.elasticsearch.index.mapper.SourceFieldMapper;
 import org.elasticsearch.index.query.QueryBuilder;
 import org.elasticsearch.index.seqno.ReplicationTracker;
 import org.elasticsearch.rest.RestStatus;
@@ -112,7 +111,6 @@ import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Locale;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
@@ -1860,9 +1858,8 @@ public abstract class ESRestTestCase extends ESTestCase {
 
         if (settings != null && settings.getAsBoolean(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) == false) {
             expectSoftDeletesWarning(request, name);
-        } else if (isSyntheticSourceConfiguredInMapping(mapping)) {
-            request.setOptions(expectVersionSpecificWarnings(v -> v.compatible(SourceFieldMapper.DEPRECATION_WARNING)));
         }
+
         final Response response = client.performRequest(request);
         try (var parser = responseAsParser(response)) {
             return CreateIndexResponse.fromXContent(parser);
@@ -1906,27 +1903,6 @@ public abstract class ESRestTestCase extends ESTestCase {
         }));
     }
 
-    @SuppressWarnings("unchecked")
-    protected static boolean isSyntheticSourceConfiguredInMapping(String mapping) {
-        if (mapping == null) {
-            return false;
-        }
-        var mappings = XContentHelper.convertToMap(
-            JsonXContent.jsonXContent,
-            mapping.trim().startsWith("{") ? mapping : '{' + mapping + '}',
-            false
-        );
-        if (mappings.containsKey("_doc")) {
-            mappings = (Map<String, Object>) mappings.get("_doc");
-        }
-        Map<String, Object> sourceMapper = (Map<String, Object>) mappings.get(SourceFieldMapper.NAME);
-        if (sourceMapper == null) {
-            return false;
-        }
-        Object mode = sourceMapper.get("mode");
-        return mode != null && mode.toString().toLowerCase(Locale.ROOT).equals("synthetic");
-    }
-
     protected static Map<String, Object> getIndexSettings(String index) throws IOException {
         Request request = new Request("GET", "/" + index + "/_settings");
         request.addParameter("flat_settings", "true");
@@ -2320,7 +2296,7 @@ public abstract class ESRestTestCase extends ESTestCase {
      */
     protected static IndexVersion minimumIndexVersion() throws IOException {
         final Request request = new Request("GET", "_nodes");
-        request.addParameter("filter_path", "nodes.*.version,nodes.*.max_index_version,nodes.*.index_version");
+        request.addParameter("filter_path", "nodes.*.version,nodes.*.max_index_version");
 
         final Response response = adminClient().performRequest(request);
         final Map<String, Object> nodes = ObjectPath.createFromResponse(response).evaluate("nodes");
@@ -2328,13 +2304,10 @@ public abstract class ESRestTestCase extends ESTestCase {
         IndexVersion minVersion = null;
         for (Map.Entry<String, Object> node : nodes.entrySet()) {
             Map<?, ?> nodeData = (Map<?, ?>) node.getValue();
-            Object versionStr = nodeData.get("index_version");
-            if (versionStr == null) {
-                versionStr = nodeData.get("max_index_version");
-            }
+            String versionStr = (String) nodeData.get("max_index_version");
             // fallback on version if index version is not there
             IndexVersion indexVersion = versionStr != null
-                ? IndexVersion.fromId(Integer.parseInt(versionStr.toString()))
+                ? IndexVersion.fromId(Integer.parseInt(versionStr))
                 : IndexVersion.fromId(
                     parseLegacyVersion((String) nodeData.get("version")).map(Version::id).orElse(IndexVersions.MINIMUM_COMPATIBLE.id())
                 );

+ 0 - 3
test/yaml-rest-runner/src/main/java/org/elasticsearch/test/rest/yaml/section/DoSection.java

@@ -20,7 +20,6 @@ import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.logging.HeaderWarning;
 import org.elasticsearch.core.Tuple;
 import org.elasticsearch.core.UpdateForV9;
-import org.elasticsearch.index.mapper.SourceFieldMapper;
 import org.elasticsearch.rest.action.admin.indices.RestPutIndexTemplateAction;
 import org.elasticsearch.test.rest.RestTestLegacyFeatures;
 import org.elasticsearch.test.rest.yaml.ClientYamlTestExecutionContext;
@@ -507,8 +506,6 @@ public class DoSection implements ExecutableSection {
             }
         }
 
-        unexpected.removeIf(s -> s.endsWith(SourceFieldMapper.DEPRECATION_WARNING + "\""));
-
         if (unexpected.isEmpty() == false
             || unmatched.isEmpty() == false
             || missing.isEmpty() == false

+ 3 - 5
x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java

@@ -18,7 +18,6 @@ import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.util.concurrent.ThreadContext;
 import org.elasticsearch.index.IndexSettings;
 import org.elasticsearch.index.mapper.DateFieldMapper;
-import org.elasticsearch.index.mapper.SourceFieldMapper;
 import org.elasticsearch.repositories.fs.FsRepository;
 import org.elasticsearch.rest.RestStatus;
 
@@ -367,10 +366,8 @@ public class FollowIndexIT extends ESCCRRestTestCase {
         final String leaderIndexName = "synthetic_leader";
         if ("leader".equals(targetCluster)) {
             logger.info("Running against leader cluster");
-            Settings settings = Settings.builder()
-                .put(SourceFieldMapper.INDEX_MAPPER_SOURCE_MODE_SETTING.getKey(), SourceFieldMapper.Mode.SYNTHETIC)
-                .build();
-            createIndex(adminClient(), leaderIndexName, settings, """
+            createIndex(adminClient(), leaderIndexName, Settings.EMPTY, """
+                "_source": {"mode": "synthetic"},
                 "properties": {"kwd": {"type": "keyword"}}}""", null);
             for (int i = 0; i < numDocs; i++) {
                 logger.info("Indexing doc [{}]", i);
@@ -395,6 +392,7 @@ public class FollowIndexIT extends ESCCRRestTestCase {
             }
             assertBusy(() -> {
                 verifyDocuments(client(), followIndexName, numDocs);
+                assertMap(getIndexMappingAsMap(followIndexName), matchesMap().extraOk().entry("_source", Map.of("mode", "synthetic")));
                 if (overrideNumberOfReplicas) {
                     assertMap(getIndexSettingsAsMap(followIndexName), matchesMap().extraOk().entry("index.number_of_replicas", "0"));
                 } else {

+ 5 - 2
x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/FieldExtractorTestCase.java

@@ -14,7 +14,6 @@ import org.elasticsearch.client.Response;
 import org.elasticsearch.client.ResponseException;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.network.NetworkAddress;
-import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.core.CheckedConsumer;
 import org.elasticsearch.geo.GeometryTestUtils;
 import org.elasticsearch.index.mapper.BlockLoader;
@@ -1457,12 +1456,16 @@ public abstract class FieldExtractorTestCase extends ESRestTestCase {
     }
 
     private static void createIndex(String name, CheckedConsumer<XContentBuilder, IOException> mapping) throws IOException {
+        Request request = new Request("PUT", "/" + name);
         XContentBuilder index = JsonXContent.contentBuilder().prettyPrint().startObject();
+        index.startObject("mappings");
         mapping.accept(index);
         index.endObject();
+        index.endObject();
         String configStr = Strings.toString(index);
         logger.info("index: {} {}", name, configStr);
-        ESRestTestCase.createIndex(name, Settings.EMPTY, configStr);
+        request.setJsonEntity(configStr);
+        client().performRequest(request);
     }
 
     /**

+ 3 - 10
x-pack/plugin/logsdb/src/main/java/org/elasticsearch/xpack/logsdb/LogsDBPlugin.java

@@ -13,7 +13,6 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.index.IndexSettingProvider;
-import org.elasticsearch.index.IndexVersion;
 import org.elasticsearch.plugins.ActionPlugin;
 import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.xpack.core.XPackPlugin;
@@ -63,16 +62,10 @@ public class LogsDBPlugin extends Plugin implements ActionPlugin {
         if (DiscoveryNode.isStateless(settings)) {
             return List.of(logsdbIndexModeSettingsProvider);
         }
-        var syntheticSettingProvider = new SyntheticSourceIndexSettingsProvider(
-            licenseService,
-            parameters.mapperServiceFactory(),
-            logsdbIndexModeSettingsProvider,
-            () -> IndexVersion.min(
-                IndexVersion.current(),
-                parameters.clusterService().state().nodes().getMaxDataNodeCompatibleIndexVersion()
-            )
+        return List.of(
+            new SyntheticSourceIndexSettingsProvider(licenseService, parameters.mapperServiceFactory(), logsdbIndexModeSettingsProvider),
+            logsdbIndexModeSettingsProvider
         );
-        return List.of(syntheticSettingProvider, logsdbIndexModeSettingsProvider);
     }
 
     @Override

+ 2 - 6
x-pack/plugin/logsdb/src/main/java/org/elasticsearch/xpack/logsdb/SyntheticSourceIndexSettingsProvider.java

@@ -26,7 +26,6 @@ import org.elasticsearch.index.mapper.SourceFieldMapper;
 import java.io.IOException;
 import java.time.Instant;
 import java.util.List;
-import java.util.function.Supplier;
 
 import static org.elasticsearch.cluster.metadata.IndexMetadata.INDEX_ROUTING_PATH;
 
@@ -40,18 +39,15 @@ final class SyntheticSourceIndexSettingsProvider implements IndexSettingProvider
     private final SyntheticSourceLicenseService syntheticSourceLicenseService;
     private final CheckedFunction<IndexMetadata, MapperService, IOException> mapperServiceFactory;
     private final LogsdbIndexModeSettingsProvider logsdbIndexModeSettingsProvider;
-    private final Supplier<IndexVersion> createdIndexVersion;
 
     SyntheticSourceIndexSettingsProvider(
         SyntheticSourceLicenseService syntheticSourceLicenseService,
         CheckedFunction<IndexMetadata, MapperService, IOException> mapperServiceFactory,
-        LogsdbIndexModeSettingsProvider logsdbIndexModeSettingsProvider,
-        Supplier<IndexVersion> createdIndexVersion
+        LogsdbIndexModeSettingsProvider logsdbIndexModeSettingsProvider
     ) {
         this.syntheticSourceLicenseService = syntheticSourceLicenseService;
         this.mapperServiceFactory = mapperServiceFactory;
         this.logsdbIndexModeSettingsProvider = logsdbIndexModeSettingsProvider;
-        this.createdIndexVersion = createdIndexVersion;
     }
 
     @Override
@@ -152,7 +148,7 @@ final class SyntheticSourceIndexSettingsProvider implements IndexSettingProvider
         );
         int shardReplicas = indexTemplateAndCreateRequestSettings.getAsInt(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0);
         var finalResolvedSettings = Settings.builder()
-            .put(IndexMetadata.SETTING_VERSION_CREATED, createdIndexVersion.get())
+            .put(IndexMetadata.SETTING_VERSION_CREATED, IndexVersion.current())
             .put(indexTemplateAndCreateRequestSettings)
             .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, dummyShards)
             .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, shardReplicas)

+ 2 - 4
x-pack/plugin/logsdb/src/test/java/org/elasticsearch/xpack/logsdb/SyntheticSourceIndexSettingsProviderTests.java

@@ -15,7 +15,6 @@ import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.core.Tuple;
 import org.elasticsearch.index.IndexMode;
 import org.elasticsearch.index.IndexSettings;
-import org.elasticsearch.index.IndexVersion;
 import org.elasticsearch.index.MapperTestUtils;
 import org.elasticsearch.index.mapper.SourceFieldMapper;
 import org.elasticsearch.license.MockLicenseState;
@@ -55,7 +54,7 @@ public class SyntheticSourceIndexSettingsProviderTests extends ESTestCase {
         provider = new SyntheticSourceIndexSettingsProvider(syntheticSourceLicenseService, im -> {
             newMapperServiceCounter.incrementAndGet();
             return MapperTestUtils.newMapperService(xContentRegistry(), createTempDir(), im.getSettings(), im.getIndex().getName());
-        }, getLogsdbIndexModeSettingsProvider(false), IndexVersion::current);
+        }, getLogsdbIndexModeSettingsProvider(false));
         newMapperServiceCounter.set(0);
     }
 
@@ -337,8 +336,7 @@ public class SyntheticSourceIndexSettingsProviderTests extends ESTestCase {
         provider = new SyntheticSourceIndexSettingsProvider(
             syntheticSourceLicenseService,
             im -> MapperTestUtils.newMapperService(xContentRegistry(), createTempDir(), im.getSettings(), im.getIndex().getName()),
-            getLogsdbIndexModeSettingsProvider(true),
-            IndexVersion::current
+            getLogsdbIndexModeSettingsProvider(true)
         );
         final Settings settings = Settings.EMPTY;
 

+ 23 - 7
x-pack/plugin/logsdb/src/yamlRestTest/resources/rest-api-spec/test/40_source_mode_setting.yml

@@ -459,7 +459,13 @@ create an index with time_series index mode and synthetic source:
       indices.get_settings:
         index: "test_time_series_index_mode_synthetic"
   - match: { test_time_series_index_mode_synthetic.settings.index.mode: time_series }
-  - match: { test_time_series_index_mode_synthetic.settings.index.mapping.source.mode: synthetic }
+
+
+  - do:
+      indices.get_mapping:
+        index: test_time_series_index_mode_synthetic
+
+  - match: { test_time_series_index_mode_synthetic.mappings._source.mode: synthetic }
 
 ---
 create an index with logsdb index mode and synthetic source:
@@ -476,7 +482,12 @@ create an index with logsdb index mode and synthetic source:
       indices.get_settings:
         index: "test_logsdb_index_mode_synthetic"
   - match: { test_logsdb_index_mode_synthetic.settings.index.mode: logsdb }
-  - match: { test_logsdb_index_mode_synthetic.settings.index.mapping.source.mode: synthetic }
+
+  - do:
+      indices.get_mapping:
+        index: test_logsdb_index_mode_synthetic
+
+  - match: { test_logsdb_index_mode_synthetic.mappings._source.mode: synthetic }
 
 ---
 create an index with time_series index mode and stored source:
@@ -501,9 +512,14 @@ create an index with time_series index mode and stored source:
   - do:
       indices.get_settings:
         index: "test_time_series_index_mode_undefined"
-  - match: { test_time_series_index_mode_undefined.settings.index.mode: time_series }
   - match: { test_time_series_index_mode_undefined.settings.index.mapping.source.mode: stored }
 
+  - do:
+      indices.get_mapping:
+        index: test_time_series_index_mode_undefined
+
+  - match: { test_time_series_index_mode_undefined.mappings._source.mode: stored }
+
 ---
 create an index with logsdb index mode and stored source:
   - do:
@@ -516,10 +532,10 @@ create an index with logsdb index mode and stored source:
               mapping.source.mode: stored
 
   - do:
-      indices.get_settings:
-        index: "test_logsdb_index_mode_undefined"
-  - match: { test_logsdb_index_mode_undefined.settings.index.mode: logsdb }
-  - match: { test_logsdb_index_mode_undefined.settings.index.mapping.source.mode: stored }
+      indices.get_mapping:
+        index: test_logsdb_index_mode_undefined
+
+  - match: { test_logsdb_index_mode_undefined.mappings._source.mode: stored }
 
 ---
 create an index with time_series index mode and disabled source: