Skip to content

Commit e1c2760

Browse files
authored
Adding support for index.number_of_replicas to data stream settings (#132748)
1 parent 7f5d17c commit e1c2760

File tree

5 files changed

+139
-9
lines changed

5 files changed

+139
-9
lines changed

modules/data-streams/src/main/java/org/elasticsearch/datastreams/action/TransportUpdateDataStreamSettingsAction.java

Lines changed: 20 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -54,7 +54,8 @@ public class TransportUpdateDataStreamSettingsAction extends TransportMasterNode
5454
UpdateDataStreamSettingsAction.Request,
5555
UpdateDataStreamSettingsAction.Response> {
5656
private static final Logger logger = LogManager.getLogger(TransportUpdateDataStreamSettingsAction.class);
57-
private static final Set<String> APPLY_TO_BACKING_INDICES = Set.of(
57+
private static final Set<String> APPLY_TO_WRITE_INDEX = Set.of("index.number_of_replicas");
58+
private static final Set<String> APPLY_TO_ALL_BACKING_INDICES = Set.of(
5859
"index.lifecycle.name",
5960
IndexSettings.PREFER_ILM,
6061
"index.refresh_interval"
@@ -158,7 +159,9 @@ private void updateSingleDataStream(
158159
logger.debug("updating settings for {}", dataStreamName);
159160
Set<String> settingsToReject = new HashSet<>();
160161
for (String settingName : settingsOverrides.keySet()) {
161-
if (APPLY_TO_BACKING_INDICES.contains(settingName) == false && APPLY_TO_DATA_STREAM_ONLY.contains(settingName) == false) {
162+
if (APPLY_TO_WRITE_INDEX.contains(settingName) == false
163+
&& APPLY_TO_ALL_BACKING_INDICES.contains(settingName) == false
164+
&& APPLY_TO_DATA_STREAM_ONLY.contains(settingName) == false) {
162165
settingsToReject.add(settingName);
163166
}
164167
}
@@ -223,19 +226,26 @@ private void updateSettingsOnIndices(
223226
TimeValue ackTimeout,
224227
ActionListener<UpdateDataStreamSettingsAction.DataStreamSettingsResponse> listener
225228
) {
226-
Map<String, Object> settingsToApply = new HashMap<>();
229+
Map<String, Object> settingsToApplyToNonWriteIndices = new HashMap<>();
230+
Map<String, Object> settingsToApplyToWriteIndex = new HashMap<>();
227231
List<String> appliedToDataStreamOnly = new ArrayList<>();
232+
List<String> appliedToDataStreamAndWriteIndexOnly = new ArrayList<>();
228233
List<String> appliedToDataStreamAndBackingIndices = new ArrayList<>();
229234
Settings effectiveSettings = dataStream.getEffectiveSettings(projectResolver.getProjectMetadata(clusterService.state()));
230235
for (String settingName : requestSettings.keySet()) {
231-
if (APPLY_TO_BACKING_INDICES.contains(settingName)) {
232-
settingsToApply.put(settingName, effectiveSettings.get(settingName));
236+
if (APPLY_TO_WRITE_INDEX.contains(settingName)) {
237+
settingsToApplyToWriteIndex.put(settingName, effectiveSettings.get(settingName));
238+
appliedToDataStreamAndWriteIndexOnly.add(settingName);
239+
} else if (APPLY_TO_ALL_BACKING_INDICES.contains(settingName)) {
240+
settingsToApplyToWriteIndex.put(settingName, effectiveSettings.get(settingName));
241+
settingsToApplyToNonWriteIndices.put(settingName, effectiveSettings.get(settingName));
233242
appliedToDataStreamAndBackingIndices.add(settingName);
234243
} else if (APPLY_TO_DATA_STREAM_ONLY.contains(settingName)) {
235244
appliedToDataStreamOnly.add(settingName);
236245
}
237246
}
238247
final List<Index> concreteIndices = dataStream.getIndices();
248+
final Index writeIndex = dataStream.getWriteIndex();
239249
final List<UpdateDataStreamSettingsAction.DataStreamSettingsResponse.IndexSettingError> indexSettingErrors = new ArrayList<>();
240250

241251
CountDownActionListener indexCountDownListener = new CountDownActionListener(
@@ -250,6 +260,7 @@ private void updateSettingsOnIndices(
250260
settingsFilter.filter(effectiveSettings),
251261
new UpdateDataStreamSettingsAction.DataStreamSettingsResponse.IndicesSettingsResult(
252262
appliedToDataStreamOnly,
263+
appliedToDataStreamAndWriteIndexOnly,
253264
appliedToDataStreamAndBackingIndices,
254265
indexSettingErrors
255266
)
@@ -259,11 +270,13 @@ private void updateSettingsOnIndices(
259270
);
260271

261272
indexCountDownListener.onResponse(null); // handles the case where there were zero indices
262-
Settings applyToIndexSettings = builder().loadFromMap(settingsToApply).build();
273+
Settings applyToNonWriteIndexSettings = builder().loadFromMap(settingsToApplyToNonWriteIndices).build();
274+
Settings applyToWriteIndexSettings = builder().loadFromMap(settingsToApplyToWriteIndex).build();
263275
for (Index index : concreteIndices) {
276+
Settings settings = index.equals(writeIndex) ? applyToWriteIndexSettings : applyToNonWriteIndexSettings;
264277
updateSettingsOnSingleIndex(
265278
index,
266-
applyToIndexSettings,
279+
settings,
267280
dryRun,
268281
masterNodeTimeout,
269282
ackTimeout,

modules/data-streams/src/yamlRestTest/resources/rest-api-spec/test/data_stream/240_data_stream_settings.yml

Lines changed: 95 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -495,3 +495,98 @@ setup:
495495
index: my-component-only-data-stream-1
496496
- match: { .$idx0name.settings.index.lifecycle.name: "my-policy" }
497497
- match: { .$idx0name.settings.index.lifecycle.prefer_ilm: null }
498+
499+
---
500+
"Test write index only setting":
501+
- requires:
502+
cluster_features: [ "logs_stream" ]
503+
reason: requires setting 'logs_stream' to get or set data stream settings
504+
- do:
505+
allowed_warnings:
506+
- "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"
507+
indices.put_index_template:
508+
name: my-template
509+
body:
510+
index_patterns: [ my-data-stream-* ]
511+
data_stream: { }
512+
template:
513+
settings:
514+
number_of_replicas: 0
515+
lifecycle.name: my-policy
516+
517+
- do:
518+
indices.create_data_stream:
519+
name: my-data-stream-1
520+
521+
- do:
522+
cluster.health:
523+
index: "my-data-stream-1"
524+
wait_for_status: green
525+
526+
- do:
527+
indices.get_data_stream_settings:
528+
name: my-data-stream-1
529+
- match: { data_streams.0.name: my-data-stream-1 }
530+
- match: { data_streams.0.settings: {} }
531+
- match: { data_streams.0.effective_settings.index.number_of_shards: null }
532+
- match: { data_streams.0.effective_settings.index.number_of_replicas: "0" }
533+
534+
- do:
535+
indices.rollover:
536+
alias: "my-data-stream-1"
537+
538+
- do:
539+
cluster.health:
540+
index: "my-data-stream-1"
541+
wait_for_status: green
542+
543+
- do:
544+
indices.put_data_stream_settings:
545+
name: my-data-stream-1
546+
body:
547+
index:
548+
number_of_replicas: 1
549+
- match: { data_streams.0.name: my-data-stream-1 }
550+
- match: { data_streams.0.applied_to_data_stream: true }
551+
- match: { data_streams.0.index_settings_results.applied_to_data_stream_only: []}
552+
- match: { data_streams.0.index_settings_results.applied_to_data_stream_and_backing_indices: []}
553+
- length: { data_streams.0.index_settings_results.applied_to_data_stream_and_write_indices: 1 }
554+
- match: { data_streams.0.effective_settings.index.number_of_replicas: "1" }
555+
556+
- do:
557+
indices.rollover:
558+
alias: "my-data-stream-1"
559+
560+
- do:
561+
cluster.health:
562+
index: "my-data-stream-1"
563+
wait_for_status: yellow
564+
565+
- do:
566+
indices.get_data_stream_settings:
567+
name: my-data-stream-1
568+
- match: { data_streams.0.name: my-data-stream-1 }
569+
- match: { data_streams.0.effective_settings.index.number_of_replicas: "1" }
570+
571+
- do:
572+
indices.get_data_stream:
573+
name: my-data-stream-1
574+
- match: { data_streams.0.name: my-data-stream-1 }
575+
- match: { data_streams.0.settings.index.number_of_replicas: "1" }
576+
- match: { data_streams.0.effective_settings: null }
577+
578+
- do:
579+
indices.get_data_stream:
580+
name: my-data-stream-1
581+
- set: { data_streams.0.indices.0.index_name: idx0name }
582+
- set: { data_streams.0.indices.1.index_name: idx1name }
583+
- set: { data_streams.0.indices.2.index_name: idx2name }
584+
585+
# We expect that index.number_of_replicas only gets updated on the current write index and any future indices. We have
586+
# done one rollover since setting the setting, so it ought to be updated in the two most recent indices only:
587+
- do:
588+
indices.get_settings:
589+
index: my-data-stream-1
590+
- match: { .$idx0name.settings.index.number_of_replicas: "0" }
591+
- match: { .$idx1name.settings.index.number_of_replicas: "1" }
592+
- match: { .$idx2name.settings.index.number_of_replicas: "1" }

server/src/main/java/org/elasticsearch/TransportVersions.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -365,6 +365,7 @@ static TransportVersion def(int id) {
365365
public static final TransportVersion ESQL_LOOKUP_JOIN_ON_MANY_FIELDS = def(9_139_0_00);
366366
public static final TransportVersion SIMULATE_INGEST_EFFECTIVE_MAPPING = def(9_140_0_00);
367367
public static final TransportVersion RESOLVE_INDEX_MODE_ADDED = def(9_141_0_00);
368+
public static final TransportVersion DATA_STREAM_WRITE_INDEX_ONLY_SETTINGS = def(9_142_0_00);
368369

369370
/*
370371
* STOP! READ THIS FIRST! No, really,

server/src/main/java/org/elasticsearch/action/datastreams/UpdateDataStreamSettingsAction.java

Lines changed: 14 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -232,19 +232,28 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws
232232

233233
public record IndicesSettingsResult(
234234
List<String> appliedToDataStreamOnly,
235+
List<String> appliedToDataStreamAndWriteIndex,
235236
List<String> appliedToDataStreamAndBackingIndices,
236237
List<IndexSettingError> indexSettingErrors
237238
) implements ToXContent, Writeable {
238239

239-
public static final IndicesSettingsResult EMPTY = new IndicesSettingsResult(List.of(), List.of(), List.of());
240+
public static final IndicesSettingsResult EMPTY = new IndicesSettingsResult(List.of(), List.of(), List.of(), List.of());
240241

241242
public IndicesSettingsResult(StreamInput in) throws IOException {
242-
this(in.readStringCollectionAsList(), in.readStringCollectionAsList(), in.readCollectionAsList(IndexSettingError::new));
243+
this(
244+
in.readStringCollectionAsList(),
245+
in.getTransportVersion().onOrAfter(TransportVersions.DATA_STREAM_WRITE_INDEX_ONLY_SETTINGS)
246+
? in.readStringCollectionAsList()
247+
: List.of(),
248+
in.readStringCollectionAsList(),
249+
in.readCollectionAsList(IndexSettingError::new)
250+
);
243251
}
244252

245253
@Override
246254
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
247255
builder.field("applied_to_data_stream_only", appliedToDataStreamOnly);
256+
builder.field("applied_to_data_stream_and_write_indices", appliedToDataStreamAndWriteIndex);
248257
builder.field("applied_to_data_stream_and_backing_indices", appliedToDataStreamAndBackingIndices);
249258
if (indexSettingErrors.isEmpty() == false) {
250259
builder.field("errors", indexSettingErrors);
@@ -255,6 +264,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws
255264
@Override
256265
public void writeTo(StreamOutput out) throws IOException {
257266
out.writeStringCollection(appliedToDataStreamOnly);
267+
if (out.getTransportVersion().onOrAfter(TransportVersions.DATA_STREAM_WRITE_INDEX_ONLY_SETTINGS)) {
268+
out.writeStringCollection(appliedToDataStreamAndWriteIndex);
269+
}
258270
out.writeStringCollection(appliedToDataStreamAndBackingIndices);
259271
out.writeCollection(indexSettingErrors, (out1, value) -> value.writeTo(out1));
260272
}

server/src/test/java/org/elasticsearch/action/datastreams/UpdateDataStreamSettingsActionResponseTests.java

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -39,6 +39,7 @@ public void testToXContent() throws IOException {
3939
Map<String, String> dataStream1Settings = Map.of("setting1", "value1", "setting2", "value2");
4040
Map<String, String> dataStream1EffectiveSettings = Map.of("setting1", "value1", "setting2", "value2", "setting3", "value3");
4141
List<String> dataStream1AppliedToDataStreamOnly = randomList(10, () -> randomAlphanumericOfLength(10));
42+
List<String> dataStream1AppliedToWriteIndexOnly = randomList(10, () -> randomAlphanumericOfLength(10));
4243
List<String> dataStream1AppliedToBackingIndices = randomList(10, () -> randomAlphanumericOfLength(10));
4344
List<IndexSettingError> dataStream1IndexErrors = randomList(
4445
10,
@@ -47,6 +48,7 @@ public void testToXContent() throws IOException {
4748
Map<String, String> dataStream2Settings = Map.of("setting4", "value4", "setting5", "value5");
4849
Map<String, String> dataStream2EffectiveSettings = Map.of("setting4", "value4", "setting5", "value5", "settings6", "value6");
4950
List<String> dataStream2AppliedToDataStreamOnly = randomList(10, () -> randomAlphanumericOfLength(10));
51+
List<String> dataStream2AppliedToWriteIndexOnly = randomList(10, () -> randomAlphanumericOfLength(10));
5052
List<String> dataStream2AppliedToBackingIndices = randomList(10, () -> randomAlphanumericOfLength(10));
5153
List<IndexSettingError> dataStream2IndexErrors = randomList(
5254
10,
@@ -65,6 +67,7 @@ public void testToXContent() throws IOException {
6567
Settings.builder().loadFromMap(dataStream1EffectiveSettings).build(),
6668
new UpdateDataStreamSettingsAction.DataStreamSettingsResponse.IndicesSettingsResult(
6769
dataStream1AppliedToDataStreamOnly,
70+
dataStream1AppliedToWriteIndexOnly,
6871
dataStream1AppliedToBackingIndices,
6972
dataStream1IndexErrors
7073
)
@@ -78,6 +81,7 @@ public void testToXContent() throws IOException {
7881
Settings.builder().loadFromMap(dataStream2EffectiveSettings).build(),
7982
new UpdateDataStreamSettingsAction.DataStreamSettingsResponse.IndicesSettingsResult(
8083
dataStream2AppliedToDataStreamOnly,
84+
dataStream2AppliedToWriteIndexOnly,
8185
dataStream2AppliedToBackingIndices,
8286
dataStream2IndexErrors
8387
)
@@ -110,6 +114,7 @@ public void testToXContent() throws IOException {
110114
dataStream1Settings,
111115
dataStream1EffectiveSettings,
112116
dataStream1AppliedToDataStreamOnly,
117+
dataStream1AppliedToWriteIndexOnly,
113118
dataStream1AppliedToBackingIndices,
114119
dataStream1IndexErrors
115120
),
@@ -120,6 +125,7 @@ public void testToXContent() throws IOException {
120125
dataStream2Settings,
121126
dataStream2EffectiveSettings,
122127
dataStream2AppliedToDataStreamOnly,
128+
dataStream2AppliedToWriteIndexOnly,
123129
dataStream2AppliedToBackingIndices,
124130
dataStream2IndexErrors
125131
)
@@ -137,6 +143,7 @@ private Map<String, Object> buildExpectedMap(
137143
Map<String, String> settings,
138144
Map<String, String> effectiveSettings,
139145
List<String> appliedToDataStreamOnly,
146+
List<String> appliedToWriteIndexOnly,
140147
List<String> appliedToIndices,
141148
List<IndexSettingError> indexErrors
142149
) {
@@ -150,6 +157,7 @@ private Map<String, Object> buildExpectedMap(
150157
result.put("effective_settings", effectiveSettings);
151158
Map<String, Object> indexSettingsResults = new HashMap<>();
152159
indexSettingsResults.put("applied_to_data_stream_only", appliedToDataStreamOnly);
160+
indexSettingsResults.put("applied_to_data_stream_and_write_indices", appliedToWriteIndexOnly);
153161
indexSettingsResults.put("applied_to_data_stream_and_backing_indices", appliedToIndices);
154162
if (indexErrors.isEmpty() == false) {
155163
indexSettingsResults.put(
@@ -181,6 +189,7 @@ private UpdateDataStreamSettingsAction.DataStreamSettingsResponse randomDataStre
181189

182190
private UpdateDataStreamSettingsAction.DataStreamSettingsResponse.IndicesSettingsResult randomIndicesSettingsResult() {
183191
return new UpdateDataStreamSettingsAction.DataStreamSettingsResponse.IndicesSettingsResult(
192+
randomList(10, () -> randomAlphanumericOfLength(20)),
184193
randomList(10, () -> randomAlphanumericOfLength(20)),
185194
randomList(10, () -> randomAlphanumericOfLength(20)),
186195
randomList(10, this::randomIndexSettingError)

0 commit comments

Comments
 (0)