diff --git a/modules/data-streams/src/main/java/org/elasticsearch/datastreams/action/TransportUpdateDataStreamSettingsAction.java b/modules/data-streams/src/main/java/org/elasticsearch/datastreams/action/TransportUpdateDataStreamSettingsAction.java index 86bae239bec1d..4bcbab3ad352c 100644 --- a/modules/data-streams/src/main/java/org/elasticsearch/datastreams/action/TransportUpdateDataStreamSettingsAction.java +++ b/modules/data-streams/src/main/java/org/elasticsearch/datastreams/action/TransportUpdateDataStreamSettingsAction.java @@ -54,7 +54,8 @@ public class TransportUpdateDataStreamSettingsAction extends TransportMasterNode UpdateDataStreamSettingsAction.Request, UpdateDataStreamSettingsAction.Response> { private static final Logger logger = LogManager.getLogger(TransportUpdateDataStreamSettingsAction.class); - private static final Set APPLY_TO_BACKING_INDICES = Set.of( + private static final Set APPLY_TO_WRITE_INDEX = Set.of("index.number_of_replicas"); + private static final Set APPLY_TO_ALL_BACKING_INDICES = Set.of( "index.lifecycle.name", IndexSettings.PREFER_ILM, "index.refresh_interval" @@ -158,7 +159,9 @@ private void updateSingleDataStream( logger.debug("updating settings for {}", dataStreamName); Set settingsToReject = new HashSet<>(); for (String settingName : settingsOverrides.keySet()) { - if (APPLY_TO_BACKING_INDICES.contains(settingName) == false && APPLY_TO_DATA_STREAM_ONLY.contains(settingName) == false) { + if (APPLY_TO_WRITE_INDEX.contains(settingName) == false + && APPLY_TO_ALL_BACKING_INDICES.contains(settingName) == false + && APPLY_TO_DATA_STREAM_ONLY.contains(settingName) == false) { settingsToReject.add(settingName); } } @@ -223,19 +226,26 @@ private void updateSettingsOnIndices( TimeValue ackTimeout, ActionListener listener ) { - Map settingsToApply = new HashMap<>(); + Map settingsToApplyToNonWriteIndices = new HashMap<>(); + Map settingsToApplyToWriteIndex = new HashMap<>(); List appliedToDataStreamOnly = new ArrayList<>(); + List appliedToDataStreamAndWriteIndexOnly = new ArrayList<>(); List appliedToDataStreamAndBackingIndices = new ArrayList<>(); Settings effectiveSettings = dataStream.getEffectiveSettings(projectResolver.getProjectMetadata(clusterService.state())); for (String settingName : requestSettings.keySet()) { - if (APPLY_TO_BACKING_INDICES.contains(settingName)) { - settingsToApply.put(settingName, effectiveSettings.get(settingName)); + if (APPLY_TO_WRITE_INDEX.contains(settingName)) { + settingsToApplyToWriteIndex.put(settingName, effectiveSettings.get(settingName)); + appliedToDataStreamAndWriteIndexOnly.add(settingName); + } else if (APPLY_TO_ALL_BACKING_INDICES.contains(settingName)) { + settingsToApplyToWriteIndex.put(settingName, effectiveSettings.get(settingName)); + settingsToApplyToNonWriteIndices.put(settingName, effectiveSettings.get(settingName)); appliedToDataStreamAndBackingIndices.add(settingName); } else if (APPLY_TO_DATA_STREAM_ONLY.contains(settingName)) { appliedToDataStreamOnly.add(settingName); } } final List concreteIndices = dataStream.getIndices(); + final Index writeIndex = dataStream.getWriteIndex(); final List indexSettingErrors = new ArrayList<>(); CountDownActionListener indexCountDownListener = new CountDownActionListener( @@ -250,6 +260,7 @@ private void updateSettingsOnIndices( settingsFilter.filter(effectiveSettings), new UpdateDataStreamSettingsAction.DataStreamSettingsResponse.IndicesSettingsResult( appliedToDataStreamOnly, + appliedToDataStreamAndWriteIndexOnly, appliedToDataStreamAndBackingIndices, indexSettingErrors ) @@ -259,11 +270,13 @@ private void updateSettingsOnIndices( ); indexCountDownListener.onResponse(null); // handles the case where there were zero indices - Settings applyToIndexSettings = builder().loadFromMap(settingsToApply).build(); + Settings applyToNonWriteIndexSettings = builder().loadFromMap(settingsToApplyToNonWriteIndices).build(); + Settings applyToWriteIndexSettings = builder().loadFromMap(settingsToApplyToWriteIndex).build(); for (Index index : concreteIndices) { + Settings settings = index.equals(writeIndex) ? applyToWriteIndexSettings : applyToNonWriteIndexSettings; updateSettingsOnSingleIndex( index, - applyToIndexSettings, + settings, dryRun, masterNodeTimeout, ackTimeout, diff --git a/modules/data-streams/src/yamlRestTest/resources/rest-api-spec/test/data_stream/240_data_stream_settings.yml b/modules/data-streams/src/yamlRestTest/resources/rest-api-spec/test/data_stream/240_data_stream_settings.yml index a16cd2ac236d6..0f7268975d6bb 100644 --- a/modules/data-streams/src/yamlRestTest/resources/rest-api-spec/test/data_stream/240_data_stream_settings.yml +++ b/modules/data-streams/src/yamlRestTest/resources/rest-api-spec/test/data_stream/240_data_stream_settings.yml @@ -495,3 +495,98 @@ setup: index: my-component-only-data-stream-1 - match: { .$idx0name.settings.index.lifecycle.name: "my-policy" } - match: { .$idx0name.settings.index.lifecycle.prefer_ilm: null } + +--- +"Test write index only setting": + - requires: + cluster_features: [ "logs_stream" ] + reason: requires setting 'logs_stream' to get or set data stream settings + - do: + allowed_warnings: + - "index template [my-template] has index patterns [my-data-stream-*] matching patterns from existing older templates [global] with patterns (global => [*]); this template [my-template] will take precedence during new index creation" + indices.put_index_template: + name: my-template + body: + index_patterns: [ my-data-stream-* ] + data_stream: { } + template: + settings: + number_of_replicas: 0 + lifecycle.name: my-policy + + - do: + indices.create_data_stream: + name: my-data-stream-1 + + - do: + cluster.health: + index: "my-data-stream-1" + wait_for_status: green + + - do: + indices.get_data_stream_settings: + name: my-data-stream-1 + - match: { data_streams.0.name: my-data-stream-1 } + - match: { data_streams.0.settings: {} } + - match: { data_streams.0.effective_settings.index.number_of_shards: null } + - match: { data_streams.0.effective_settings.index.number_of_replicas: "0" } + + - do: + indices.rollover: + alias: "my-data-stream-1" + + - do: + cluster.health: + index: "my-data-stream-1" + wait_for_status: green + + - do: + indices.put_data_stream_settings: + name: my-data-stream-1 + body: + index: + number_of_replicas: 1 + - match: { data_streams.0.name: my-data-stream-1 } + - match: { data_streams.0.applied_to_data_stream: true } + - match: { data_streams.0.index_settings_results.applied_to_data_stream_only: []} + - match: { data_streams.0.index_settings_results.applied_to_data_stream_and_backing_indices: []} + - length: { data_streams.0.index_settings_results.applied_to_data_stream_and_write_indices: 1 } + - match: { data_streams.0.effective_settings.index.number_of_replicas: "1" } + + - do: + indices.rollover: + alias: "my-data-stream-1" + + - do: + cluster.health: + index: "my-data-stream-1" + wait_for_status: yellow + + - do: + indices.get_data_stream_settings: + name: my-data-stream-1 + - match: { data_streams.0.name: my-data-stream-1 } + - match: { data_streams.0.effective_settings.index.number_of_replicas: "1" } + + - do: + indices.get_data_stream: + name: my-data-stream-1 + - match: { data_streams.0.name: my-data-stream-1 } + - match: { data_streams.0.settings.index.number_of_replicas: "1" } + - match: { data_streams.0.effective_settings: null } + + - do: + indices.get_data_stream: + name: my-data-stream-1 + - set: { data_streams.0.indices.0.index_name: idx0name } + - set: { data_streams.0.indices.1.index_name: idx1name } + - set: { data_streams.0.indices.2.index_name: idx2name } + + # We expect that index.number_of_replicas only gets updated on the current write index and any future indices. We have + # done one rollover since setting the setting, so it ought to be updated in the two most recent indices only: + - do: + indices.get_settings: + index: my-data-stream-1 + - match: { .$idx0name.settings.index.number_of_replicas: "0" } + - match: { .$idx1name.settings.index.number_of_replicas: "1" } + - match: { .$idx2name.settings.index.number_of_replicas: "1" } diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java index 5a026b6e1660b..e215cefde903c 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersions.java +++ b/server/src/main/java/org/elasticsearch/TransportVersions.java @@ -365,6 +365,7 @@ static TransportVersion def(int id) { public static final TransportVersion ESQL_LOOKUP_JOIN_ON_MANY_FIELDS = def(9_139_0_00); public static final TransportVersion SIMULATE_INGEST_EFFECTIVE_MAPPING = def(9_140_0_00); public static final TransportVersion RESOLVE_INDEX_MODE_ADDED = def(9_141_0_00); + public static final TransportVersion DATA_STREAM_WRITE_INDEX_ONLY_SETTINGS = def(9_142_0_00); /* * STOP! READ THIS FIRST! No, really, diff --git a/server/src/main/java/org/elasticsearch/action/datastreams/UpdateDataStreamSettingsAction.java b/server/src/main/java/org/elasticsearch/action/datastreams/UpdateDataStreamSettingsAction.java index bca0b1650581b..e2d990ba5110e 100644 --- a/server/src/main/java/org/elasticsearch/action/datastreams/UpdateDataStreamSettingsAction.java +++ b/server/src/main/java/org/elasticsearch/action/datastreams/UpdateDataStreamSettingsAction.java @@ -232,19 +232,28 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws public record IndicesSettingsResult( List appliedToDataStreamOnly, + List appliedToDataStreamAndWriteIndex, List appliedToDataStreamAndBackingIndices, List indexSettingErrors ) implements ToXContent, Writeable { - public static final IndicesSettingsResult EMPTY = new IndicesSettingsResult(List.of(), List.of(), List.of()); + public static final IndicesSettingsResult EMPTY = new IndicesSettingsResult(List.of(), List.of(), List.of(), List.of()); public IndicesSettingsResult(StreamInput in) throws IOException { - this(in.readStringCollectionAsList(), in.readStringCollectionAsList(), in.readCollectionAsList(IndexSettingError::new)); + this( + in.readStringCollectionAsList(), + in.getTransportVersion().onOrAfter(TransportVersions.DATA_STREAM_WRITE_INDEX_ONLY_SETTINGS) + ? in.readStringCollectionAsList() + : List.of(), + in.readStringCollectionAsList(), + in.readCollectionAsList(IndexSettingError::new) + ); } @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.field("applied_to_data_stream_only", appliedToDataStreamOnly); + builder.field("applied_to_data_stream_and_write_indices", appliedToDataStreamAndWriteIndex); builder.field("applied_to_data_stream_and_backing_indices", appliedToDataStreamAndBackingIndices); if (indexSettingErrors.isEmpty() == false) { builder.field("errors", indexSettingErrors); @@ -255,6 +264,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws @Override public void writeTo(StreamOutput out) throws IOException { out.writeStringCollection(appliedToDataStreamOnly); + if (out.getTransportVersion().onOrAfter(TransportVersions.DATA_STREAM_WRITE_INDEX_ONLY_SETTINGS)) { + out.writeStringCollection(appliedToDataStreamAndWriteIndex); + } out.writeStringCollection(appliedToDataStreamAndBackingIndices); out.writeCollection(indexSettingErrors, (out1, value) -> value.writeTo(out1)); } diff --git a/server/src/test/java/org/elasticsearch/action/datastreams/UpdateDataStreamSettingsActionResponseTests.java b/server/src/test/java/org/elasticsearch/action/datastreams/UpdateDataStreamSettingsActionResponseTests.java index a249703beb30b..e08c28a6639ca 100644 --- a/server/src/test/java/org/elasticsearch/action/datastreams/UpdateDataStreamSettingsActionResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/datastreams/UpdateDataStreamSettingsActionResponseTests.java @@ -39,6 +39,7 @@ public void testToXContent() throws IOException { Map dataStream1Settings = Map.of("setting1", "value1", "setting2", "value2"); Map dataStream1EffectiveSettings = Map.of("setting1", "value1", "setting2", "value2", "setting3", "value3"); List dataStream1AppliedToDataStreamOnly = randomList(10, () -> randomAlphanumericOfLength(10)); + List dataStream1AppliedToWriteIndexOnly = randomList(10, () -> randomAlphanumericOfLength(10)); List dataStream1AppliedToBackingIndices = randomList(10, () -> randomAlphanumericOfLength(10)); List dataStream1IndexErrors = randomList( 10, @@ -47,6 +48,7 @@ public void testToXContent() throws IOException { Map dataStream2Settings = Map.of("setting4", "value4", "setting5", "value5"); Map dataStream2EffectiveSettings = Map.of("setting4", "value4", "setting5", "value5", "settings6", "value6"); List dataStream2AppliedToDataStreamOnly = randomList(10, () -> randomAlphanumericOfLength(10)); + List dataStream2AppliedToWriteIndexOnly = randomList(10, () -> randomAlphanumericOfLength(10)); List dataStream2AppliedToBackingIndices = randomList(10, () -> randomAlphanumericOfLength(10)); List dataStream2IndexErrors = randomList( 10, @@ -65,6 +67,7 @@ public void testToXContent() throws IOException { Settings.builder().loadFromMap(dataStream1EffectiveSettings).build(), new UpdateDataStreamSettingsAction.DataStreamSettingsResponse.IndicesSettingsResult( dataStream1AppliedToDataStreamOnly, + dataStream1AppliedToWriteIndexOnly, dataStream1AppliedToBackingIndices, dataStream1IndexErrors ) @@ -78,6 +81,7 @@ public void testToXContent() throws IOException { Settings.builder().loadFromMap(dataStream2EffectiveSettings).build(), new UpdateDataStreamSettingsAction.DataStreamSettingsResponse.IndicesSettingsResult( dataStream2AppliedToDataStreamOnly, + dataStream2AppliedToWriteIndexOnly, dataStream2AppliedToBackingIndices, dataStream2IndexErrors ) @@ -110,6 +114,7 @@ public void testToXContent() throws IOException { dataStream1Settings, dataStream1EffectiveSettings, dataStream1AppliedToDataStreamOnly, + dataStream1AppliedToWriteIndexOnly, dataStream1AppliedToBackingIndices, dataStream1IndexErrors ), @@ -120,6 +125,7 @@ public void testToXContent() throws IOException { dataStream2Settings, dataStream2EffectiveSettings, dataStream2AppliedToDataStreamOnly, + dataStream2AppliedToWriteIndexOnly, dataStream2AppliedToBackingIndices, dataStream2IndexErrors ) @@ -137,6 +143,7 @@ private Map buildExpectedMap( Map settings, Map effectiveSettings, List appliedToDataStreamOnly, + List appliedToWriteIndexOnly, List appliedToIndices, List indexErrors ) { @@ -150,6 +157,7 @@ private Map buildExpectedMap( result.put("effective_settings", effectiveSettings); Map indexSettingsResults = new HashMap<>(); indexSettingsResults.put("applied_to_data_stream_only", appliedToDataStreamOnly); + indexSettingsResults.put("applied_to_data_stream_and_write_indices", appliedToWriteIndexOnly); indexSettingsResults.put("applied_to_data_stream_and_backing_indices", appliedToIndices); if (indexErrors.isEmpty() == false) { indexSettingsResults.put( @@ -181,6 +189,7 @@ private UpdateDataStreamSettingsAction.DataStreamSettingsResponse randomDataStre private UpdateDataStreamSettingsAction.DataStreamSettingsResponse.IndicesSettingsResult randomIndicesSettingsResult() { return new UpdateDataStreamSettingsAction.DataStreamSettingsResponse.IndicesSettingsResult( + randomList(10, () -> randomAlphanumericOfLength(20)), randomList(10, () -> randomAlphanumericOfLength(20)), randomList(10, () -> randomAlphanumericOfLength(20)), randomList(10, this::randomIndexSettingError)