|
11 | 11 | import org.elasticsearch.action.datastreams.GetDataStreamAction; |
12 | 12 | import org.elasticsearch.cluster.ClusterName; |
13 | 13 | import org.elasticsearch.cluster.ClusterState; |
| 14 | +import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; |
14 | 15 | import org.elasticsearch.cluster.metadata.DataStream; |
15 | 16 | import org.elasticsearch.cluster.metadata.DataStreamFailureStoreSettings; |
16 | 17 | import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention; |
17 | 18 | import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionSettings; |
18 | 19 | import org.elasticsearch.cluster.metadata.DataStreamTestHelper; |
| 20 | +import org.elasticsearch.cluster.metadata.IndexMetadata; |
19 | 21 | import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; |
20 | 22 | import org.elasticsearch.cluster.metadata.Metadata; |
| 23 | +import org.elasticsearch.cluster.metadata.Template; |
21 | 24 | import org.elasticsearch.common.settings.ClusterSettings; |
22 | 25 | import org.elasticsearch.common.settings.Settings; |
23 | 26 | import org.elasticsearch.core.TimeValue; |
|
26 | 29 | import org.elasticsearch.index.IndexMode; |
27 | 30 | import org.elasticsearch.index.IndexNotFoundException; |
28 | 31 | import org.elasticsearch.index.IndexSettingProviders; |
| 32 | +import org.elasticsearch.index.IndexSettings; |
29 | 33 | import org.elasticsearch.indices.SystemIndices; |
30 | 34 | import org.elasticsearch.indices.TestIndexNameExpressionResolver; |
31 | 35 | import org.elasticsearch.test.ESTestCase; |
32 | 36 |
|
33 | 37 | import java.time.Instant; |
34 | 38 | import java.time.temporal.ChronoUnit; |
| 39 | +import java.util.ArrayList; |
35 | 40 | import java.util.List; |
36 | 41 | import java.util.Set; |
| 42 | +import java.util.stream.Collectors; |
37 | 43 |
|
| 44 | +import static org.elasticsearch.cluster.metadata.DataStream.getDefaultBackingIndexName; |
| 45 | +import static org.elasticsearch.cluster.metadata.DataStreamTestHelper.createIndexMetadata; |
38 | 46 | import static org.elasticsearch.cluster.metadata.DataStreamTestHelper.getClusterStateWithDataStreams; |
39 | 47 | import static org.elasticsearch.test.LambdaMatchers.transformedItemsMatch; |
40 | 48 | import static org.elasticsearch.test.LambdaMatchers.transformedMatch; |
@@ -312,9 +320,9 @@ public void testGetTimeSeriesMixedDataStream() { |
312 | 320 | null |
313 | 321 | ); |
314 | 322 |
|
315 | | - var name1 = DataStream.getDefaultBackingIndexName("ds-1", 1, instant.toEpochMilli()); |
316 | | - var name2 = DataStream.getDefaultBackingIndexName("ds-1", 2, instant.toEpochMilli()); |
317 | | - var name3 = DataStream.getDefaultBackingIndexName("ds-1", 3, twoHoursAgo.toEpochMilli()); |
| 323 | + var name1 = getDefaultBackingIndexName("ds-1", 1, instant.toEpochMilli()); |
| 324 | + var name2 = getDefaultBackingIndexName("ds-1", 2, instant.toEpochMilli()); |
| 325 | + var name3 = getDefaultBackingIndexName("ds-1", 3, twoHoursAgo.toEpochMilli()); |
318 | 326 | assertThat( |
319 | 327 | response.getDataStreams(), |
320 | 328 | contains( |
@@ -534,4 +542,112 @@ public void testProvidersAffectMode() { |
534 | 542 | equalTo("standard") |
535 | 543 | ); |
536 | 544 | } |
| 545 | + |
| 546 | + public void testGetEffectiveSettingsTemplateOnlySettings() { |
| 547 | + // Set a lifecycle only in the template, and make sure that is in the response: |
| 548 | + GetDataStreamAction.Request req = new GetDataStreamAction.Request(TEST_REQUEST_TIMEOUT, new String[] {}); |
| 549 | + final String templatePolicy = "templatePolicy"; |
| 550 | + final String templateIndexMode = IndexMode.LOOKUP.getName(); |
| 551 | + final String dataStreamPolicy = "dataStreamPolicy"; |
| 552 | + final String dataStreamIndexMode = IndexMode.LOGSDB.getName(); |
| 553 | + |
| 554 | + ClusterState state = getClusterStateWithDataStreamWithSettings( |
| 555 | + Settings.builder() |
| 556 | + .put(IndexMetadata.LIFECYCLE_NAME, templatePolicy) |
| 557 | + .put(IndexSettings.MODE.getKey(), templateIndexMode) |
| 558 | + .build(), |
| 559 | + Settings.EMPTY |
| 560 | + ); |
| 561 | + |
| 562 | + GetDataStreamAction.Response response = TransportGetDataStreamsAction.innerOperation( |
| 563 | + state, |
| 564 | + req, |
| 565 | + resolver, |
| 566 | + systemIndices, |
| 567 | + ClusterSettings.createBuiltInClusterSettings(), |
| 568 | + dataStreamGlobalRetentionSettings, |
| 569 | + emptyDataStreamFailureStoreSettings, |
| 570 | + new IndexSettingProviders(Set.of()), |
| 571 | + null |
| 572 | + ); |
| 573 | + assertNotNull(response.getDataStreams()); |
| 574 | + assertThat(response.getDataStreams().size(), equalTo(1)); |
| 575 | + assertThat(response.getDataStreams().get(0).getIlmPolicy(), equalTo(templatePolicy)); |
| 576 | + assertThat(response.getDataStreams().get(0).getIndexModeName(), equalTo(templateIndexMode)); |
| 577 | + } |
| 578 | + |
| 579 | + public void testGetEffectiveSettings() { |
| 580 | + GetDataStreamAction.Request req = new GetDataStreamAction.Request(TEST_REQUEST_TIMEOUT, new String[] {}); |
| 581 | + final String templatePolicy = "templatePolicy"; |
| 582 | + final String templateIndexMode = IndexMode.LOOKUP.getName(); |
| 583 | + final String dataStreamPolicy = "dataStreamPolicy"; |
| 584 | + final String dataStreamIndexMode = IndexMode.LOGSDB.getName(); |
| 585 | + // Now set a lifecycle in both the template and the data stream, and make sure the response has the data stream one: |
| 586 | + ClusterState state = getClusterStateWithDataStreamWithSettings( |
| 587 | + Settings.builder() |
| 588 | + .put(IndexMetadata.LIFECYCLE_NAME, templatePolicy) |
| 589 | + .put(IndexSettings.MODE.getKey(), templateIndexMode) |
| 590 | + .build(), |
| 591 | + Settings.builder() |
| 592 | + .put(IndexMetadata.LIFECYCLE_NAME, dataStreamPolicy) |
| 593 | + .put(IndexSettings.MODE.getKey(), dataStreamIndexMode) |
| 594 | + .build() |
| 595 | + ); |
| 596 | + GetDataStreamAction.Response response = TransportGetDataStreamsAction.innerOperation( |
| 597 | + state, |
| 598 | + req, |
| 599 | + resolver, |
| 600 | + systemIndices, |
| 601 | + ClusterSettings.createBuiltInClusterSettings(), |
| 602 | + dataStreamGlobalRetentionSettings, |
| 603 | + emptyDataStreamFailureStoreSettings, |
| 604 | + new IndexSettingProviders(Set.of()), |
| 605 | + null |
| 606 | + ); |
| 607 | + assertNotNull(response.getDataStreams()); |
| 608 | + assertThat(response.getDataStreams().size(), equalTo(1)); |
| 609 | + assertThat(response.getDataStreams().get(0).getIlmPolicy(), equalTo(dataStreamPolicy)); |
| 610 | + assertThat(response.getDataStreams().get(0).getIndexModeName(), equalTo(dataStreamIndexMode)); |
| 611 | + } |
| 612 | + |
| 613 | + private static ClusterState getClusterStateWithDataStreamWithSettings(Settings templateSettings, Settings dataStreamSettings) { |
| 614 | + String dataStreamName = "data-stream-1"; |
| 615 | + int numberOfBackingIndices = randomIntBetween(1, 5); |
| 616 | + long currentTime = System.currentTimeMillis(); |
| 617 | + int replicas = 0; |
| 618 | + boolean replicated = false; |
| 619 | + Metadata.Builder builder = Metadata.builder(); |
| 620 | + builder.put( |
| 621 | + "template_1", |
| 622 | + ComposableIndexTemplate.builder() |
| 623 | + .indexPatterns(List.of("*")) |
| 624 | + .template(Template.builder().settings(templateSettings)) |
| 625 | + .dataStreamTemplate(new ComposableIndexTemplate.DataStreamTemplate()) |
| 626 | + .build() |
| 627 | + ); |
| 628 | + |
| 629 | + List<IndexMetadata> backingIndices = new ArrayList<>(); |
| 630 | + for (int backingIndexNumber = 1; backingIndexNumber <= numberOfBackingIndices; backingIndexNumber++) { |
| 631 | + backingIndices.add( |
| 632 | + createIndexMetadata( |
| 633 | + getDefaultBackingIndexName(dataStreamName, backingIndexNumber, currentTime), |
| 634 | + true, |
| 635 | + templateSettings, |
| 636 | + replicas |
| 637 | + ) |
| 638 | + ); |
| 639 | + } |
| 640 | + List<IndexMetadata> allIndices = new ArrayList<>(backingIndices); |
| 641 | + |
| 642 | + DataStream ds = DataStream.builder( |
| 643 | + dataStreamName, |
| 644 | + backingIndices.stream().map(IndexMetadata::getIndex).collect(Collectors.toList()) |
| 645 | + ).setGeneration(numberOfBackingIndices).setSettings(dataStreamSettings).setReplicated(replicated).build(); |
| 646 | + builder.put(ds); |
| 647 | + |
| 648 | + for (IndexMetadata index : allIndices) { |
| 649 | + builder.put(index, false); |
| 650 | + } |
| 651 | + return ClusterState.builder(new ClusterName("_name")).metadata(builder.build()).build(); |
| 652 | + } |
537 | 653 | } |
0 commit comments