[Failure store] Introduce dedicated failure store lifecycle configuration (#127314)
The failure store is a set of data stream indices that are used to store certain type of ingestion failures. Until this moment they were sharing the configuration of the backing indices. We understand that the two data sets have different lifecycle needs. We believe that typically the failures will need to be retained much less than the data. Considering this we believe the lifecycle needs of the failures also more limited and they fit better the simplicity of the data stream lifecycle feature. This allows the user to only set the desired retention and we will perform the rollover and other maintenance tasks without the user having to think about them. Furthermore, having only one lifecycle management feature allows us to ensure that these data is managed by default. This PR introduces the following: Configuration We extend the failure store configuration to allow lifecycle configuration too, this configuration reflects the user's configuration only as shown below: PUT _data_stream/*/options { "failure_store": { "lifecycle": { "data_retention": "5d" } } } GET _data_stream/*/options { "data_streams": [ { "name": "my-ds", "options": { "failure_store": { "lifecycle": { "data_retention": "5d" } } } } ] } To retrieve the effective configuration you need to use the GET data streams API, see #126668 Functionality The data stream lifecycle (DLM) will manage the failure indices regardless if the failure store is enabled or not. This will ensure that if the failure store gets disabled we will not have stagnant data. The data stream options APIs reflect only the user's configuration. The GET data stream API should be used to check the current state of the effective failure store configuration. Telemetry We extend the data stream failure store telemetry to also include the lifecycle telemetry. { "data_streams": { "available": true, "enabled": true, "data_streams": 10, "indices_count": 50, "failure_store": { "explicitly_enabled_count": 1, "effectively_enabled_count": 15, "failure_indices_count": 30 "lifecycle": { "explicitly_enabled_count": 5, "effectively_enabled_count": 20, "data_retention": { "configured_data_streams": 5, "minimum_millis": X, "maximum_millis": Y, "average_millis": Z, }, "effective_retention": { "retained_data_streams": 20, "minimum_millis": X, "maximum_millis": Y, "average_millis": Z }, "global_retention": { "max": { "defined": false }, "default": { "defined": true, <------ this is the default value applicable for the failure store "millis": X } } } } } Implementation details We ensure that partially reset failure store will create valid failure store configuration. We ensure that when a node communicates with a note with a previous version it will ensure it will not send an invalid failure store configuration enabled: null.
This commit is contained in:
parent
f56de526bd
commit
03d77816cf
|
@ -0,0 +1,5 @@
|
|||
pr: 127314
|
||||
summary: "[Failure store] Introduce dedicated failure store lifecycle configuration"
|
||||
area: Data streams
|
||||
type: enhancement
|
||||
issues: []
|
|
@ -78,6 +78,7 @@ tasks.named("yamlRestCompatTestTransform").configure({ task ->
|
|||
task.skipTest("data_stream/210_rollover_failure_store/Lazily roll over a data stream's failure store after an ingest failure", "Rolling over a data stream using target_failure_store is no longer supported.")
|
||||
task.skipTest("data_stream/210_rollover_failure_store/Lazily roll over a data stream's failure store after a shard failure", "Rolling over a data stream using target_failure_store is no longer supported.")
|
||||
task.skipTest("data_stream/210_rollover_failure_store/Roll over a data stream's failure store without conditions", "Rolling over a data stream using target_failure_store is no longer supported.")
|
||||
task.skipTest("data_stream/240_failure_store_info/Get failure store info from explicitly enabled failure store and disabled lifecycle", "failure store lifecycle is not using anymore the data stream lifecycle")
|
||||
})
|
||||
|
||||
configurations {
|
||||
|
|
|
@ -28,19 +28,20 @@ import org.elasticsearch.action.datastreams.CreateDataStreamAction;
|
|||
import org.elasticsearch.action.datastreams.DeleteDataStreamAction;
|
||||
import org.elasticsearch.action.datastreams.GetDataStreamAction;
|
||||
import org.elasticsearch.action.datastreams.ModifyDataStreamsAction;
|
||||
import org.elasticsearch.action.datastreams.PutDataStreamOptionsAction;
|
||||
import org.elasticsearch.action.datastreams.lifecycle.ErrorEntry;
|
||||
import org.elasticsearch.action.datastreams.lifecycle.ExplainDataStreamLifecycleAction;
|
||||
import org.elasticsearch.action.datastreams.lifecycle.ExplainIndexDataStreamLifecycle;
|
||||
import org.elasticsearch.action.datastreams.lifecycle.PutDataStreamLifecycleAction;
|
||||
import org.elasticsearch.action.downsample.DownsampleConfig;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
|
||||
import org.elasticsearch.cluster.coordination.StableMasterHealthIndicatorService;
|
||||
import org.elasticsearch.cluster.metadata.ComposableIndexTemplate;
|
||||
import org.elasticsearch.cluster.metadata.DataStream;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamAction;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamFailureStore;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamLifecycle;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamTestHelper;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamOptions;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetadata;
|
||||
import org.elasticsearch.cluster.metadata.Metadata;
|
||||
import org.elasticsearch.cluster.metadata.Template;
|
||||
|
@ -71,7 +72,6 @@ import org.elasticsearch.indices.SystemDataStreamDescriptor;
|
|||
import org.elasticsearch.plugins.Plugin;
|
||||
import org.elasticsearch.plugins.SystemIndexPlugin;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
import org.elasticsearch.test.transport.MockTransportService;
|
||||
import org.elasticsearch.xcontent.ToXContent;
|
||||
|
@ -893,17 +893,8 @@ public class DataStreamLifecycleServiceIT extends ESIntegTestCase {
|
|||
}
|
||||
|
||||
public void testLifecycleAppliedToFailureStore() throws Exception {
|
||||
// We configure a lifecycle with downsampling to ensure it doesn't fail
|
||||
DataStreamLifecycle.Template lifecycle = DataStreamLifecycle.dataLifecycleBuilder()
|
||||
DataStreamLifecycle.Template lifecycle = DataStreamLifecycle.failuresLifecycleBuilder()
|
||||
.dataRetention(TimeValue.timeValueSeconds(20))
|
||||
.downsampling(
|
||||
List.of(
|
||||
new DataStreamLifecycle.DownsamplingRound(
|
||||
TimeValue.timeValueMillis(10),
|
||||
new DownsampleConfig(new DateHistogramInterval("10m"))
|
||||
)
|
||||
)
|
||||
)
|
||||
.buildTemplate();
|
||||
|
||||
putComposableIndexTemplate("id1", """
|
||||
|
@ -917,7 +908,7 @@ public class DataStreamLifecycleServiceIT extends ESIntegTestCase {
|
|||
"type": "boolean"
|
||||
}
|
||||
}
|
||||
}""", List.of("metrics-fs*"), Settings.builder().put("index.number_of_replicas", 0).build(), null, lifecycle, true);
|
||||
}""", List.of("metrics-fs*"), Settings.builder().put("index.number_of_replicas", 0).build(), null, null, lifecycle, true);
|
||||
|
||||
String dataStreamName = "metrics-fs";
|
||||
CreateDataStreamAction.Request createDataStreamRequest = new CreateDataStreamAction.Request(
|
||||
|
@ -958,7 +949,7 @@ public class DataStreamLifecycleServiceIT extends ESIntegTestCase {
|
|||
);
|
||||
});
|
||||
|
||||
updateLifecycle(dataStreamName, TimeValue.timeValueSeconds(1));
|
||||
updateFailureStoreConfiguration(dataStreamName, true, TimeValue.timeValueSeconds(1));
|
||||
|
||||
// And finally apply retention
|
||||
assertBusy(() -> {
|
||||
|
@ -1027,7 +1018,20 @@ public class DataStreamLifecycleServiceIT extends ESIntegTestCase {
|
|||
List<String> patterns,
|
||||
@Nullable Settings settings,
|
||||
@Nullable Map<String, Object> metadata,
|
||||
@Nullable DataStreamLifecycle.Template lifecycle,
|
||||
@Nullable DataStreamLifecycle.Template dataLifecycle,
|
||||
boolean withFailureStore
|
||||
) throws IOException {
|
||||
putComposableIndexTemplate(id, mappings, patterns, settings, metadata, dataLifecycle, null, withFailureStore);
|
||||
}
|
||||
|
||||
static void putComposableIndexTemplate(
|
||||
String id,
|
||||
@Nullable String mappings,
|
||||
List<String> patterns,
|
||||
@Nullable Settings settings,
|
||||
@Nullable Map<String, Object> metadata,
|
||||
@Nullable DataStreamLifecycle.Template dataLifecycle,
|
||||
@Nullable DataStreamLifecycle.Template failuresLifecycle,
|
||||
boolean withFailureStore
|
||||
) throws IOException {
|
||||
TransportPutComposableIndexTemplateAction.Request request = new TransportPutComposableIndexTemplateAction.Request(id);
|
||||
|
@ -1038,8 +1042,10 @@ public class DataStreamLifecycleServiceIT extends ESIntegTestCase {
|
|||
Template.builder()
|
||||
.settings(settings)
|
||||
.mappings(mappings == null ? null : CompressedXContent.fromJSON(mappings))
|
||||
.lifecycle(lifecycle)
|
||||
.dataStreamOptions(DataStreamTestHelper.createDataStreamOptionsTemplate(withFailureStore))
|
||||
.lifecycle(dataLifecycle)
|
||||
.dataStreamOptions(
|
||||
new DataStreamOptions.Template(new DataStreamFailureStore.Template(withFailureStore, failuresLifecycle))
|
||||
)
|
||||
)
|
||||
.metadata(metadata)
|
||||
.dataStreamTemplate(new ComposableIndexTemplate.DataStreamTemplate())
|
||||
|
@ -1058,6 +1064,16 @@ public class DataStreamLifecycleServiceIT extends ESIntegTestCase {
|
|||
assertAcked(client().execute(PutDataStreamLifecycleAction.INSTANCE, putDataLifecycleRequest));
|
||||
}
|
||||
|
||||
static void updateFailureStoreConfiguration(String dataStreamName, boolean enabled, TimeValue retention) {
|
||||
PutDataStreamOptionsAction.Request putDataOptionsRequest = new PutDataStreamOptionsAction.Request(
|
||||
TEST_REQUEST_TIMEOUT,
|
||||
TEST_REQUEST_TIMEOUT,
|
||||
new String[] { dataStreamName },
|
||||
new DataStreamFailureStore(enabled, DataStreamLifecycle.failuresLifecycleBuilder().dataRetention(retention).build())
|
||||
);
|
||||
assertAcked(client().execute(PutDataStreamOptionsAction.INSTANCE, putDataOptionsRequest));
|
||||
}
|
||||
|
||||
/*
|
||||
* This test plugin adds `.system-test` as a known system data stream. The data stream is not created by this plugin. But if it is
|
||||
* created, it will be a system data stream.
|
||||
|
|
|
@ -264,7 +264,7 @@ public class ExplainDataStreamLifecycleIT extends ESIntegTestCase {
|
|||
List.of("metrics-foo*"),
|
||||
null,
|
||||
null,
|
||||
DataStreamLifecycle.Template.DATA_DEFAULT,
|
||||
null,
|
||||
new DataStreamOptions.Template(DataStreamFailureStore.builder().enabled(true).buildTemplate())
|
||||
);
|
||||
String dataStreamName = "metrics-foo";
|
||||
|
@ -354,26 +354,30 @@ public class ExplainDataStreamLifecycleIT extends ESIntegTestCase {
|
|||
).actionGet();
|
||||
assertThat(response.getIndices().size(), is(1));
|
||||
for (ExplainIndexDataStreamLifecycle explainIndex : response.getIndices()) {
|
||||
assertThat(explainIndex.isManagedByLifecycle(), is(true));
|
||||
assertThat(explainIndex.getIndexCreationDate(), notNullValue());
|
||||
assertThat(explainIndex.getLifecycle(), notNullValue());
|
||||
assertThat(explainIndex.getLifecycle().dataRetention(), nullValue());
|
||||
|
||||
if (internalCluster().numDataNodes() > 1) {
|
||||
// If the number of nodes is 1 then the cluster will be yellow so forcemerge will report an error if it has run
|
||||
assertThat(explainIndex.getError(), nullValue());
|
||||
}
|
||||
|
||||
if (explainIndex.getIndex().equals(firstGenerationIndex)) {
|
||||
// first generation index was rolled over
|
||||
assertThat(explainIndex.getRolloverDate(), notNullValue());
|
||||
assertThat(explainIndex.getTimeSinceRollover(System::currentTimeMillis), notNullValue());
|
||||
assertThat(explainIndex.getGenerationTime(System::currentTimeMillis), notNullValue());
|
||||
if (explainIndex.getIndex().startsWith(DataStream.BACKING_INDEX_PREFIX)) {
|
||||
assertThat(explainIndex.isManagedByLifecycle(), is(false));
|
||||
} else {
|
||||
// the write index has not been rolled over yet
|
||||
assertThat(explainIndex.getRolloverDate(), nullValue());
|
||||
assertThat(explainIndex.getTimeSinceRollover(System::currentTimeMillis), nullValue());
|
||||
assertThat(explainIndex.getGenerationTime(System::currentTimeMillis), nullValue());
|
||||
assertThat(explainIndex.isManagedByLifecycle(), is(true));
|
||||
assertThat(explainIndex.getIndexCreationDate(), notNullValue());
|
||||
assertThat(explainIndex.getLifecycle(), notNullValue());
|
||||
assertThat(explainIndex.getLifecycle().dataRetention(), nullValue());
|
||||
|
||||
if (internalCluster().numDataNodes() > 1) {
|
||||
// If the number of nodes is 1 then the cluster will be yellow so forcemerge will report an error if it has run
|
||||
assertThat(explainIndex.getError(), nullValue());
|
||||
}
|
||||
|
||||
if (explainIndex.getIndex().equals(firstGenerationIndex)) {
|
||||
// first generation index was rolled over
|
||||
assertThat(explainIndex.getRolloverDate(), notNullValue());
|
||||
assertThat(explainIndex.getTimeSinceRollover(System::currentTimeMillis), notNullValue());
|
||||
assertThat(explainIndex.getGenerationTime(System::currentTimeMillis), notNullValue());
|
||||
} else {
|
||||
// the write index has not been rolled over yet
|
||||
assertThat(explainIndex.getRolloverDate(), nullValue());
|
||||
assertThat(explainIndex.getTimeSinceRollover(System::currentTimeMillis), nullValue());
|
||||
assertThat(explainIndex.getGenerationTime(System::currentTimeMillis), nullValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -42,16 +42,35 @@ public class DataStreamGlobalRetentionIT extends DisabledSecurityDataStreamTestC
|
|||
"index_patterns": ["my-data-stream*"],
|
||||
"data_stream": {},
|
||||
"template": {
|
||||
"lifecycle": {}
|
||||
"settings": {
|
||||
"number_of_replicas": 0
|
||||
},
|
||||
"mappings": {
|
||||
"properties": {
|
||||
"count": {
|
||||
"type": "long"
|
||||
}
|
||||
}
|
||||
},
|
||||
"lifecycle": {},
|
||||
"data_stream_options": {
|
||||
"failure_store": {
|
||||
"enabled": true
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
""");
|
||||
assertOK(client().performRequest(putComposableIndexTemplateRequest));
|
||||
|
||||
// Create a data streams with one doc
|
||||
// Index one doc, this will trigger a rollover
|
||||
Request createDocRequest = new Request("POST", "/my-data-stream/_doc?refresh=true");
|
||||
createDocRequest.setJsonEntity("{ \"@timestamp\": \"2022-12-12\"}");
|
||||
assertOK(client().performRequest(createDocRequest));
|
||||
// Index one doc that will fail, this will create the failure store
|
||||
createDocRequest = new Request("POST", "/my-data-stream/_doc?refresh=true");
|
||||
createDocRequest.setJsonEntity("{ \"@timestamp\": \"2022-12-12\", \"count\": \"not-a-number\"}");
|
||||
assertOK(client().performRequest(createDocRequest));
|
||||
}
|
||||
|
||||
@After
|
||||
|
@ -64,7 +83,7 @@ public class DataStreamGlobalRetentionIT extends DisabledSecurityDataStreamTestC
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testDataStreamRetention() throws Exception {
|
||||
// Set global retention and add retention to the data stream
|
||||
// Set global retention and add retention to the data stream & failure store
|
||||
{
|
||||
updateClusterSettings(
|
||||
Settings.builder()
|
||||
|
@ -78,6 +97,18 @@ public class DataStreamGlobalRetentionIT extends DisabledSecurityDataStreamTestC
|
|||
"data_retention": "10s"
|
||||
}""");
|
||||
assertAcknowledged(client().performRequest(request));
|
||||
|
||||
request = new Request("PUT", "_data_stream/my-data-stream/_options");
|
||||
request.setJsonEntity("""
|
||||
{
|
||||
"failure_store": {
|
||||
"enabled": true,
|
||||
"lifecycle": {
|
||||
"data_retention": "10s"
|
||||
}
|
||||
}
|
||||
}""");
|
||||
assertAcknowledged(client().performRequest(request));
|
||||
}
|
||||
|
||||
// Verify that the effective retention matches the default retention
|
||||
|
@ -92,6 +123,10 @@ public class DataStreamGlobalRetentionIT extends DisabledSecurityDataStreamTestC
|
|||
assertThat(lifecycle.get("effective_retention"), is("10s"));
|
||||
assertThat(lifecycle.get("retention_determined_by"), is("data_stream_configuration"));
|
||||
assertThat(lifecycle.get("data_retention"), is("10s"));
|
||||
Map<String, Object> failuresLifecycle = ((Map<String, Map<String, Object>>) dataStream.get("failure_store")).get("lifecycle");
|
||||
assertThat(failuresLifecycle.get("effective_retention"), is("10s"));
|
||||
assertThat(failuresLifecycle.get("retention_determined_by"), is("data_stream_configuration"));
|
||||
assertThat(failuresLifecycle.get("data_retention"), is("10s"));
|
||||
}
|
||||
|
||||
// Verify that the first generation index was removed
|
||||
|
@ -101,8 +136,11 @@ public class DataStreamGlobalRetentionIT extends DisabledSecurityDataStreamTestC
|
|||
assertThat(dataStream.get("name"), is("my-data-stream"));
|
||||
List<Object> backingIndices = (List<Object>) dataStream.get("indices");
|
||||
assertThat(backingIndices.size(), is(1));
|
||||
List<Object> failureIndices = (List<Object>) ((Map<String, Object>) dataStream.get("failure_store")).get("indices");
|
||||
assertThat(failureIndices.size(), is(1));
|
||||
// 2 backing indices created + 1 for the deleted index
|
||||
assertThat(dataStream.get("generation"), is(3));
|
||||
// 2 failure indices created + 1 for the deleted failure index
|
||||
assertThat(dataStream.get("generation"), is(6));
|
||||
}, 20, TimeUnit.SECONDS);
|
||||
}
|
||||
|
||||
|
@ -123,6 +161,10 @@ public class DataStreamGlobalRetentionIT extends DisabledSecurityDataStreamTestC
|
|||
assertThat(lifecycle.get("effective_retention"), is("10s"));
|
||||
assertThat(lifecycle.get("retention_determined_by"), is("default_global_retention"));
|
||||
assertThat(lifecycle.get("data_retention"), nullValue());
|
||||
Map<String, Object> failuresLifecycle = ((Map<String, Map<String, Object>>) dataStream.get("failure_store")).get("lifecycle");
|
||||
assertThat(failuresLifecycle.get("effective_retention"), is("10s"));
|
||||
assertThat(failuresLifecycle.get("retention_determined_by"), is("default_global_retention"));
|
||||
assertThat(failuresLifecycle.get("data_retention"), nullValue());
|
||||
}
|
||||
|
||||
// Verify that the first generation index was removed
|
||||
|
@ -132,8 +174,11 @@ public class DataStreamGlobalRetentionIT extends DisabledSecurityDataStreamTestC
|
|||
assertThat(dataStream.get("name"), is("my-data-stream"));
|
||||
List<Object> backingIndices = (List<Object>) dataStream.get("indices");
|
||||
assertThat(backingIndices.size(), is(1));
|
||||
List<Object> failureIndices = (List<Object>) ((Map<String, Object>) dataStream.get("failure_store")).get("indices");
|
||||
assertThat(failureIndices.size(), is(1));
|
||||
// 2 backing indices created + 1 for the deleted index
|
||||
assertThat(dataStream.get("generation"), is(3));
|
||||
// 2 failure indices created + 1 for the deleted failure index
|
||||
assertThat(dataStream.get("generation"), is(6));
|
||||
}, 20, TimeUnit.SECONDS);
|
||||
}
|
||||
|
||||
|
@ -157,6 +202,24 @@ public class DataStreamGlobalRetentionIT extends DisabledSecurityDataStreamTestC
|
|||
containsString("The retention provided [30d] is exceeding the max allowed data retention of this project [10s]")
|
||||
);
|
||||
}
|
||||
try {
|
||||
Request request = new Request("PUT", "_data_stream/my-data-stream/_options");
|
||||
request.setJsonEntity("""
|
||||
{
|
||||
"failure_store": {
|
||||
"lifecycle": {
|
||||
"data_retention": "30d"
|
||||
}
|
||||
}
|
||||
}""");
|
||||
assertAcknowledged(client().performRequest(request));
|
||||
fail("Should have returned a warning about data retention exceeding the max retention");
|
||||
} catch (WarningFailureException warningFailureException) {
|
||||
assertThat(
|
||||
warningFailureException.getMessage(),
|
||||
containsString("The retention provided [30d] is exceeding the max allowed data retention of this project [10s]")
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
// Verify that the effective retention matches the max retention
|
||||
|
@ -175,6 +238,14 @@ public class DataStreamGlobalRetentionIT extends DisabledSecurityDataStreamTestC
|
|||
} else {
|
||||
assertThat(lifecycle.get("data_retention"), nullValue());
|
||||
}
|
||||
Map<String, Object> failuresLifecycle = ((Map<String, Map<String, Object>>) dataStream.get("failure_store")).get("lifecycle");
|
||||
assertThat(failuresLifecycle.get("effective_retention"), is("10s"));
|
||||
assertThat(failuresLifecycle.get("retention_determined_by"), is("max_global_retention"));
|
||||
if (withDataStreamLevelRetention) {
|
||||
assertThat(failuresLifecycle.get("data_retention"), is("30d"));
|
||||
} else {
|
||||
assertThat(failuresLifecycle.get("data_retention"), nullValue());
|
||||
}
|
||||
}
|
||||
|
||||
// Verify that the first generation index was removed
|
||||
|
@ -184,8 +255,11 @@ public class DataStreamGlobalRetentionIT extends DisabledSecurityDataStreamTestC
|
|||
assertThat(dataStream.get("name"), is("my-data-stream"));
|
||||
List<Object> backingIndices = (List<Object>) dataStream.get("indices");
|
||||
assertThat(backingIndices.size(), is(1));
|
||||
List<Object> failureIndices = (List<Object>) ((Map<String, Object>) dataStream.get("failure_store")).get("indices");
|
||||
assertThat(failureIndices.size(), is(1));
|
||||
// 2 backing indices created + 1 for the deleted index
|
||||
assertThat(dataStream.get("generation"), is(3));
|
||||
// 2 failure indices created + 1 for the deleted failure index
|
||||
assertThat(dataStream.get("generation"), is(6));
|
||||
}, 20, TimeUnit.SECONDS);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,6 +18,7 @@ import org.elasticsearch.action.datastreams.GetDataStreamSettingsAction;
|
|||
import org.elasticsearch.action.datastreams.MigrateToDataStreamAction;
|
||||
import org.elasticsearch.action.datastreams.ModifyDataStreamsAction;
|
||||
import org.elasticsearch.action.datastreams.PromoteDataStreamAction;
|
||||
import org.elasticsearch.action.datastreams.PutDataStreamOptionsAction;
|
||||
import org.elasticsearch.action.datastreams.UpdateDataStreamSettingsAction;
|
||||
import org.elasticsearch.action.datastreams.lifecycle.ExplainDataStreamLifecycleAction;
|
||||
import org.elasticsearch.action.datastreams.lifecycle.GetDataStreamLifecycleAction;
|
||||
|
@ -60,7 +61,6 @@ import org.elasticsearch.datastreams.lifecycle.rest.RestGetDataStreamLifecycleAc
|
|||
import org.elasticsearch.datastreams.lifecycle.rest.RestPutDataStreamLifecycleAction;
|
||||
import org.elasticsearch.datastreams.options.action.DeleteDataStreamOptionsAction;
|
||||
import org.elasticsearch.datastreams.options.action.GetDataStreamOptionsAction;
|
||||
import org.elasticsearch.datastreams.options.action.PutDataStreamOptionsAction;
|
||||
import org.elasticsearch.datastreams.options.action.TransportDeleteDataStreamOptionsAction;
|
||||
import org.elasticsearch.datastreams.options.action.TransportGetDataStreamOptionsAction;
|
||||
import org.elasticsearch.datastreams.options.action.TransportPutDataStreamOptionsAction;
|
||||
|
|
|
@ -365,7 +365,10 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
|
|||
int affectedDataStreams = 0;
|
||||
for (DataStream dataStream : project.dataStreams().values()) {
|
||||
clearErrorStoreForUnmanagedIndices(project, dataStream);
|
||||
if (dataStream.getDataLifecycle() == null) {
|
||||
var dataLifecycleEnabled = dataStream.getDataLifecycle() != null && dataStream.getDataLifecycle().enabled();
|
||||
var failureLifecycle = dataStream.getFailuresLifecycle();
|
||||
var failuresLifecycleEnabled = failureLifecycle != null && failureLifecycle.enabled();
|
||||
if (dataLifecycleEnabled == false && failuresLifecycleEnabled == false) {
|
||||
continue;
|
||||
}
|
||||
|
||||
|
@ -902,23 +905,29 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
|
|||
*/
|
||||
Set<Index> maybeExecuteRetention(ProjectMetadata project, DataStream dataStream, Set<Index> indicesToExcludeForRemainingRun) {
|
||||
DataStreamGlobalRetention globalRetention = dataStream.isSystem() ? null : globalRetentionSettings.get();
|
||||
List<Index> backingIndicesOlderThanRetention = dataStream.getBackingIndicesPastRetention(
|
||||
var dataRetention = getRetention(dataStream, globalRetention, false);
|
||||
var failureRetention = getRetention(dataStream, globalRetention, true);
|
||||
if (dataRetention == null && failureRetention == null) {
|
||||
return Set.of();
|
||||
}
|
||||
List<Index> backingIndicesOlderThanRetention = dataStream.getIndicesPastRetention(
|
||||
project::index,
|
||||
nowSupplier,
|
||||
globalRetention
|
||||
dataRetention,
|
||||
false
|
||||
);
|
||||
List<Index> failureIndicesOlderThanRetention = dataStream.getFailureIndicesPastRetention(
|
||||
List<Index> failureIndicesOlderThanRetention = dataStream.getIndicesPastRetention(
|
||||
project::index,
|
||||
nowSupplier,
|
||||
globalRetention
|
||||
failureRetention,
|
||||
true
|
||||
);
|
||||
if (backingIndicesOlderThanRetention.isEmpty() && failureIndicesOlderThanRetention.isEmpty()) {
|
||||
return Set.of();
|
||||
}
|
||||
Set<Index> indicesToBeRemoved = new HashSet<>();
|
||||
if (backingIndicesOlderThanRetention.isEmpty() == false) {
|
||||
assert dataStream.getDataLifecycle() != null : "data stream should have failure lifecycle if we have 'old' indices";
|
||||
TimeValue dataRetention = dataStream.getDataLifecycle().getEffectiveDataRetention(globalRetention, dataStream.isInternal());
|
||||
assert dataStream.getDataLifecycle() != null : "data stream should have data lifecycle if we have 'old' indices";
|
||||
for (Index index : backingIndicesOlderThanRetention) {
|
||||
if (indicesToExcludeForRemainingRun.contains(index) == false) {
|
||||
IndexMetadata backingIndex = project.index(index);
|
||||
|
@ -951,8 +960,7 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
|
|||
}
|
||||
}
|
||||
if (failureIndicesOlderThanRetention.isEmpty() == false) {
|
||||
assert dataStream.getFailuresLifecycle() != null : "data stream should have failure lifecycle if we have 'old' indices";
|
||||
var failureRetention = dataStream.getFailuresLifecycle().getEffectiveDataRetention(globalRetention, dataStream.isInternal());
|
||||
assert dataStream.getFailuresLifecycle() != null : "data stream should have failures lifecycle if we have 'old' indices";
|
||||
for (Index index : failureIndicesOlderThanRetention) {
|
||||
if (indicesToExcludeForRemainingRun.contains(index) == false) {
|
||||
IndexMetadata failureIndex = project.index(index);
|
||||
|
@ -1352,6 +1360,14 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
|
|||
return customMetadata != null && customMetadata.containsKey(FORCE_MERGE_COMPLETED_TIMESTAMP_METADATA_KEY);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
private static TimeValue getRetention(DataStream dataStream, DataStreamGlobalRetention globalRetention, boolean failureStore) {
|
||||
DataStreamLifecycle lifecycle = failureStore ? dataStream.getFailuresLifecycle() : dataStream.getDataLifecycle();
|
||||
return lifecycle == null || lifecycle.enabled() == false
|
||||
? null
|
||||
: lifecycle.getEffectiveDataRetention(globalRetention, dataStream.isInternal());
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the duration of the last run in millis or null if the service hasn't completed a run yet.
|
||||
*/
|
||||
|
@ -1474,10 +1490,10 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
|
|||
TimeValue dataRetention,
|
||||
boolean rolloverFailureStore
|
||||
) {
|
||||
RolloverRequest rolloverRequest = new RolloverRequest(dataStream, null).masterNodeTimeout(TimeValue.MAX_VALUE);
|
||||
if (rolloverFailureStore) {
|
||||
rolloverRequest.setRolloverTarget(IndexNameExpressionResolver.combineSelector(dataStream, IndexComponentSelector.FAILURES));
|
||||
}
|
||||
var rolloverTarget = rolloverFailureStore
|
||||
? IndexNameExpressionResolver.combineSelector(dataStream, IndexComponentSelector.FAILURES)
|
||||
: dataStream;
|
||||
RolloverRequest rolloverRequest = new RolloverRequest(rolloverTarget, null).masterNodeTimeout(TimeValue.MAX_VALUE);
|
||||
rolloverRequest.setConditions(rolloverConfiguration.resolveRolloverConditions(dataRetention));
|
||||
return rolloverRequest;
|
||||
}
|
||||
|
|
|
@ -10,6 +10,7 @@ package org.elasticsearch.datastreams.options.action;
|
|||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.datastreams.DataStreamsActionUtil;
|
||||
import org.elasticsearch.action.datastreams.PutDataStreamOptionsAction;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.master.AcknowledgedResponse;
|
||||
import org.elasticsearch.action.support.master.AcknowledgedTransportMasterNodeProjectAction;
|
||||
|
|
|
@ -12,6 +12,7 @@ import org.elasticsearch.action.support.IndicesOptions;
|
|||
import org.elasticsearch.client.internal.node.NodeClient;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.datastreams.options.action.GetDataStreamOptionsAction;
|
||||
import org.elasticsearch.datastreams.rest.RestGetDataStreamsAction;
|
||||
import org.elasticsearch.rest.BaseRestHandler;
|
||||
import org.elasticsearch.rest.RestRequest;
|
||||
import org.elasticsearch.rest.RestUtils;
|
||||
|
@ -21,12 +22,15 @@ import org.elasticsearch.rest.action.RestCancellableNodeClient;
|
|||
import org.elasticsearch.rest.action.RestRefCountedChunkedToXContentListener;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.elasticsearch.rest.RestRequest.Method.GET;
|
||||
|
||||
@ServerlessScope(Scope.PUBLIC)
|
||||
public class RestGetDataStreamOptionsAction extends BaseRestHandler {
|
||||
|
||||
private static final Set<String> CAPABILITIES = Set.of(RestGetDataStreamsAction.FAILURES_LIFECYCLE_API_CAPABILITY);
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return "get_data_stream_options_action";
|
||||
|
@ -56,4 +60,9 @@ public class RestGetDataStreamOptionsAction extends BaseRestHandler {
|
|||
public boolean allowSystemIndexAccessByDefault() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> supportedCapabilities() {
|
||||
return CAPABILITIES;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -8,10 +8,11 @@
|
|||
*/
|
||||
package org.elasticsearch.datastreams.options.rest;
|
||||
|
||||
import org.elasticsearch.action.datastreams.PutDataStreamOptionsAction;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.client.internal.node.NodeClient;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.datastreams.options.action.PutDataStreamOptionsAction;
|
||||
import org.elasticsearch.datastreams.rest.RestGetDataStreamsAction;
|
||||
import org.elasticsearch.rest.BaseRestHandler;
|
||||
import org.elasticsearch.rest.RestRequest;
|
||||
import org.elasticsearch.rest.Scope;
|
||||
|
@ -21,6 +22,7 @@ import org.elasticsearch.xcontent.XContentParser;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.elasticsearch.rest.RestRequest.Method.PUT;
|
||||
import static org.elasticsearch.rest.RestUtils.getAckTimeout;
|
||||
|
@ -29,6 +31,8 @@ import static org.elasticsearch.rest.RestUtils.getMasterNodeTimeout;
|
|||
@ServerlessScope(Scope.PUBLIC)
|
||||
public class RestPutDataStreamOptionsAction extends BaseRestHandler {
|
||||
|
||||
private static final Set<String> CAPABILITIES = Set.of(RestGetDataStreamsAction.FAILURES_LIFECYCLE_API_CAPABILITY);
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return "put_data_stream_options_action";
|
||||
|
@ -55,4 +59,9 @@ public class RestPutDataStreamOptionsAction extends BaseRestHandler {
|
|||
return channel -> client.execute(PutDataStreamOptionsAction.INSTANCE, putOptionsRequest, new RestToXContentListener<>(channel));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> supportedCapabilities() {
|
||||
return CAPABILITIES;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -15,7 +15,9 @@ import org.elasticsearch.action.admin.indices.template.put.TransportPutComposabl
|
|||
import org.elasticsearch.action.downsample.DownsampleConfig;
|
||||
import org.elasticsearch.cluster.metadata.ComposableIndexTemplate;
|
||||
import org.elasticsearch.cluster.metadata.DataStream;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamFailureStore;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamLifecycle;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamOptions;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetadata;
|
||||
import org.elasticsearch.cluster.metadata.ProjectMetadata;
|
||||
import org.elasticsearch.cluster.metadata.ResettableValue;
|
||||
|
@ -56,7 +58,7 @@ public class DataStreamLifecycleFixtures {
|
|||
@Nullable DataStreamLifecycle lifecycle,
|
||||
Long now
|
||||
) {
|
||||
return createDataStream(builder, dataStreamName, backingIndicesCount, 0, backingIndicesSettings, lifecycle, now);
|
||||
return createDataStream(builder, dataStreamName, backingIndicesCount, 0, backingIndicesSettings, lifecycle, null, now);
|
||||
}
|
||||
|
||||
public static DataStream createDataStream(
|
||||
|
@ -65,7 +67,8 @@ public class DataStreamLifecycleFixtures {
|
|||
int backingIndicesCount,
|
||||
int failureIndicesCount,
|
||||
Settings.Builder backingIndicesSettings,
|
||||
@Nullable DataStreamLifecycle lifecycle,
|
||||
@Nullable DataStreamLifecycle dataLifecycle,
|
||||
@Nullable DataStreamLifecycle failuresLifecycle,
|
||||
Long now
|
||||
) {
|
||||
final List<Index> backingIndices = new ArrayList<>();
|
||||
|
@ -100,7 +103,18 @@ public class DataStreamLifecycleFixtures {
|
|||
builder.put(indexMetadata, false);
|
||||
failureIndices.add(indexMetadata.getIndex());
|
||||
}
|
||||
return newInstance(dataStreamName, backingIndices, backingIndicesCount, null, false, lifecycle, failureIndices);
|
||||
return newInstance(
|
||||
dataStreamName,
|
||||
backingIndices,
|
||||
backingIndicesCount,
|
||||
null,
|
||||
false,
|
||||
dataLifecycle,
|
||||
failureIndices,
|
||||
new DataStreamOptions(
|
||||
DataStreamFailureStore.builder().enabled(failureIndices.isEmpty() == false).lifecycle(failuresLifecycle).build()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
static void putComposableIndexTemplate(
|
||||
|
|
|
@ -37,6 +37,7 @@ import org.elasticsearch.cluster.TestShardRoutingRoleStrategies;
|
|||
import org.elasticsearch.cluster.block.ClusterBlock;
|
||||
import org.elasticsearch.cluster.block.ClusterBlocks;
|
||||
import org.elasticsearch.cluster.metadata.DataStream;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamFailureStore;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionSettings;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamLifecycle;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamLifecycle.DownsamplingRound;
|
||||
|
@ -200,13 +201,18 @@ public class DataStreamLifecycleServiceTests extends ESTestCase {
|
|||
String dataStreamName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
|
||||
int numBackingIndices = 3;
|
||||
ProjectMetadata.Builder builder = ProjectMetadata.builder(randomProjectIdOrDefault());
|
||||
DataStreamLifecycle zeroRetentionDataLifecycle = DataStreamLifecycle.dataLifecycleBuilder().dataRetention(TimeValue.ZERO).build();
|
||||
DataStreamLifecycle zeroRetentionFailuresLifecycle = DataStreamLifecycle.failuresLifecycleBuilder()
|
||||
.dataRetention(TimeValue.ZERO)
|
||||
.build();
|
||||
DataStream dataStream = createDataStream(
|
||||
builder,
|
||||
dataStreamName,
|
||||
numBackingIndices,
|
||||
2,
|
||||
settings(IndexVersion.current()),
|
||||
DataStreamLifecycle.dataLifecycleBuilder().dataRetention(TimeValue.ZERO).build(),
|
||||
zeroRetentionDataLifecycle,
|
||||
zeroRetentionFailuresLifecycle,
|
||||
now
|
||||
);
|
||||
builder.put(dataStream);
|
||||
|
@ -270,6 +276,7 @@ public class DataStreamLifecycleServiceTests extends ESTestCase {
|
|||
numFailureIndices,
|
||||
settings(IndexVersion.current()),
|
||||
DataStreamLifecycle.dataLifecycleBuilder().dataRetention(TimeValue.timeValueDays(700)).build(),
|
||||
null,
|
||||
now
|
||||
);
|
||||
builder.put(dataStream);
|
||||
|
@ -1483,6 +1490,7 @@ public class DataStreamLifecycleServiceTests extends ESTestCase {
|
|||
numFailureIndices,
|
||||
settings(IndexVersion.current()),
|
||||
DataStreamLifecycle.DEFAULT_DATA_LIFECYCLE,
|
||||
null,
|
||||
now
|
||||
).copy().setDataStreamOptions(dataStreamOptions).build(); // failure store is managed even when disabled
|
||||
builder.put(dataStream);
|
||||
|
@ -1512,25 +1520,31 @@ public class DataStreamLifecycleServiceTests extends ESTestCase {
|
|||
numBackingIndices,
|
||||
2,
|
||||
settings(IndexVersion.current()),
|
||||
DataStreamLifecycle.dataLifecycleBuilder().dataRetention(TimeValue.ZERO).build(),
|
||||
null,
|
||||
null,
|
||||
now
|
||||
).copy().setDataStreamOptions(DataStreamOptions.FAILURE_STORE_DISABLED).build(); // failure store is managed even when disabled
|
||||
).copy()
|
||||
.setDataStreamOptions(
|
||||
new DataStreamOptions(
|
||||
DataStreamFailureStore.builder()
|
||||
.enabled(false)
|
||||
.lifecycle(DataStreamLifecycle.failuresLifecycleBuilder().dataRetention(TimeValue.ZERO).build())
|
||||
.build()
|
||||
)
|
||||
)
|
||||
.build(); // failure store is managed even when disabled
|
||||
builder.put(dataStream);
|
||||
|
||||
ClusterState state = ClusterState.builder(ClusterName.DEFAULT).putProjectMetadata(builder).build();
|
||||
|
||||
dataStreamLifecycleService.run(state);
|
||||
assertThat(clientSeenRequests.size(), is(3));
|
||||
assertThat(clientSeenRequests.get(0), instanceOf(RolloverRequest.class));
|
||||
RolloverRequest rolloverBackingIndexRequest = (RolloverRequest) clientSeenRequests.get(0);
|
||||
assertThat(rolloverBackingIndexRequest.getRolloverTarget(), is(dataStreamName));
|
||||
assertThat(clientSeenRequests.get(1), instanceOf(RolloverRequest.class));
|
||||
RolloverRequest rolloverFailureIndexRequest = (RolloverRequest) clientSeenRequests.get(1);
|
||||
assertThat(clientSeenRequests.size(), is(2));
|
||||
RolloverRequest rolloverFailureIndexRequest = (RolloverRequest) clientSeenRequests.get(0);
|
||||
assertThat(
|
||||
rolloverFailureIndexRequest.getRolloverTarget(),
|
||||
is(IndexNameExpressionResolver.combineSelector(dataStreamName, IndexComponentSelector.FAILURES))
|
||||
);
|
||||
assertThat(((DeleteIndexRequest) clientSeenRequests.get(2)).indices()[0], is(dataStream.getFailureIndices().get(0).getName()));
|
||||
assertThat(((DeleteIndexRequest) clientSeenRequests.get(1)).indices()[0], is(dataStream.getFailureIndices().get(0).getName()));
|
||||
}
|
||||
|
||||
public void testMaybeExecuteRetentionSuccessfulDownsampledIndex() {
|
||||
|
|
|
@ -45,6 +45,13 @@ teardown:
|
|||
|
||||
---
|
||||
"Edit data stream options":
|
||||
- requires:
|
||||
reason: "Data stream failures lifecycle was added in 8.19+"
|
||||
test_runner_features: [ capabilities, allowed_warnings ]
|
||||
capabilities:
|
||||
- method: PUT
|
||||
path: /_data_stream/{target}/_options
|
||||
capabilities: [ 'failure_store.lifecycle' ]
|
||||
- do:
|
||||
indices.get_data_stream_options:
|
||||
name: "failure-data-stream"
|
||||
|
@ -58,6 +65,8 @@ teardown:
|
|||
body:
|
||||
failure_store:
|
||||
enabled: false
|
||||
lifecycle:
|
||||
data_retention: 30d
|
||||
- is_true: acknowledged
|
||||
|
||||
- do:
|
||||
|
@ -66,6 +75,9 @@ teardown:
|
|||
- length: { data_streams: 1 }
|
||||
- match: { data_streams.0.name: failure-data-stream }
|
||||
- match: { data_streams.0.options.failure_store.enabled: false }
|
||||
- is_false: data_streams.0.options.failure_store.enabled
|
||||
- match: { data_streams.0.options.failure_store.lifecycle.enabled: true }
|
||||
- match: { data_streams.0.options.failure_store.lifecycle.data_retention: 30d }
|
||||
|
||||
- do:
|
||||
indices.delete_data_stream_options:
|
||||
|
@ -113,3 +125,53 @@ teardown:
|
|||
number_of_replicas: 0
|
||||
data_stream_options: null
|
||||
- is_true: acknowledged
|
||||
|
||||
---
|
||||
"Test partially resetting failure store options in template composition":
|
||||
- requires:
|
||||
reason: "Data stream failure stores config in templates was added in 8.16+"
|
||||
test_runner_features: [ capabilities, allowed_warnings ]
|
||||
capabilities:
|
||||
- method: POST
|
||||
path: /_component_template/{template}
|
||||
capabilities: [ 'data_stream_options.failure_store.lifecycle' ]
|
||||
- do:
|
||||
cluster.put_component_template:
|
||||
name: my-component-template
|
||||
body:
|
||||
template:
|
||||
data_stream_options:
|
||||
failure_store:
|
||||
enabled: true
|
||||
lifecycle:
|
||||
data_retention: 10d
|
||||
- is_true: acknowledged
|
||||
|
||||
- do:
|
||||
cluster.put_component_template:
|
||||
name: reset-lifecycle-template
|
||||
body:
|
||||
template:
|
||||
data_stream_options:
|
||||
failure_store:
|
||||
lifecycle: null
|
||||
- is_true: acknowledged
|
||||
|
||||
- do:
|
||||
cluster.put_component_template:
|
||||
name: reset-enabled-template
|
||||
body:
|
||||
template:
|
||||
data_stream_options:
|
||||
failure_store:
|
||||
enabled: null
|
||||
- is_true: acknowledged
|
||||
|
||||
- do:
|
||||
indices.simulate_index_template:
|
||||
name: my-simulated-template
|
||||
body:
|
||||
index_patterns: [ no-data-stream ]
|
||||
composed_of: [my-component-template, reset-lifecycle-template, reset-enabled-template]
|
||||
data_stream: {}
|
||||
- match: { template.data_stream_options.failure_store: null }
|
||||
|
|
|
@ -22,7 +22,6 @@ setup:
|
|||
type: date
|
||||
count:
|
||||
type: long
|
||||
lifecycle: {}
|
||||
data_stream_options:
|
||||
failure_store:
|
||||
enabled: true
|
||||
|
@ -44,7 +43,6 @@ setup:
|
|||
type: date
|
||||
count:
|
||||
type: long
|
||||
lifecycle: {}
|
||||
data_stream: { }
|
||||
|
||||
- do:
|
||||
|
@ -161,10 +159,13 @@ teardown:
|
|||
- is_true: acknowledged
|
||||
|
||||
- do:
|
||||
indices.put_data_lifecycle:
|
||||
indices.put_data_stream_options:
|
||||
name: "fs-data-stream"
|
||||
body:
|
||||
enabled: false
|
||||
failure_store:
|
||||
enabled: true
|
||||
lifecycle:
|
||||
enabled: false
|
||||
|
||||
- is_true: acknowledged
|
||||
|
||||
|
|
|
@ -228,6 +228,7 @@ public class TransportVersions {
|
|||
public static final TransportVersion DENSE_VECTOR_OFF_HEAP_STATS = def(9_062_00_0);
|
||||
public static final TransportVersion RANDOM_SAMPLER_QUERY_BUILDER = def(9_063_0_00);
|
||||
public static final TransportVersion SETTINGS_IN_DATA_STREAMS = def(9_064_0_00);
|
||||
public static final TransportVersion INTRODUCE_FAILURES_LIFECYCLE = def(9_065_0_00);
|
||||
|
||||
/*
|
||||
* STOP! READ THIS FIRST! No, really,
|
||||
|
|
|
@ -422,7 +422,7 @@ public class GetDataStreamAction extends ActionType<GetDataStreamAction.Response
|
|||
builder.field(DataStream.ROLLOVER_ON_WRITE_FIELD.getPreferredName(), dataStream.getFailureComponent().isRolloverOnWrite());
|
||||
indicesToXContent(builder, dataStream.getFailureIndices(), true);
|
||||
addAutoShardingEvent(builder, params, dataStream.getFailureComponent().getAutoShardingEvent());
|
||||
DataStreamLifecycle failuresLifecycle = dataStream.getFailuresLifecycle();
|
||||
DataStreamLifecycle failuresLifecycle = dataStream.getFailuresLifecycle(failureStoreEffectivelyEnabled);
|
||||
if (failuresLifecycle != null) {
|
||||
builder.field(LIFECYCLE_FIELD.getPreferredName());
|
||||
failuresLifecycle.toXContent(builder, params, rolloverConfiguration, globalRetention, dataStream.isInternal());
|
||||
|
|
|
@ -7,7 +7,7 @@
|
|||
* License v3.0 only", or the "Server Side Public License, v 1".
|
||||
*/
|
||||
|
||||
package org.elasticsearch.datastreams.options.action;
|
||||
package org.elasticsearch.action.datastreams;
|
||||
|
||||
import org.elasticsearch.action.ActionRequestValidationException;
|
||||
import org.elasticsearch.action.ActionType;
|
|
@ -229,6 +229,8 @@ public final class DataStream implements SimpleDiffable<DataStream>, ToXContentO
|
|||
this.system = system;
|
||||
this.allowCustomRouting = allowCustomRouting;
|
||||
this.indexMode = indexMode;
|
||||
assert lifecycle == null || lifecycle.targetsFailureStore() == false : "Invalid lifecycle type for data lifecycle";
|
||||
|
||||
this.lifecycle = lifecycle;
|
||||
this.dataStreamOptions = dataStreamOptions == null ? DataStreamOptions.EMPTY : dataStreamOptions;
|
||||
assert backingIndices.indices.isEmpty() == false;
|
||||
|
@ -594,12 +596,31 @@ public final class DataStream implements SimpleDiffable<DataStream>, ToXContentO
|
|||
}
|
||||
|
||||
/**
|
||||
* Retrieves the lifecycle configuration meant for the failure store. Currently, it's the same with {@link #getDataLifecycle()}
|
||||
* but it will change.
|
||||
* Retrieves the effective lifecycle configuration for the failure store. This can be either the configuration provided
|
||||
* by a user or the default lifecycle if there are failure indices. NOTE: this does not take into consideration if the
|
||||
* failure store is enabled by a cluster setting, please use {@link DataStream#getFailuresLifecycle(Boolean)}.
|
||||
*/
|
||||
@Nullable
|
||||
public DataStreamLifecycle getFailuresLifecycle() {
|
||||
return lifecycle;
|
||||
return getFailuresLifecycle(
|
||||
dataStreamOptions != null && dataStreamOptions.failureStore() != null ? dataStreamOptions.failureStore().enabled() : null
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves the effective lifecycle configuration for the failure store. This can be either the configuration provided
|
||||
* by a user or the default lifecycle if there are failure indices or if the failure store is enabled by a cluster setting.
|
||||
*/
|
||||
@Nullable
|
||||
public DataStreamLifecycle getFailuresLifecycle(Boolean effectivelyEnabledFailureStore) {
|
||||
// When there is a lifecycle configured by the user we return it.
|
||||
if (dataStreamOptions.failureStore() != null && dataStreamOptions.failureStore().lifecycle() != null) {
|
||||
return dataStreamOptions.failureStore().lifecycle();
|
||||
}
|
||||
// If there are failure indices we always provide the default lifecycle as a default
|
||||
return Boolean.TRUE.equals(effectivelyEnabledFailureStore) || getFailureIndices().isEmpty() == false
|
||||
? DataStreamLifecycle.DEFAULT_FAILURE_LIFECYCLE
|
||||
: null;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1034,52 +1055,24 @@ public final class DataStream implements SimpleDiffable<DataStream>, ToXContentO
|
|||
}
|
||||
|
||||
/**
|
||||
* Iterate over the backing indices and return the ones that are managed by the data stream lifecycle and past the
|
||||
* configured retention in their lifecycle.
|
||||
* Iterate over the backing or failure indices depending on <code>failureStore</code> and return the ones that are managed by the
|
||||
* data stream lifecycle and past the configured retention in their lifecycle.
|
||||
* NOTE that this specifically does not return the write index of the data stream as usually retention
|
||||
* is treated differently for the write index (i.e. they first need to be rolled over)
|
||||
*/
|
||||
public List<Index> getBackingIndicesPastRetention(
|
||||
public List<Index> getIndicesPastRetention(
|
||||
Function<String, IndexMetadata> indexMetadataSupplier,
|
||||
LongSupplier nowSupplier,
|
||||
DataStreamGlobalRetention globalRetention
|
||||
TimeValue effectiveRetention,
|
||||
boolean failureStore
|
||||
) {
|
||||
if (getDataLifecycle() == null
|
||||
|| getDataLifecycle().enabled() == false
|
||||
|| getDataLifecycle().getEffectiveDataRetention(globalRetention, isInternal()) == null) {
|
||||
if (effectiveRetention == null) {
|
||||
return List.of();
|
||||
}
|
||||
|
||||
List<Index> indicesPastRetention = getNonWriteIndicesOlderThan(
|
||||
getIndices(),
|
||||
getDataLifecycle().getEffectiveDataRetention(globalRetention, isInternal()),
|
||||
indexMetadataSupplier,
|
||||
this::isIndexManagedByDataStreamLifecycle,
|
||||
nowSupplier
|
||||
);
|
||||
return indicesPastRetention;
|
||||
}
|
||||
|
||||
/**
|
||||
* Iterate over the failure indices and return the ones that are managed by the data stream lifecycle and past the
|
||||
* configured retention in their lifecycle.
|
||||
* NOTE that this specifically does not return the write index of the data stream as usually retention
|
||||
* is treated differently for the write index (i.e. they first need to be rolled over)
|
||||
*/
|
||||
public List<Index> getFailureIndicesPastRetention(
|
||||
Function<String, IndexMetadata> indexMetadataSupplier,
|
||||
LongSupplier nowSupplier,
|
||||
DataStreamGlobalRetention globalRetention
|
||||
) {
|
||||
if (getFailuresLifecycle() == null
|
||||
|| getFailuresLifecycle().enabled() == false
|
||||
|| getFailuresLifecycle().getEffectiveDataRetention(globalRetention, isInternal()) == null) {
|
||||
return List.of();
|
||||
}
|
||||
|
||||
List<Index> indicesPastRetention = getNonWriteIndicesOlderThan(
|
||||
getFailureIndices(),
|
||||
getFailuresLifecycle().getEffectiveDataRetention(globalRetention, isInternal()),
|
||||
getDataStreamIndices(failureStore).getIndices(),
|
||||
effectiveRetention,
|
||||
indexMetadataSupplier,
|
||||
this::isIndexManagedByDataStreamLifecycle,
|
||||
nowSupplier
|
||||
|
@ -1194,6 +1187,7 @@ public final class DataStream implements SimpleDiffable<DataStream>, ToXContentO
|
|||
* access method.
|
||||
*/
|
||||
private boolean isIndexManagedByDataStreamLifecycle(IndexMetadata indexMetadata) {
|
||||
var lifecycle = getDataLifecycleForIndex(indexMetadata.getIndex());
|
||||
if (indexMetadata.getLifecyclePolicyName() != null && lifecycle != null && lifecycle.enabled()) {
|
||||
// when both ILM and data stream lifecycle are configured, choose depending on the configured preference for this backing index
|
||||
return PREFER_ILM_SETTING.get(indexMetadata.getSettings()) == false;
|
||||
|
@ -1406,7 +1400,11 @@ public final class DataStream implements SimpleDiffable<DataStream>, ToXContentO
|
|||
PARSER.declareBoolean(ConstructingObjectParser.optionalConstructorArg(), SYSTEM_FIELD);
|
||||
PARSER.declareBoolean(ConstructingObjectParser.optionalConstructorArg(), ALLOW_CUSTOM_ROUTING);
|
||||
PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), INDEX_MODE);
|
||||
PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(), (p, c) -> DataStreamLifecycle.fromXContent(p), LIFECYCLE);
|
||||
PARSER.declareObject(
|
||||
ConstructingObjectParser.optionalConstructorArg(),
|
||||
(p, c) -> DataStreamLifecycle.dataLifecycleFromXContent(p),
|
||||
LIFECYCLE
|
||||
);
|
||||
PARSER.declareBoolean(ConstructingObjectParser.optionalConstructorArg(), ROLLOVER_ON_WRITE_FIELD);
|
||||
PARSER.declareObject(
|
||||
ConstructingObjectParser.optionalConstructorArg(),
|
||||
|
|
|
@ -9,12 +9,14 @@
|
|||
|
||||
package org.elasticsearch.cluster.metadata;
|
||||
|
||||
import org.elasticsearch.TransportVersions;
|
||||
import org.elasticsearch.cluster.Diff;
|
||||
import org.elasticsearch.cluster.SimpleDiffable;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.core.Nullable;
|
||||
import org.elasticsearch.xcontent.ConstructingObjectParser;
|
||||
import org.elasticsearch.xcontent.ObjectParser;
|
||||
import org.elasticsearch.xcontent.ParseField;
|
||||
|
@ -28,20 +30,33 @@ import java.io.IOException;
|
|||
* Holds the data stream failure store metadata that enable or disable the failure store of a data stream. Currently, it
|
||||
* supports the following configurations only explicitly enabling or disabling the failure store
|
||||
*/
|
||||
public record DataStreamFailureStore(Boolean enabled) implements SimpleDiffable<DataStreamFailureStore>, ToXContentObject {
|
||||
public record DataStreamFailureStore(@Nullable Boolean enabled, @Nullable DataStreamLifecycle lifecycle)
|
||||
implements
|
||||
SimpleDiffable<DataStreamFailureStore>,
|
||||
ToXContentObject {
|
||||
|
||||
public static final String FAILURE_STORE = "failure_store";
|
||||
public static final String ENABLED = "enabled";
|
||||
public static final String LIFECYCLE = "lifecycle";
|
||||
private static final String EMPTY_FAILURE_STORE_ERROR_MESSAGE =
|
||||
"Failure store configuration should have at least one non-null configuration value.";
|
||||
|
||||
public static final ParseField ENABLED_FIELD = new ParseField(ENABLED);
|
||||
public static final ParseField LIFECYCLE_FIELD = new ParseField(LIFECYCLE);
|
||||
|
||||
public static final ConstructingObjectParser<DataStreamFailureStore, Void> PARSER = new ConstructingObjectParser<>(
|
||||
FAILURE_STORE,
|
||||
false,
|
||||
(args, unused) -> new DataStreamFailureStore((Boolean) args[0])
|
||||
(args, unused) -> new DataStreamFailureStore((Boolean) args[0], (DataStreamLifecycle) args[1])
|
||||
);
|
||||
|
||||
static {
|
||||
PARSER.declareBoolean(ConstructingObjectParser.optionalConstructorArg(), ENABLED_FIELD);
|
||||
PARSER.declareObject(
|
||||
ConstructingObjectParser.optionalConstructorArg(),
|
||||
(p, unused) -> DataStreamLifecycle.failureLifecycleFromXContent(p),
|
||||
LIFECYCLE_FIELD
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -50,13 +65,19 @@ public record DataStreamFailureStore(Boolean enabled) implements SimpleDiffable<
|
|||
* @throws IllegalArgumentException when all the constructor arguments are null
|
||||
*/
|
||||
public DataStreamFailureStore {
|
||||
if (enabled == null) {
|
||||
throw new IllegalArgumentException("Failure store configuration should have at least one non-null configuration value.");
|
||||
if (enabled == null && lifecycle == null) {
|
||||
throw new IllegalArgumentException(EMPTY_FAILURE_STORE_ERROR_MESSAGE);
|
||||
}
|
||||
assert lifecycle == null || lifecycle.targetsFailureStore() : "Invalid type for failures lifecycle";
|
||||
}
|
||||
|
||||
public DataStreamFailureStore(StreamInput in) throws IOException {
|
||||
this(in.readOptionalBoolean());
|
||||
this(
|
||||
in.readOptionalBoolean(),
|
||||
in.getTransportVersion().onOrAfter(TransportVersions.INTRODUCE_FAILURES_LIFECYCLE)
|
||||
? in.readOptionalWriteable(DataStreamLifecycle::new)
|
||||
: null
|
||||
);
|
||||
}
|
||||
|
||||
public static Diff<DataStreamFailureStore> readDiffFrom(StreamInput in) throws IOException {
|
||||
|
@ -66,6 +87,9 @@ public record DataStreamFailureStore(Boolean enabled) implements SimpleDiffable<
|
|||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeOptionalBoolean(enabled);
|
||||
if (out.getTransportVersion().onOrAfter(TransportVersions.INTRODUCE_FAILURES_LIFECYCLE)) {
|
||||
out.writeOptionalWriteable(lifecycle);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -79,6 +103,9 @@ public record DataStreamFailureStore(Boolean enabled) implements SimpleDiffable<
|
|||
if (enabled != null) {
|
||||
builder.field(ENABLED_FIELD.getPreferredName(), enabled);
|
||||
}
|
||||
if (lifecycle != null) {
|
||||
builder.field(LIFECYCLE_FIELD.getPreferredName(), lifecycle);
|
||||
}
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
@ -91,13 +118,19 @@ public record DataStreamFailureStore(Boolean enabled) implements SimpleDiffable<
|
|||
* This class is only used in template configuration. It wraps the fields of {@link DataStreamFailureStore} with {@link ResettableValue}
|
||||
* to allow a user to signal when they want to reset any previously encountered values during template composition.
|
||||
*/
|
||||
public record Template(ResettableValue<Boolean> enabled) implements Writeable, ToXContentObject {
|
||||
public record Template(ResettableValue<Boolean> enabled, ResettableValue<DataStreamLifecycle.Template> lifecycle)
|
||||
implements
|
||||
Writeable,
|
||||
ToXContentObject {
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public static final ConstructingObjectParser<Template, Void> PARSER = new ConstructingObjectParser<>(
|
||||
"failure_store_template",
|
||||
false,
|
||||
(args, unused) -> new Template(args[0] == null ? ResettableValue.undefined() : (ResettableValue<Boolean>) args[0])
|
||||
(args, unused) -> new Template(
|
||||
args[0] == null ? ResettableValue.undefined() : (ResettableValue<Boolean>) args[0],
|
||||
args[1] == null ? ResettableValue.undefined() : (ResettableValue<DataStreamLifecycle.Template>) args[1]
|
||||
)
|
||||
);
|
||||
|
||||
static {
|
||||
|
@ -109,26 +142,43 @@ public record DataStreamFailureStore(Boolean enabled) implements SimpleDiffable<
|
|||
ENABLED_FIELD,
|
||||
ObjectParser.ValueType.BOOLEAN_OR_NULL
|
||||
);
|
||||
PARSER.declareField(
|
||||
ConstructingObjectParser.optionalConstructorArg(),
|
||||
(p, c) -> p.currentToken() == XContentParser.Token.VALUE_NULL
|
||||
? ResettableValue.reset()
|
||||
: ResettableValue.create(DataStreamLifecycle.Template.failuresLifecycleTemplatefromXContent(p)),
|
||||
LIFECYCLE_FIELD,
|
||||
ObjectParser.ValueType.OBJECT_OR_NULL
|
||||
);
|
||||
}
|
||||
|
||||
public Template(Boolean enabled) {
|
||||
this(ResettableValue.create(enabled));
|
||||
public Template(@Nullable Boolean enabled, @Nullable DataStreamLifecycle.Template lifecycle) {
|
||||
this(ResettableValue.create(enabled), ResettableValue.create(lifecycle));
|
||||
}
|
||||
|
||||
public Template {
|
||||
if (enabled.get() == null) {
|
||||
throw new IllegalArgumentException("Failure store configuration should have at least one non-null configuration value.");
|
||||
if (enabled.isDefined() == false && lifecycle.isDefined() == false) {
|
||||
throw new IllegalArgumentException(EMPTY_FAILURE_STORE_ERROR_MESSAGE);
|
||||
}
|
||||
assert lifecycle.get() == null || lifecycle.mapAndGet(l -> l.toDataStreamLifecycle().targetsFailureStore())
|
||||
: "Invalid lifecycle type in failure store template";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
ResettableValue.write(out, enabled, StreamOutput::writeBoolean);
|
||||
if (out.getTransportVersion().onOrAfter(TransportVersions.INTRODUCE_FAILURES_LIFECYCLE)) {
|
||||
ResettableValue.write(out, lifecycle, (o, v) -> v.writeTo(o));
|
||||
}
|
||||
}
|
||||
|
||||
public static Template read(StreamInput in) throws IOException {
|
||||
ResettableValue<Boolean> enabled = ResettableValue.read(in, StreamInput::readBoolean);
|
||||
return new Template(enabled);
|
||||
ResettableValue<DataStreamLifecycle.Template> lifecycle = ResettableValue.undefined();
|
||||
if (in.getTransportVersion().onOrAfter(TransportVersions.INTRODUCE_FAILURES_LIFECYCLE)) {
|
||||
lifecycle = ResettableValue.read(in, DataStreamLifecycle.Template::read);
|
||||
}
|
||||
return new Template(enabled, lifecycle);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -139,6 +189,7 @@ public record DataStreamFailureStore(Boolean enabled) implements SimpleDiffable<
|
|||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
enabled.toXContent(builder, params, ENABLED_FIELD.getPreferredName());
|
||||
lifecycle.toXContent(builder, params, LIFECYCLE_FIELD.getPreferredName());
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
@ -147,10 +198,6 @@ public record DataStreamFailureStore(Boolean enabled) implements SimpleDiffable<
|
|||
return PARSER.parse(parser, null);
|
||||
}
|
||||
|
||||
public DataStreamFailureStore toFailureStore() {
|
||||
return new DataStreamFailureStore(enabled.get());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return Strings.toString(this, true, true);
|
||||
|
@ -171,22 +218,25 @@ public record DataStreamFailureStore(Boolean enabled) implements SimpleDiffable<
|
|||
|
||||
/**
|
||||
* Builder that is able to create either a DataStreamFailureStore or its respective Template.
|
||||
* Furthermore, its update methods can be used to compose templates.
|
||||
* Furthermore, its composeTemplate method during template composition.
|
||||
*/
|
||||
public static class Builder {
|
||||
private Boolean enabled = null;
|
||||
private DataStreamLifecycle.Builder lifecycleBuilder = null;
|
||||
|
||||
private Builder() {}
|
||||
|
||||
private Builder(Template template) {
|
||||
if (template != null) {
|
||||
enabled = template.enabled.get();
|
||||
lifecycleBuilder = template.lifecycle.mapAndGet(l -> DataStreamLifecycle.builder(l));
|
||||
}
|
||||
}
|
||||
|
||||
private Builder(DataStreamFailureStore failureStore) {
|
||||
if (failureStore != null) {
|
||||
enabled = failureStore.enabled;
|
||||
lifecycleBuilder = failureStore.lifecycle == null ? null : DataStreamLifecycle.builder(failureStore.lifecycle);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -195,26 +245,72 @@ public record DataStreamFailureStore(Boolean enabled) implements SimpleDiffable<
|
|||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Composes the provided enabled value with the current one. Because enabled is a resettable boolean, if it is defined
|
||||
* it will overwrite the current value.
|
||||
*/
|
||||
public Builder enabled(ResettableValue<Boolean> enabled) {
|
||||
if (enabled.shouldReset()) {
|
||||
this.enabled = null;
|
||||
} else if (enabled.isDefined()) {
|
||||
if (enabled.isDefined()) {
|
||||
this.enabled = enabled.get();
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder composeTemplate(DataStreamFailureStore.Template failureStore) {
|
||||
this.enabled(failureStore.enabled());
|
||||
public Builder lifecycle(DataStreamLifecycle lifecycle) {
|
||||
this.lifecycleBuilder = lifecycle == null ? null : DataStreamLifecycle.builder(lifecycle);
|
||||
return this;
|
||||
}
|
||||
|
||||
public DataStreamFailureStore build() {
|
||||
return new DataStreamFailureStore(enabled);
|
||||
/**
|
||||
* Composes the provided lifecycle value with the current one. Because lifecycle is a resettable template that can be merged,
|
||||
* if it is defined it will delegate to {@link DataStreamLifecycle.Builder#composeTemplate(DataStreamLifecycle.Template)} to
|
||||
* correctly compose the contents.
|
||||
*/
|
||||
public Builder lifecycle(ResettableValue<DataStreamLifecycle.Template> lifecycle) {
|
||||
if (lifecycle.shouldReset()) {
|
||||
this.lifecycleBuilder = null;
|
||||
} else if (lifecycle.isDefined()) {
|
||||
if (this.lifecycleBuilder == null) {
|
||||
this.lifecycleBuilder = DataStreamLifecycle.builder(lifecycle.get());
|
||||
} else {
|
||||
this.lifecycleBuilder.composeTemplate(lifecycle.get());
|
||||
}
|
||||
}
|
||||
return this;
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Composes the provided failure store template with this builder.
|
||||
*/
|
||||
public Builder composeTemplate(DataStreamFailureStore.Template failureStore) {
|
||||
this.enabled(failureStore.enabled());
|
||||
this.lifecycle(failureStore.lifecycle());
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds a valid DataStreamFailureStore configuration.
|
||||
* @return the object or null if all the values were null.
|
||||
*/
|
||||
@Nullable
|
||||
public DataStreamFailureStore build() {
|
||||
if (enabled == null && lifecycleBuilder == null) {
|
||||
return null;
|
||||
}
|
||||
return new DataStreamFailureStore(enabled, lifecycleBuilder == null ? null : lifecycleBuilder.build());
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds a valid template for the DataStreamFailureStore configuration.
|
||||
* @return the template or null if all the values were null.
|
||||
*/
|
||||
@Nullable
|
||||
public DataStreamFailureStore.Template buildTemplate() {
|
||||
return new Template(enabled);
|
||||
if (enabled == null && lifecycleBuilder == null) {
|
||||
return null;
|
||||
}
|
||||
return new Template(enabled, lifecycleBuilder == null ? null : lifecycleBuilder.buildTemplate());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -40,8 +40,7 @@ public class DataStreamFailureStoreDefinition {
|
|||
IndexMetadata.SETTING_NUMBER_OF_SHARDS,
|
||||
IndexMetadata.SETTING_NUMBER_OF_REPLICAS,
|
||||
IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS,
|
||||
IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(),
|
||||
IndexMetadata.LIFECYCLE_NAME
|
||||
IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey()
|
||||
);
|
||||
public static final Set<String> SUPPORTED_USER_SETTINGS_PREFIXES = Set.of(
|
||||
IndexMetadata.INDEX_ROUTING_REQUIRE_GROUP_PREFIX + ".",
|
||||
|
|
|
@ -36,19 +36,23 @@ import org.elasticsearch.xcontent.XContentBuilder;
|
|||
import org.elasticsearch.xcontent.XContentParser;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.elasticsearch.xcontent.ConstructingObjectParser.constructorArg;
|
||||
|
||||
/**
|
||||
* Holds the data stream lifecycle metadata that are configuring how a data stream is managed. Currently, it supports the following
|
||||
* configurations:
|
||||
* - enabled
|
||||
* - data retention
|
||||
* - downsampling
|
||||
* Holds the data stream lifecycle configuration that defines how the data stream indices of a data stream are managed. The lifecycle also
|
||||
* has a type that determines the type of index component it can manage. Currently, we support data and failures.
|
||||
* Lifecycle supports the following configurations:
|
||||
* - enabled, applicable to data and failures
|
||||
* - data retention, applicable to data and failures
|
||||
* - downsampling, applicable only to data
|
||||
*/
|
||||
public class DataStreamLifecycle implements SimpleDiffable<DataStreamLifecycle>, ToXContentObject {
|
||||
|
||||
|
@ -66,6 +70,8 @@ public class DataStreamLifecycle implements SimpleDiffable<DataStreamLifecycle>,
|
|||
"true"
|
||||
);
|
||||
public static final Tuple<TimeValue, RetentionSource> INFINITE_RETENTION = Tuple.tuple(null, RetentionSource.DATA_STREAM_CONFIGURATION);
|
||||
private static final String DOWNSAMPLING_NOT_SUPPORTED_ERROR_MESSAGE =
|
||||
"Failure store lifecycle does not support downsampling, please remove the downsampling configuration.";
|
||||
|
||||
/**
|
||||
* Check if {@link #DATA_STREAMS_LIFECYCLE_ONLY_SETTING_NAME} is present and set to {@code true}, indicating that
|
||||
|
@ -87,6 +93,7 @@ public class DataStreamLifecycle implements SimpleDiffable<DataStreamLifecycle>,
|
|||
);
|
||||
|
||||
public static final DataStreamLifecycle DEFAULT_DATA_LIFECYCLE = DataStreamLifecycle.createDataLifecycle(null, null, null);
|
||||
public static final DataStreamLifecycle DEFAULT_FAILURE_LIFECYCLE = DataStreamLifecycle.createFailuresLifecycle(null, null);
|
||||
|
||||
public static final String DATA_STREAM_LIFECYCLE_ORIGIN = "data_stream_lifecycle";
|
||||
|
||||
|
@ -98,10 +105,10 @@ public class DataStreamLifecycle implements SimpleDiffable<DataStreamLifecycle>,
|
|||
private static final ParseField ROLLOVER_FIELD = new ParseField("rollover");
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public static final ConstructingObjectParser<DataStreamLifecycle, Void> PARSER = new ConstructingObjectParser<>(
|
||||
public static final ConstructingObjectParser<DataStreamLifecycle, LifecycleType> PARSER = new ConstructingObjectParser<>(
|
||||
"lifecycle",
|
||||
false,
|
||||
(args, unused) -> new DataStreamLifecycle((Boolean) args[0], (TimeValue) args[1], (List<DownsamplingRound>) args[2])
|
||||
(args, lt) -> new DataStreamLifecycle(lt, (Boolean) args[0], (TimeValue) args[1], (List<DownsamplingRound>) args[2])
|
||||
);
|
||||
|
||||
static {
|
||||
|
@ -121,22 +128,34 @@ public class DataStreamLifecycle implements SimpleDiffable<DataStreamLifecycle>,
|
|||
if (p.currentToken() == XContentParser.Token.VALUE_NULL) {
|
||||
return null;
|
||||
} else {
|
||||
return AbstractObjectParser.parseArray(p, c, DownsamplingRound::fromXContent);
|
||||
return AbstractObjectParser.parseArray(p, null, DownsamplingRound::fromXContent);
|
||||
}
|
||||
}, DOWNSAMPLING_FIELD, ObjectParser.ValueType.OBJECT_ARRAY_OR_NULL);
|
||||
}
|
||||
|
||||
private final LifecycleType lifecycleType;
|
||||
private final boolean enabled;
|
||||
@Nullable
|
||||
private final TimeValue dataRetention;
|
||||
@Nullable
|
||||
private final List<DownsamplingRound> downsampling;
|
||||
|
||||
// Visible for testing, preferably use the factory methods that are specialised by lifecycle type,
|
||||
// for example for the data component.
|
||||
DataStreamLifecycle(@Nullable Boolean enabled, @Nullable TimeValue dataRetention, @Nullable List<DownsamplingRound> downsampling) {
|
||||
/**
|
||||
* This constructor is visible for testing, please use {@link DataStreamLifecycle#createDataLifecycle(Boolean, TimeValue, List)} or
|
||||
* {@link DataStreamLifecycle#createFailuresLifecycle(Boolean, TimeValue)}.
|
||||
*/
|
||||
DataStreamLifecycle(
|
||||
LifecycleType lifecycleType,
|
||||
@Nullable Boolean enabled,
|
||||
@Nullable TimeValue dataRetention,
|
||||
@Nullable List<DownsamplingRound> downsampling
|
||||
) {
|
||||
this.lifecycleType = lifecycleType;
|
||||
this.enabled = enabled == null || enabled;
|
||||
this.dataRetention = dataRetention;
|
||||
if (lifecycleType == LifecycleType.FAILURES && downsampling != null) {
|
||||
throw new IllegalArgumentException(DOWNSAMPLING_NOT_SUPPORTED_ERROR_MESSAGE);
|
||||
}
|
||||
DownsamplingRound.validateRounds(downsampling);
|
||||
this.downsampling = downsampling;
|
||||
}
|
||||
|
@ -150,7 +169,15 @@ public class DataStreamLifecycle implements SimpleDiffable<DataStreamLifecycle>,
|
|||
@Nullable TimeValue dataRetention,
|
||||
@Nullable List<DownsamplingRound> downsampling
|
||||
) {
|
||||
return new DataStreamLifecycle(enabled, dataRetention, downsampling);
|
||||
return new DataStreamLifecycle(LifecycleType.DATA, enabled, dataRetention, downsampling);
|
||||
}
|
||||
|
||||
/**
|
||||
* This factory method creates a lifecycle applicable for the failures index component of a data stream. This
|
||||
* means it supports only enabling and retention.
|
||||
*/
|
||||
public static DataStreamLifecycle createFailuresLifecycle(@Nullable Boolean enabled, @Nullable TimeValue dataRetention) {
|
||||
return new DataStreamLifecycle(LifecycleType.FAILURES, enabled, dataRetention, null);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -160,6 +187,17 @@ public class DataStreamLifecycle implements SimpleDiffable<DataStreamLifecycle>,
|
|||
return enabled;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true if the lifecycle manages the failure store, false otherwise
|
||||
*/
|
||||
public boolean targetsFailureStore() {
|
||||
return lifecycleType == LifecycleType.FAILURES;
|
||||
}
|
||||
|
||||
public String getLifecycleType() {
|
||||
return lifecycleType.label;
|
||||
}
|
||||
|
||||
/**
|
||||
* The least amount of time data should be kept by elasticsearch. The effective retention is a function with three parameters,
|
||||
* the {@link DataStreamLifecycle#dataRetention}, the global retention and whether this lifecycle is associated with an internal
|
||||
|
@ -272,14 +310,15 @@ public class DataStreamLifecycle implements SimpleDiffable<DataStreamLifecycle>,
|
|||
if (o == null || getClass() != o.getClass()) return false;
|
||||
|
||||
final DataStreamLifecycle that = (DataStreamLifecycle) o;
|
||||
return Objects.equals(dataRetention, that.dataRetention)
|
||||
return lifecycleType == that.lifecycleType
|
||||
&& Objects.equals(dataRetention, that.dataRetention)
|
||||
&& Objects.equals(downsampling, that.downsampling)
|
||||
&& enabled == that.enabled;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(enabled, dataRetention, downsampling);
|
||||
return Objects.hash(lifecycleType, enabled, dataRetention, downsampling);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -302,6 +341,9 @@ public class DataStreamLifecycle implements SimpleDiffable<DataStreamLifecycle>,
|
|||
}
|
||||
out.writeBoolean(enabled());
|
||||
}
|
||||
if (out.getTransportVersion().onOrAfter(TransportVersions.INTRODUCE_FAILURES_LIFECYCLE)) {
|
||||
lifecycleType.writeTo(out);
|
||||
}
|
||||
}
|
||||
|
||||
public DataStreamLifecycle(StreamInput in) throws IOException {
|
||||
|
@ -327,6 +369,9 @@ public class DataStreamLifecycle implements SimpleDiffable<DataStreamLifecycle>,
|
|||
downsampling = null;
|
||||
enabled = true;
|
||||
}
|
||||
lifecycleType = in.getTransportVersion().onOrAfter(TransportVersions.INTRODUCE_FAILURES_LIFECYCLE)
|
||||
? LifecycleType.read(in)
|
||||
: LifecycleType.DATA;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -372,7 +417,16 @@ public class DataStreamLifecycle implements SimpleDiffable<DataStreamLifecycle>,
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return Strings.toString(this, true, true);
|
||||
return "DataStreamLifecycle{"
|
||||
+ "lifecycleTarget="
|
||||
+ lifecycleType
|
||||
+ ", enabled="
|
||||
+ enabled
|
||||
+ ", dataRetention="
|
||||
+ dataRetention
|
||||
+ ", downsampling="
|
||||
+ downsampling
|
||||
+ '}';
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -422,13 +476,23 @@ public class DataStreamLifecycle implements SimpleDiffable<DataStreamLifecycle>,
|
|||
}
|
||||
|
||||
/**
|
||||
* This method deserialises XContent format as it was generated ONLY by {@link DataStreamLifecycle#toXContent(XContentBuilder, Params)}.
|
||||
* It does not support the output of
|
||||
* This method de-serialises a data lifecycle as it was generated ONLY by
|
||||
* {@link DataStreamLifecycle#toXContent(XContentBuilder, Params)}. It does not support the output of
|
||||
* {@link DataStreamLifecycle#toXContent(XContentBuilder, Params, RolloverConfiguration, DataStreamGlobalRetention, boolean)} because
|
||||
* this output is enriched with derived fields we do not handle in this deserialisation.
|
||||
* this output is enriched with derived fields we do not handle in this de-serialisation.
|
||||
*/
|
||||
public static DataStreamLifecycle fromXContent(XContentParser parser) throws IOException {
|
||||
return PARSER.parse(parser, null);
|
||||
public static DataStreamLifecycle dataLifecycleFromXContent(XContentParser parser) throws IOException {
|
||||
return PARSER.parse(parser, LifecycleType.DATA);
|
||||
}
|
||||
|
||||
/**
|
||||
* This method de-serialises a failures lifecycle as it was generated ONLY by
|
||||
* {@link DataStreamLifecycle#toXContent(XContentBuilder, Params)}. It does not support the output of
|
||||
* {@link DataStreamLifecycle#toXContent(XContentBuilder, Params, RolloverConfiguration, DataStreamGlobalRetention, boolean)} because
|
||||
* this output is enriched with derived fields we do not handle in this de-serialisation.
|
||||
*/
|
||||
public static DataStreamLifecycle failureLifecycleFromXContent(XContentParser parser) throws IOException {
|
||||
return PARSER.parse(parser, LifecycleType.FAILURES);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -552,20 +616,36 @@ public class DataStreamLifecycle implements SimpleDiffable<DataStreamLifecycle>,
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This factory method creates a lifecycle template applicable for the data index component of a data stream. This
|
||||
* means it supports all configuration applicable for backing indices.
|
||||
*/
|
||||
public static Template createDataLifecycleTemplate(
|
||||
boolean enabled,
|
||||
TimeValue dataRetention,
|
||||
List<DataStreamLifecycle.DownsamplingRound> downsampling
|
||||
) {
|
||||
return new Template(enabled, ResettableValue.create(dataRetention), ResettableValue.create(downsampling));
|
||||
return new Template(LifecycleType.DATA, enabled, ResettableValue.create(dataRetention), ResettableValue.create(downsampling));
|
||||
}
|
||||
|
||||
/**
|
||||
* This factory method creates a lifecycle template applicable for the data index component of a data stream. This
|
||||
* means it supports all configuration applicable for backing indices.
|
||||
*/
|
||||
public static Template createDataLifecycleTemplate(
|
||||
boolean enabled,
|
||||
ResettableValue<TimeValue> dataRetention,
|
||||
ResettableValue<List<DataStreamLifecycle.DownsamplingRound>> downsampling
|
||||
) {
|
||||
return new Template(enabled, dataRetention, downsampling);
|
||||
return new Template(LifecycleType.DATA, enabled, dataRetention, downsampling);
|
||||
}
|
||||
|
||||
/**
|
||||
* This factory method creates a lifecycle template applicable for the failures index component of a data stream. This
|
||||
* means it supports only setting the enabled and the retention.
|
||||
*/
|
||||
public static Template createFailuresLifecycleTemplate(boolean enabled, TimeValue dataRetention) {
|
||||
return new Template(LifecycleType.FAILURES, enabled, ResettableValue.create(dataRetention), ResettableValue.undefined());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -573,32 +653,43 @@ public class DataStreamLifecycle implements SimpleDiffable<DataStreamLifecycle>,
|
|||
* to allow value reset during template composition.
|
||||
*/
|
||||
public record Template(
|
||||
LifecycleType lifecycleType,
|
||||
boolean enabled,
|
||||
ResettableValue<TimeValue> dataRetention,
|
||||
ResettableValue<List<DataStreamLifecycle.DownsamplingRound>> downsampling
|
||||
) implements ToXContentObject, Writeable {
|
||||
|
||||
Template(boolean enabled, TimeValue dataRetention, List<DataStreamLifecycle.DownsamplingRound> downsampling) {
|
||||
this(enabled, ResettableValue.create(dataRetention), ResettableValue.create(downsampling));
|
||||
Template(
|
||||
LifecycleType lifecycleType,
|
||||
boolean enabled,
|
||||
TimeValue dataRetention,
|
||||
List<DataStreamLifecycle.DownsamplingRound> downsampling
|
||||
) {
|
||||
this(lifecycleType, enabled, ResettableValue.create(dataRetention), ResettableValue.create(downsampling));
|
||||
}
|
||||
|
||||
public Template {
|
||||
if (lifecycleType == LifecycleType.FAILURES && downsampling.get() != null) {
|
||||
throw new IllegalArgumentException(DOWNSAMPLING_NOT_SUPPORTED_ERROR_MESSAGE);
|
||||
}
|
||||
if (downsampling.isDefined() && downsampling.get() != null) {
|
||||
DownsamplingRound.validateRounds(downsampling.get());
|
||||
}
|
||||
}
|
||||
|
||||
public static final DataStreamLifecycle.Template DATA_DEFAULT = new DataStreamLifecycle.Template(
|
||||
LifecycleType.DATA,
|
||||
true,
|
||||
ResettableValue.undefined(),
|
||||
ResettableValue.undefined()
|
||||
);
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public static final ConstructingObjectParser<DataStreamLifecycle.Template, Void> PARSER = new ConstructingObjectParser<>(
|
||||
public static final ConstructingObjectParser<DataStreamLifecycle.Template, LifecycleType> PARSER = new ConstructingObjectParser<>(
|
||||
"lifecycle_template",
|
||||
false,
|
||||
(args, unused) -> new DataStreamLifecycle.Template(
|
||||
(args, lt) -> new DataStreamLifecycle.Template(
|
||||
lt,
|
||||
args[0] == null || (boolean) args[0],
|
||||
args[1] == null ? ResettableValue.undefined() : (ResettableValue<TimeValue>) args[1],
|
||||
args[2] == null ? ResettableValue.undefined() : (ResettableValue<List<DataStreamLifecycle.DownsamplingRound>>) args[2]
|
||||
|
@ -617,7 +708,7 @@ public class DataStreamLifecycle implements SimpleDiffable<DataStreamLifecycle>,
|
|||
if (p.currentToken() == XContentParser.Token.VALUE_NULL) {
|
||||
return ResettableValue.reset();
|
||||
} else {
|
||||
return ResettableValue.create(AbstractObjectParser.parseArray(p, c, DownsamplingRound::fromXContent));
|
||||
return ResettableValue.create(AbstractObjectParser.parseArray(p, null, DownsamplingRound::fromXContent));
|
||||
}
|
||||
}, DOWNSAMPLING_FIELD, ObjectParser.ValueType.OBJECT_ARRAY_OR_NULL);
|
||||
}
|
||||
|
@ -642,6 +733,9 @@ public class DataStreamLifecycle implements SimpleDiffable<DataStreamLifecycle>,
|
|||
}
|
||||
out.writeBoolean(enabled);
|
||||
}
|
||||
if (out.getTransportVersion().onOrAfter(TransportVersions.INTRODUCE_FAILURES_LIFECYCLE)) {
|
||||
lifecycleType.writeTo(out);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -701,11 +795,18 @@ public class DataStreamLifecycle implements SimpleDiffable<DataStreamLifecycle>,
|
|||
}
|
||||
enabled = in.readBoolean();
|
||||
}
|
||||
return new Template(enabled, dataRetention, downsampling);
|
||||
var lifecycleTarget = in.getTransportVersion().onOrAfter(TransportVersions.INTRODUCE_FAILURES_LIFECYCLE)
|
||||
? LifecycleType.read(in)
|
||||
: LifecycleType.DATA;
|
||||
return new Template(lifecycleTarget, enabled, dataRetention, downsampling);
|
||||
}
|
||||
|
||||
public static Template fromXContent(XContentParser parser) throws IOException {
|
||||
return PARSER.parse(parser, null);
|
||||
public static Template dataLifecycleTemplatefromXContent(XContentParser parser) throws IOException {
|
||||
return PARSER.parse(parser, LifecycleType.DATA);
|
||||
}
|
||||
|
||||
public static Template failuresLifecycleTemplatefromXContent(XContentParser parser) throws IOException {
|
||||
return PARSER.parse(parser, LifecycleType.FAILURES);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -745,12 +846,7 @@ public class DataStreamLifecycle implements SimpleDiffable<DataStreamLifecycle>,
|
|||
}
|
||||
|
||||
public DataStreamLifecycle toDataStreamLifecycle() {
|
||||
return new DataStreamLifecycle(enabled, dataRetention.get(), downsampling.get());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return Strings.toString(this, true, true);
|
||||
return new DataStreamLifecycle(lifecycleType, enabled, dataRetention.get(), downsampling.get());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -762,35 +858,53 @@ public class DataStreamLifecycle implements SimpleDiffable<DataStreamLifecycle>,
|
|||
return new Builder(template);
|
||||
}
|
||||
|
||||
/**
|
||||
* This builder factory initialises a builder of a data lifecycle, meaning when it builds we will either get a data lifecycle or a
|
||||
* data lifecycle template.
|
||||
*/
|
||||
public static Builder dataLifecycleBuilder() {
|
||||
return new Builder();
|
||||
return new Builder(LifecycleType.DATA);
|
||||
}
|
||||
|
||||
/**
|
||||
* This builder factory initialises a builder of a failures lifecycle, meaning when it builds we will either get a failures lifecycle or
|
||||
* a failures lifecycle template, if downsampling is not null the final "building" will throw an exception.
|
||||
*/
|
||||
public static Builder failuresLifecycleBuilder() {
|
||||
return new Builder(LifecycleType.FAILURES);
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds and composes the data stream lifecycle or the respective template.
|
||||
*/
|
||||
public static class Builder {
|
||||
private final LifecycleType lifecycleType;
|
||||
private boolean enabled = true;
|
||||
@Nullable
|
||||
private TimeValue dataRetention = null;
|
||||
@Nullable
|
||||
private List<DownsamplingRound> downsampling = null;
|
||||
|
||||
private Builder() {}
|
||||
private Builder(LifecycleType lifecycleType) {
|
||||
this.lifecycleType = lifecycleType;
|
||||
}
|
||||
|
||||
private Builder(DataStreamLifecycle.Template template) {
|
||||
lifecycleType = template.lifecycleType();
|
||||
enabled = template.enabled();
|
||||
dataRetention = template.dataRetention().get();
|
||||
downsampling = template.downsampling().get();
|
||||
}
|
||||
|
||||
private Builder(DataStreamLifecycle lifecycle) {
|
||||
lifecycleType = lifecycle.lifecycleType;
|
||||
enabled = lifecycle.enabled();
|
||||
dataRetention = lifecycle.dataRetention();
|
||||
downsampling = lifecycle.downsampling();
|
||||
}
|
||||
|
||||
public Builder composeTemplate(DataStreamLifecycle.Template template) {
|
||||
assert lifecycleType == template.lifecycleType() : "Trying to compose templates with different lifecycle types";
|
||||
enabled(template.enabled());
|
||||
dataRetention(template.dataRetention());
|
||||
downsampling(template.downsampling());
|
||||
|
@ -827,11 +941,39 @@ public class DataStreamLifecycle implements SimpleDiffable<DataStreamLifecycle>,
|
|||
}
|
||||
|
||||
public DataStreamLifecycle build() {
|
||||
return new DataStreamLifecycle(enabled, dataRetention, downsampling);
|
||||
return new DataStreamLifecycle(lifecycleType, enabled, dataRetention, downsampling);
|
||||
}
|
||||
|
||||
public Template buildTemplate() {
|
||||
return new Template(enabled, dataRetention, downsampling);
|
||||
return new Template(lifecycleType, enabled, dataRetention, downsampling);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Defines the target index component managed by the lifecycle. Currently, it supports data and failures.
|
||||
* Visible for testing
|
||||
*/
|
||||
enum LifecycleType implements Writeable {
|
||||
DATA("data", (byte) 0),
|
||||
FAILURES("failures", (byte) 1);
|
||||
|
||||
private final String label;
|
||||
private final byte id;
|
||||
private static final Map<Byte, LifecycleType> REGISTRY = Arrays.stream(LifecycleType.values())
|
||||
.collect(Collectors.toMap(l -> l.id, Function.identity()));
|
||||
|
||||
LifecycleType(String label, byte id) {
|
||||
this.label = label;
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.write(id);
|
||||
}
|
||||
|
||||
public static LifecycleType read(StreamInput in) throws IOException {
|
||||
return REGISTRY.get(in.readByte());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -9,6 +9,7 @@
|
|||
|
||||
package org.elasticsearch.cluster.metadata;
|
||||
|
||||
import org.elasticsearch.TransportVersions;
|
||||
import org.elasticsearch.cluster.Diff;
|
||||
import org.elasticsearch.cluster.SimpleDiffable;
|
||||
import org.elasticsearch.common.Strings;
|
||||
|
@ -37,8 +38,8 @@ public record DataStreamOptions(@Nullable DataStreamFailureStore failureStore)
|
|||
ToXContentObject {
|
||||
|
||||
public static final ParseField FAILURE_STORE_FIELD = new ParseField(FAILURE_STORE);
|
||||
public static final DataStreamOptions FAILURE_STORE_ENABLED = new DataStreamOptions(new DataStreamFailureStore(true));
|
||||
public static final DataStreamOptions FAILURE_STORE_DISABLED = new DataStreamOptions(new DataStreamFailureStore(false));
|
||||
public static final DataStreamOptions FAILURE_STORE_ENABLED = new DataStreamOptions(new DataStreamFailureStore(true, null));
|
||||
public static final DataStreamOptions FAILURE_STORE_DISABLED = new DataStreamOptions(new DataStreamFailureStore(false, null));
|
||||
public static final DataStreamOptions EMPTY = new DataStreamOptions(null);
|
||||
|
||||
public static final ConstructingObjectParser<DataStreamOptions, Void> PARSER = new ConstructingObjectParser<>(
|
||||
|
@ -72,7 +73,15 @@ public record DataStreamOptions(@Nullable DataStreamFailureStore failureStore)
|
|||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeOptionalWriteable(failureStore);
|
||||
if (out.getTransportVersion().onOrAfter(TransportVersions.INTRODUCE_FAILURES_LIFECYCLE)
|
||||
|| failureStore == null
|
||||
|| failureStore().enabled() != null) {
|
||||
out.writeOptionalWriteable(failureStore);
|
||||
} else {
|
||||
// When communicating with older versions we need to ensure we do not sent an invalid failure store config.
|
||||
// If the enabled flag is not defined, we treat it as null.
|
||||
out.writeOptionalWriteable(null);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -129,7 +138,18 @@ public record DataStreamOptions(@Nullable DataStreamFailureStore failureStore)
|
|||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
ResettableValue.write(out, failureStore, (o, v) -> v.writeTo(o));
|
||||
if (out.getTransportVersion().onOrAfter(TransportVersions.INTRODUCE_FAILURES_LIFECYCLE)
|
||||
|| failureStore.get() == null
|
||||
|| failureStore().mapAndGet(DataStreamFailureStore.Template::enabled).get() != null) {
|
||||
ResettableValue.write(out, failureStore, (o, v) -> v.writeTo(o));
|
||||
// When communicating with older versions we need to ensure we do not sent an invalid failure store config.
|
||||
} else {
|
||||
// If the enabled flag is not defined, we treat failure store as not defined, if reset we treat the failure store as reset
|
||||
ResettableValue<DataStreamFailureStore.Template> bwcFailureStore = failureStore.get().enabled().shouldReset()
|
||||
? ResettableValue.reset()
|
||||
: ResettableValue.undefined();
|
||||
ResettableValue.write(out, bwcFailureStore, (o, v) -> v.writeTo(o));
|
||||
}
|
||||
}
|
||||
|
||||
public static Template read(StreamInput in) throws IOException {
|
||||
|
@ -182,7 +202,7 @@ public record DataStreamOptions(@Nullable DataStreamFailureStore failureStore)
|
|||
}
|
||||
|
||||
/**
|
||||
* Updates this builder with the values of the provided template. This is not a replacement necessarily, the
|
||||
* Composes this builder with the values of the provided template. This is not a replacement necessarily, the
|
||||
* inner values will be merged.
|
||||
*/
|
||||
public Builder composeTemplate(DataStreamOptions.Template options) {
|
||||
|
@ -190,7 +210,7 @@ public record DataStreamOptions(@Nullable DataStreamFailureStore failureStore)
|
|||
}
|
||||
|
||||
/**
|
||||
* Updates the current failure store configuration with the provided value. This is not a replacement necessarily, if both
|
||||
* Composes the current failure store configuration with the provided value. This is not a replacement necessarily, if both
|
||||
* instance contain data the configurations are merged.
|
||||
*/
|
||||
public Builder failureStore(ResettableValue<DataStreamFailureStore.Template> newFailureStore) {
|
||||
|
|
|
@ -1154,7 +1154,9 @@ public class IndexNameExpressionResolver {
|
|||
if (requiredAliases == null) {
|
||||
requiredAliases = new ArrayList<>(aliasesForDataStream.size());
|
||||
}
|
||||
String alias = isData ? dataStreamAlias.getName() : dataStreamAlias.getName() + "::failures";
|
||||
String alias = isData
|
||||
? dataStreamAlias.getName()
|
||||
: combineSelector(dataStreamAlias.getName(), IndexComponentSelector.FAILURES);
|
||||
requiredAliases.add(alias);
|
||||
} else {
|
||||
// we have a non-required alias for this data stream so no need to check further
|
||||
|
|
|
@ -368,9 +368,17 @@ public class MetadataDataStreamsService {
|
|||
@Nullable DataStreamOptions dataStreamOptions
|
||||
) {
|
||||
ProjectMetadata.Builder builder = ProjectMetadata.builder(project);
|
||||
boolean onlyInternalDataStreams = true;
|
||||
for (var dataStreamName : dataStreamNames) {
|
||||
var dataStream = validateDataStream(project, dataStreamName);
|
||||
builder.put(dataStream.copy().setDataStreamOptions(dataStreamOptions).build());
|
||||
onlyInternalDataStreams = onlyInternalDataStreams && dataStream.isInternal();
|
||||
}
|
||||
if (dataStreamOptions != null && dataStreamOptions.failureStore() != null && dataStreamOptions.failureStore().lifecycle() != null) {
|
||||
// We don't issue any warnings if all data streams are internal data streams
|
||||
dataStreamOptions.failureStore()
|
||||
.lifecycle()
|
||||
.addWarningHeaderIfDataRetentionNotEffective(globalRetentionSettings.get(), onlyInternalDataStreams);
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
|
|
|
@ -93,7 +93,7 @@ public class Template implements SimpleDiffable<Template>, ToXContentObject {
|
|||
}, ALIASES);
|
||||
PARSER.declareObject(
|
||||
ConstructingObjectParser.optionalConstructorArg(),
|
||||
(p, c) -> DataStreamLifecycle.Template.fromXContent(p),
|
||||
(p, c) -> DataStreamLifecycle.Template.dataLifecycleTemplatefromXContent(p),
|
||||
LIFECYCLE
|
||||
);
|
||||
PARSER.declareObjectOrNull(
|
||||
|
@ -125,6 +125,8 @@ public class Template implements SimpleDiffable<Template>, ToXContentObject {
|
|||
this.settings = settings;
|
||||
this.mappings = mappings;
|
||||
this.aliases = aliases;
|
||||
assert lifecycle == null || lifecycle.toDataStreamLifecycle().targetsFailureStore() == false
|
||||
: "Invalid lifecycle type for data lifecycle";
|
||||
this.lifecycle = lifecycle;
|
||||
assert dataStreamOptions != null : "Template does not accept null values, please use Resettable.undefined()";
|
||||
this.dataStreamOptions = dataStreamOptions;
|
||||
|
|
|
@ -29,8 +29,9 @@ import static org.elasticsearch.rest.RestUtils.getMasterNodeTimeout;
|
|||
@ServerlessScope(Scope.PUBLIC)
|
||||
public class RestPutComponentTemplateAction extends BaseRestHandler {
|
||||
|
||||
public static final String SUPPORTS_FAILURE_STORE_LIFECYCLE = "data_stream_options.failure_store.lifecycle";
|
||||
public static final String SUPPORTS_FAILURE_STORE = "data_stream_options.failure_store";
|
||||
private static final Set<String> capabilities = Set.of(SUPPORTS_FAILURE_STORE);
|
||||
private static final Set<String> capabilities = Set.of(SUPPORTS_FAILURE_STORE, SUPPORTS_FAILURE_STORE_LIFECYCLE);
|
||||
|
||||
@Override
|
||||
public List<Route> routes() {
|
||||
|
|
|
@ -26,11 +26,12 @@ import static org.elasticsearch.rest.RestRequest.Method.POST;
|
|||
import static org.elasticsearch.rest.RestRequest.Method.PUT;
|
||||
import static org.elasticsearch.rest.RestUtils.getMasterNodeTimeout;
|
||||
import static org.elasticsearch.rest.action.admin.indices.RestPutComponentTemplateAction.SUPPORTS_FAILURE_STORE;
|
||||
import static org.elasticsearch.rest.action.admin.indices.RestPutComponentTemplateAction.SUPPORTS_FAILURE_STORE_LIFECYCLE;
|
||||
|
||||
@ServerlessScope(Scope.PUBLIC)
|
||||
public class RestPutComposableIndexTemplateAction extends BaseRestHandler {
|
||||
|
||||
private static final Set<String> capabilities = Set.of(SUPPORTS_FAILURE_STORE);
|
||||
private static final Set<String> capabilities = Set.of(SUPPORTS_FAILURE_STORE, SUPPORTS_FAILURE_STORE_LIFECYCLE);
|
||||
|
||||
@Override
|
||||
public List<Route> routes() {
|
||||
|
|
|
@ -34,7 +34,6 @@ public class DataStreamFailureStoreDefinitionTests extends ESTestCase {
|
|||
.put(IndexMetadata.SETTING_INDEX_HIDDEN, true)
|
||||
.put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, "0-10")
|
||||
.put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), "1s")
|
||||
.put(IndexMetadata.LIFECYCLE_NAME, "my-policy")
|
||||
.put(IndexMetadata.INDEX_ROUTING_REQUIRE_GROUP_PREFIX + "." + randomAlphaOfLength(4), randomAlphaOfLength(4))
|
||||
.put(IndexMetadata.INDEX_ROUTING_INCLUDE_GROUP_PREFIX + "." + randomAlphaOfLength(4), randomAlphaOfLength(4))
|
||||
.put(IndexMetadata.INDEX_ROUTING_EXCLUDE_GROUP_PREFIX + "." + randomAlphaOfLength(4), randomAlphaOfLength(4));
|
||||
|
@ -61,7 +60,7 @@ public class DataStreamFailureStoreDefinitionTests extends ESTestCase {
|
|||
expectedBuilder = Settings.builder().put(builder.build());
|
||||
assertThat(
|
||||
DataStreamFailureStoreDefinition.filterUserDefinedSettings(builder).keys().size(),
|
||||
equalTo(expectedBuilder.keys().size() - 2)
|
||||
equalTo(expectedBuilder.keys().size() - 3)
|
||||
);
|
||||
assertThat(
|
||||
DataStreamFailureStoreDefinition.filterUserDefinedSettings(builder).keys().contains(IndexSettings.MODE.getKey()),
|
||||
|
|
|
@ -10,6 +10,7 @@
|
|||
package org.elasticsearch.cluster.metadata;
|
||||
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.test.AbstractXContentSerializingTestCase;
|
||||
import org.elasticsearch.xcontent.XContentParser;
|
||||
|
||||
|
@ -17,6 +18,8 @@ import java.io.IOException;
|
|||
|
||||
import static org.hamcrest.Matchers.containsString;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.not;
|
||||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
||||
public class DataStreamFailureStoreTemplateTests extends AbstractXContentSerializingTestCase<DataStreamFailureStore.Template> {
|
||||
|
||||
|
@ -32,7 +35,20 @@ public class DataStreamFailureStoreTemplateTests extends AbstractXContentSeriali
|
|||
|
||||
@Override
|
||||
protected DataStreamFailureStore.Template mutateInstance(DataStreamFailureStore.Template instance) {
|
||||
return new DataStreamFailureStore.Template(instance.enabled().map(v -> v == false));
|
||||
var enabled = instance.enabled();
|
||||
var lifecycle = instance.lifecycle();
|
||||
switch (randomIntBetween(0, 1)) {
|
||||
case 0 -> enabled = enabled.get() != null && lifecycle.get() != null && randomBoolean()
|
||||
? randomEmptyResettableValue()
|
||||
: ResettableValue.create(Boolean.FALSE.equals(enabled.get()));
|
||||
case 1 -> lifecycle = lifecycle.get() != null && enabled.get() != null && randomBoolean()
|
||||
? randomEmptyResettableValue()
|
||||
: ResettableValue.create(
|
||||
randomValueOtherThan(lifecycle.get(), DataStreamLifecycleTemplateTests::randomFailuresLifecycleTemplate)
|
||||
);
|
||||
default -> throw new IllegalArgumentException("illegal randomisation branch");
|
||||
}
|
||||
return new DataStreamFailureStore.Template(enabled, lifecycle);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -41,26 +57,98 @@ public class DataStreamFailureStoreTemplateTests extends AbstractXContentSeriali
|
|||
}
|
||||
|
||||
static DataStreamFailureStore.Template randomFailureStoreTemplate() {
|
||||
return new DataStreamFailureStore.Template(randomBoolean());
|
||||
boolean enabledDefined = randomBoolean();
|
||||
boolean lifecycleDefined = enabledDefined == false || randomBoolean();
|
||||
return new DataStreamFailureStore.Template(
|
||||
enabledDefined ? ResettableValue.create(randomBoolean()) : randomEmptyResettableValue(),
|
||||
lifecycleDefined
|
||||
? ResettableValue.create(DataStreamLifecycleTemplateTests.randomFailuresLifecycleTemplate())
|
||||
: randomEmptyResettableValue()
|
||||
);
|
||||
}
|
||||
|
||||
public void testInvalidEmptyConfiguration() {
|
||||
Exception exception = expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> new DataStreamFailureStore.Template(randomBoolean() ? ResettableValue.undefined() : ResettableValue.reset())
|
||||
() -> new DataStreamFailureStore.Template(ResettableValue.undefined(), ResettableValue.undefined())
|
||||
);
|
||||
assertThat(exception.getMessage(), containsString("at least one non-null configuration value"));
|
||||
}
|
||||
|
||||
public void testTemplateComposition() {
|
||||
// Merging a template with itself, remains the same
|
||||
boolean enabled = randomBoolean();
|
||||
DataStreamFailureStore.Template template = new DataStreamFailureStore.Template(enabled);
|
||||
DataStreamFailureStore.Template template = new DataStreamFailureStore.Template(
|
||||
enabled,
|
||||
randomBoolean() ? null : DataStreamLifecycleTemplateTests.randomFailuresLifecycleTemplate()
|
||||
);
|
||||
DataStreamFailureStore.Template result = DataStreamFailureStore.builder(template).composeTemplate(template).buildTemplate();
|
||||
assertThat(result, equalTo(template));
|
||||
assertThat(result, equalTo(normalise(template)));
|
||||
|
||||
DataStreamFailureStore.Template negatedTemplate = new DataStreamFailureStore.Template(enabled == false);
|
||||
result = DataStreamFailureStore.builder(template).composeTemplate(negatedTemplate).buildTemplate();
|
||||
;
|
||||
assertThat(result, equalTo(negatedTemplate));
|
||||
// Override only enabled and keep lifecycle undefined
|
||||
DataStreamFailureStore.Template negatedEnabledTemplate = DataStreamFailureStore.builder(template)
|
||||
.enabled(enabled == false)
|
||||
.buildTemplate();
|
||||
result = DataStreamFailureStore.builder(template).composeTemplate(negatedEnabledTemplate).buildTemplate();
|
||||
assertThat(result, equalTo(normalise(new DataStreamFailureStore.Template(enabled == false, template.lifecycle().get()))));
|
||||
|
||||
// Override only lifecycle and ensure it is merged
|
||||
enabled = false; // Ensure it's not the default to ensure that it will not be overwritten
|
||||
TimeValue retention = randomPositiveTimeValue();
|
||||
DataStreamFailureStore.Template template1 = DataStreamFailureStore.builder()
|
||||
.lifecycle(DataStreamLifecycle.failuresLifecycleBuilder().dataRetention(retention).build())
|
||||
.buildTemplate();
|
||||
DataStreamFailureStore.Template template2 = DataStreamFailureStore.builder()
|
||||
.lifecycle(DataStreamLifecycle.failuresLifecycleBuilder().enabled(enabled).build())
|
||||
.buildTemplate();
|
||||
result = DataStreamFailureStore.builder(template1).composeTemplate(template2).buildTemplate();
|
||||
assertThat(result.lifecycle().get().enabled(), equalTo(enabled));
|
||||
assertThat(result.lifecycle().get().dataRetention().get(), equalTo(retention));
|
||||
|
||||
// Test reset
|
||||
DataStreamFailureStore.Template fullyFilledTemplate = DataStreamFailureStore.builder()
|
||||
.enabled(ResettableValue.create(randomBoolean()))
|
||||
.lifecycle(DataStreamLifecycleTests.randomFailuresLifecycle())
|
||||
.buildTemplate();
|
||||
result = DataStreamFailureStore.builder(fullyFilledTemplate)
|
||||
.composeTemplate(
|
||||
new DataStreamFailureStore.Template(
|
||||
ResettableValue.reset(),
|
||||
ResettableValue.create(DataStreamLifecycleTemplateTests.randomFailuresLifecycleTemplate())
|
||||
)
|
||||
)
|
||||
.buildTemplate();
|
||||
assertThat(result.enabled(), equalTo(ResettableValue.undefined()));
|
||||
assertThat(result.lifecycle(), not(equalTo(ResettableValue.undefined())));
|
||||
result = DataStreamFailureStore.builder(fullyFilledTemplate)
|
||||
.composeTemplate(new DataStreamFailureStore.Template(ResettableValue.create(randomBoolean()), ResettableValue.reset()))
|
||||
.buildTemplate();
|
||||
assertThat(result.enabled(), not(equalTo(ResettableValue.undefined())));
|
||||
assertThat(result.lifecycle(), equalTo(ResettableValue.undefined()));
|
||||
|
||||
// Test resetting all values
|
||||
result = DataStreamFailureStore.builder(fullyFilledTemplate)
|
||||
.composeTemplate(new DataStreamFailureStore.Template(ResettableValue.reset(), ResettableValue.reset()))
|
||||
.buildTemplate();
|
||||
assertThat(result, nullValue());
|
||||
}
|
||||
|
||||
private static <T> ResettableValue<T> randomEmptyResettableValue() {
|
||||
return randomBoolean() ? ResettableValue.undefined() : ResettableValue.reset();
|
||||
}
|
||||
|
||||
private static DataStreamFailureStore.Template normalise(DataStreamFailureStore.Template failureStoreTemplate) {
|
||||
return new DataStreamFailureStore.Template(
|
||||
failureStoreTemplate.enabled(),
|
||||
failureStoreTemplate.lifecycle()
|
||||
.map(
|
||||
template -> new DataStreamLifecycle.Template(
|
||||
template.lifecycleType(),
|
||||
template.enabled(),
|
||||
template.dataRetention().get(),
|
||||
template.downsampling().get()
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,7 +31,16 @@ public class DataStreamFailureStoreTests extends AbstractXContentSerializingTest
|
|||
|
||||
@Override
|
||||
protected DataStreamFailureStore mutateInstance(DataStreamFailureStore instance) {
|
||||
return new DataStreamFailureStore(instance.enabled() == false);
|
||||
var enabled = instance.enabled();
|
||||
var lifecycle = instance.lifecycle();
|
||||
switch (randomIntBetween(0, 1)) {
|
||||
case 0 -> enabled = enabled != null && lifecycle != null && randomBoolean() ? null : Boolean.FALSE.equals(enabled);
|
||||
case 1 -> lifecycle = lifecycle != null && enabled != null && randomBoolean()
|
||||
? null
|
||||
: randomValueOtherThan(lifecycle, DataStreamLifecycleTests::randomFailuresLifecycle);
|
||||
default -> throw new IllegalArgumentException("illegal randomisation branch");
|
||||
}
|
||||
return new DataStreamFailureStore(enabled, lifecycle);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -40,11 +49,16 @@ public class DataStreamFailureStoreTests extends AbstractXContentSerializingTest
|
|||
}
|
||||
|
||||
static DataStreamFailureStore randomFailureStore() {
|
||||
return new DataStreamFailureStore(randomBoolean());
|
||||
boolean enabledDefined = randomBoolean();
|
||||
boolean lifecycleDefined = enabledDefined == false || randomBoolean();
|
||||
return new DataStreamFailureStore(
|
||||
enabledDefined ? randomBoolean() : null,
|
||||
lifecycleDefined ? DataStreamLifecycleTests.randomFailuresLifecycle() : null
|
||||
);
|
||||
}
|
||||
|
||||
public void testInvalidEmptyConfiguration() {
|
||||
IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> new DataStreamFailureStore((Boolean) null));
|
||||
IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> new DataStreamFailureStore(null, null));
|
||||
assertThat(exception.getMessage(), containsString("at least one non-null configuration value"));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -10,19 +10,24 @@
|
|||
package org.elasticsearch.cluster.metadata;
|
||||
|
||||
import org.elasticsearch.action.downsample.DownsampleConfig;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
|
||||
import org.elasticsearch.test.AbstractXContentSerializingTestCase;
|
||||
import org.elasticsearch.test.AbstractWireSerializingTestCase;
|
||||
import org.elasticsearch.xcontent.ToXContent;
|
||||
import org.elasticsearch.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.xcontent.XContentParser;
|
||||
import org.elasticsearch.xcontent.XContentType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.hamcrest.Matchers.containsString;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
||||
public class DataStreamLifecycleTemplateTests extends AbstractXContentSerializingTestCase<DataStreamLifecycle.Template> {
|
||||
public class DataStreamLifecycleTemplateTests extends AbstractWireSerializingTestCase<DataStreamLifecycle.Template> {
|
||||
|
||||
@Override
|
||||
protected Writeable.Reader<DataStreamLifecycle.Template> instanceReader() {
|
||||
|
@ -31,26 +36,67 @@ public class DataStreamLifecycleTemplateTests extends AbstractXContentSerializin
|
|||
|
||||
@Override
|
||||
protected DataStreamLifecycle.Template createTestInstance() {
|
||||
return randomDataLifecycleTemplate();
|
||||
return randomBoolean() ? randomDataLifecycleTemplate() : randomFailuresLifecycleTemplate();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DataStreamLifecycle.Template mutateInstance(DataStreamLifecycle.Template instance) throws IOException {
|
||||
var lifecycleTarget = instance.lifecycleType();
|
||||
var enabled = instance.enabled();
|
||||
var retention = instance.dataRetention();
|
||||
var downsampling = instance.downsampling();
|
||||
switch (randomInt(2)) {
|
||||
case 0 -> enabled = enabled == false;
|
||||
case 1 -> retention = randomValueOtherThan(retention, DataStreamLifecycleTemplateTests::randomRetention);
|
||||
case 2 -> downsampling = randomValueOtherThan(downsampling, DataStreamLifecycleTemplateTests::randomDownsampling);
|
||||
switch (randomInt(3)) {
|
||||
case 0 -> {
|
||||
lifecycleTarget = lifecycleTarget == DataStreamLifecycle.LifecycleType.DATA
|
||||
? DataStreamLifecycle.LifecycleType.FAILURES
|
||||
: DataStreamLifecycle.LifecycleType.DATA;
|
||||
if (lifecycleTarget == DataStreamLifecycle.LifecycleType.FAILURES) {
|
||||
downsampling = ResettableValue.undefined();
|
||||
}
|
||||
}
|
||||
case 1 -> enabled = enabled == false;
|
||||
case 2 -> retention = randomValueOtherThan(retention, DataStreamLifecycleTemplateTests::randomRetention);
|
||||
case 3 -> {
|
||||
downsampling = randomValueOtherThan(downsampling, DataStreamLifecycleTemplateTests::randomDownsampling);
|
||||
if (downsampling.get() != null) {
|
||||
lifecycleTarget = DataStreamLifecycle.LifecycleType.DATA;
|
||||
}
|
||||
}
|
||||
default -> throw new AssertionError("Illegal randomisation branch");
|
||||
}
|
||||
return new DataStreamLifecycle.Template(enabled, retention, downsampling);
|
||||
return new DataStreamLifecycle.Template(lifecycleTarget, enabled, retention, downsampling);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DataStreamLifecycle.Template doParseInstance(XContentParser parser) throws IOException {
|
||||
return DataStreamLifecycle.Template.fromXContent(parser);
|
||||
public void testDataLifecycleXContentSerialization() throws IOException {
|
||||
DataStreamLifecycle.Template lifecycle = randomDataLifecycleTemplate();
|
||||
try (XContentBuilder builder = XContentBuilder.builder(XContentType.JSON.xContent())) {
|
||||
builder.humanReadable(true);
|
||||
lifecycle.toXContent(builder, ToXContent.EMPTY_PARAMS, null, null, randomBoolean());
|
||||
String lifecycleJson = Strings.toString(builder);
|
||||
try (XContentParser parser = createParser(XContentType.JSON.xContent(), lifecycleJson)) {
|
||||
assertEquals(XContentParser.Token.START_OBJECT, parser.nextToken());
|
||||
var parsed = DataStreamLifecycle.Template.dataLifecycleTemplatefromXContent(parser);
|
||||
assertEquals(XContentParser.Token.END_OBJECT, parser.currentToken());
|
||||
assertNull(parser.nextToken());
|
||||
assertThat(parsed, equalTo(lifecycle));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testFailuresLifecycleXContentSerialization() throws IOException {
|
||||
DataStreamLifecycle.Template lifecycle = randomFailuresLifecycleTemplate();
|
||||
try (XContentBuilder builder = XContentBuilder.builder(XContentType.JSON.xContent())) {
|
||||
builder.humanReadable(true);
|
||||
lifecycle.toXContent(builder, ToXContent.EMPTY_PARAMS, null, null, randomBoolean());
|
||||
String lifecycleJson = Strings.toString(builder);
|
||||
try (XContentParser parser = createParser(XContentType.JSON.xContent(), lifecycleJson)) {
|
||||
assertEquals(XContentParser.Token.START_OBJECT, parser.nextToken());
|
||||
var parsed = DataStreamLifecycle.Template.failuresLifecycleTemplatefromXContent(parser);
|
||||
assertEquals(XContentParser.Token.END_OBJECT, parser.currentToken());
|
||||
assertNull(parser.nextToken());
|
||||
assertThat(parsed, equalTo(lifecycle));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testInvalidDownsamplingConfiguration() {
|
||||
|
@ -169,6 +215,35 @@ public class DataStreamLifecycleTemplateTests extends AbstractXContentSerializin
|
|||
return DataStreamLifecycle.createDataLifecycleTemplate(randomBoolean(), randomRetention(), randomDownsampling());
|
||||
}
|
||||
|
||||
public void testInvalidLifecycleConfiguration() {
|
||||
IllegalArgumentException exception = expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> new DataStreamLifecycle.Template(
|
||||
DataStreamLifecycle.LifecycleType.FAILURES,
|
||||
randomBoolean(),
|
||||
randomBoolean() ? null : DataStreamLifecycleTests.randomPositiveTimeValue(),
|
||||
DataStreamLifecycleTests.randomDownsampling()
|
||||
)
|
||||
);
|
||||
assertThat(
|
||||
exception.getMessage(),
|
||||
containsString("Failure store lifecycle does not support downsampling, please remove the downsampling configuration.")
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Failure store lifecycle doesn't support downsampling, this random lifecycle generator never defines
|
||||
* downsampling.
|
||||
*/
|
||||
public static DataStreamLifecycle.Template randomFailuresLifecycleTemplate() {
|
||||
return new DataStreamLifecycle.Template(
|
||||
DataStreamLifecycle.LifecycleType.FAILURES,
|
||||
randomBoolean(),
|
||||
randomRetention(),
|
||||
ResettableValue.undefined()
|
||||
);
|
||||
}
|
||||
|
||||
private static ResettableValue<TimeValue> randomRetention() {
|
||||
return switch (randomIntBetween(0, 2)) {
|
||||
case 0 -> ResettableValue.undefined();
|
||||
|
|
|
@ -21,7 +21,7 @@ import org.elasticsearch.common.unit.ByteSizeValue;
|
|||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.core.Tuple;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
|
||||
import org.elasticsearch.test.AbstractXContentSerializingTestCase;
|
||||
import org.elasticsearch.test.AbstractWireSerializingTestCase;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.xcontent.ToXContent;
|
||||
import org.elasticsearch.xcontent.XContentBuilder;
|
||||
|
@ -44,7 +44,7 @@ import static org.hamcrest.Matchers.equalTo;
|
|||
import static org.hamcrest.Matchers.not;
|
||||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
||||
public class DataStreamLifecycleTests extends AbstractXContentSerializingTestCase<DataStreamLifecycle> {
|
||||
public class DataStreamLifecycleTests extends AbstractWireSerializingTestCase<DataStreamLifecycle> {
|
||||
|
||||
@Override
|
||||
protected Writeable.Reader<DataStreamLifecycle> instanceReader() {
|
||||
|
@ -53,25 +53,39 @@ public class DataStreamLifecycleTests extends AbstractXContentSerializingTestCas
|
|||
|
||||
@Override
|
||||
protected DataStreamLifecycle createTestInstance() {
|
||||
return randomDataLifecycle();
|
||||
return randomBoolean() ? randomDataLifecycle() : randomFailuresLifecycle();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DataStreamLifecycle mutateInstance(DataStreamLifecycle instance) throws IOException {
|
||||
var lifecycleTarget = instance.targetsFailureStore()
|
||||
? DataStreamLifecycle.LifecycleType.FAILURES
|
||||
: DataStreamLifecycle.LifecycleType.DATA;
|
||||
var enabled = instance.enabled();
|
||||
var retention = instance.dataRetention();
|
||||
var downsampling = instance.downsampling();
|
||||
switch (randomInt(2)) {
|
||||
switch (randomInt(3)) {
|
||||
case 0 -> {
|
||||
if (instance.targetsFailureStore()) {
|
||||
lifecycleTarget = DataStreamLifecycle.LifecycleType.DATA;
|
||||
} else {
|
||||
lifecycleTarget = DataStreamLifecycle.LifecycleType.FAILURES;
|
||||
downsampling = null;
|
||||
}
|
||||
}
|
||||
case 1 -> {
|
||||
if (retention == null) {
|
||||
retention = randomPositiveTimeValue();
|
||||
} else {
|
||||
retention = randomBoolean() ? null : randomValueOtherThan(retention, ESTestCase::randomPositiveTimeValue);
|
||||
}
|
||||
}
|
||||
case 1 -> {
|
||||
case 2 -> {
|
||||
if (downsampling == null) {
|
||||
downsampling = randomDownsampling();
|
||||
if (lifecycleTarget == DataStreamLifecycle.LifecycleType.FAILURES) {
|
||||
lifecycleTarget = DataStreamLifecycle.LifecycleType.DATA;
|
||||
}
|
||||
} else {
|
||||
downsampling = randomBoolean()
|
||||
? null
|
||||
|
@ -80,12 +94,39 @@ public class DataStreamLifecycleTests extends AbstractXContentSerializingTestCas
|
|||
}
|
||||
default -> enabled = enabled == false;
|
||||
}
|
||||
return new DataStreamLifecycle(enabled, retention, downsampling);
|
||||
return new DataStreamLifecycle(lifecycleTarget, enabled, retention, downsampling);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DataStreamLifecycle doParseInstance(XContentParser parser) throws IOException {
|
||||
return DataStreamLifecycle.fromXContent(parser);
|
||||
public void testDataLifecycleXContentSerialization() throws IOException {
|
||||
DataStreamLifecycle lifecycle = randomDataLifecycle();
|
||||
try (XContentBuilder builder = XContentBuilder.builder(XContentType.JSON.xContent())) {
|
||||
builder.humanReadable(true);
|
||||
lifecycle.toXContent(builder, ToXContent.EMPTY_PARAMS, null, null, randomBoolean());
|
||||
String lifecycleJson = Strings.toString(builder);
|
||||
try (XContentParser parser = createParser(XContentType.JSON.xContent(), lifecycleJson)) {
|
||||
assertEquals(XContentParser.Token.START_OBJECT, parser.nextToken());
|
||||
var parsed = DataStreamLifecycle.dataLifecycleFromXContent(parser);
|
||||
assertEquals(XContentParser.Token.END_OBJECT, parser.currentToken());
|
||||
assertNull(parser.nextToken());
|
||||
assertThat(parsed, equalTo(lifecycle));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testFailuresLifecycleXContentSerialization() throws IOException {
|
||||
DataStreamLifecycle lifecycle = randomFailuresLifecycle();
|
||||
try (XContentBuilder builder = XContentBuilder.builder(XContentType.JSON.xContent())) {
|
||||
builder.humanReadable(true);
|
||||
lifecycle.toXContent(builder, ToXContent.EMPTY_PARAMS, null, null, randomBoolean());
|
||||
String lifecycleJson = Strings.toString(builder);
|
||||
try (XContentParser parser = createParser(XContentType.JSON.xContent(), lifecycleJson)) {
|
||||
assertEquals(XContentParser.Token.START_OBJECT, parser.nextToken());
|
||||
var parsed = DataStreamLifecycle.failureLifecycleFromXContent(parser);
|
||||
assertEquals(XContentParser.Token.END_OBJECT, parser.currentToken());
|
||||
assertNull(parser.nextToken());
|
||||
assertThat(parsed, equalTo(lifecycle));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testXContentSerializationWithRolloverAndEffectiveRetention() throws IOException {
|
||||
|
@ -132,7 +173,7 @@ public class DataStreamLifecycleTests extends AbstractXContentSerializingTestCas
|
|||
""";
|
||||
try (XContentParser parser = createParser(XContentType.JSON.xContent(), lifecycleJson)) {
|
||||
assertEquals(XContentParser.Token.START_OBJECT, parser.nextToken());
|
||||
var parsed = DataStreamLifecycle.fromXContent(parser);
|
||||
var parsed = DataStreamLifecycle.dataLifecycleFromXContent(parser);
|
||||
assertEquals(XContentParser.Token.END_OBJECT, parser.currentToken());
|
||||
assertNull(parser.nextToken());
|
||||
assertThat(parsed, equalTo(DataStreamLifecycle.DEFAULT_DATA_LIFECYCLE));
|
||||
|
@ -283,61 +324,97 @@ public class DataStreamLifecycleTests extends AbstractXContentSerializingTestCas
|
|||
public void testEffectiveRetention() {
|
||||
// No retention in the data stream lifecycle
|
||||
{
|
||||
DataStreamLifecycle noRetentionLifecycle = DataStreamLifecycle.dataLifecycleBuilder()
|
||||
DataStreamLifecycle noDataRetentionLifecycle = DataStreamLifecycle.dataLifecycleBuilder()
|
||||
.downsampling(randomDownsampling())
|
||||
.build();
|
||||
DataStreamLifecycle noFailuresRetentionLifecycle = DataStreamLifecycle.failuresLifecycleBuilder().build();
|
||||
TimeValue maxRetention = TimeValue.timeValueDays(randomIntBetween(50, 100));
|
||||
TimeValue defaultRetention = TimeValue.timeValueDays(randomIntBetween(1, 50));
|
||||
Tuple<TimeValue, DataStreamLifecycle.RetentionSource> effectiveDataRetentionWithSource = noRetentionLifecycle
|
||||
|
||||
// No global retention
|
||||
// Data lifecycle
|
||||
Tuple<TimeValue, DataStreamLifecycle.RetentionSource> effectiveDataRetentionWithSource = noDataRetentionLifecycle
|
||||
.getEffectiveDataRetentionWithSource(null, randomBoolean());
|
||||
assertThat(effectiveDataRetentionWithSource.v1(), nullValue());
|
||||
assertThat(effectiveDataRetentionWithSource.v2(), equalTo(DATA_STREAM_CONFIGURATION));
|
||||
// Failures lifecycle
|
||||
Tuple<TimeValue, DataStreamLifecycle.RetentionSource> effectiveFailuresRetentionWithSource = noFailuresRetentionLifecycle
|
||||
.getEffectiveDataRetentionWithSource(null, randomBoolean());
|
||||
assertThat(effectiveFailuresRetentionWithSource.v1(), nullValue());
|
||||
assertThat(effectiveFailuresRetentionWithSource.v2(), equalTo(DATA_STREAM_CONFIGURATION));
|
||||
|
||||
effectiveDataRetentionWithSource = noRetentionLifecycle.getEffectiveDataRetentionWithSource(
|
||||
new DataStreamGlobalRetention(null, maxRetention),
|
||||
false
|
||||
);
|
||||
// Only max retention
|
||||
DataStreamGlobalRetention onlyMaxRetention = new DataStreamGlobalRetention(null, maxRetention);
|
||||
// Data lifecycle
|
||||
effectiveDataRetentionWithSource = noDataRetentionLifecycle.getEffectiveDataRetentionWithSource(onlyMaxRetention, false);
|
||||
assertThat(effectiveDataRetentionWithSource.v1(), equalTo(maxRetention));
|
||||
assertThat(effectiveDataRetentionWithSource.v2(), equalTo(MAX_GLOBAL_RETENTION));
|
||||
|
||||
effectiveDataRetentionWithSource = noRetentionLifecycle.getEffectiveDataRetentionWithSource(
|
||||
// Default configuration with data lifecycle
|
||||
effectiveDataRetentionWithSource = noDataRetentionLifecycle.getEffectiveDataRetentionWithSource(
|
||||
new DataStreamGlobalRetention(defaultRetention, null),
|
||||
false
|
||||
);
|
||||
assertThat(effectiveDataRetentionWithSource.v1(), equalTo(defaultRetention));
|
||||
assertThat(effectiveDataRetentionWithSource.v2(), equalTo(DEFAULT_GLOBAL_RETENTION));
|
||||
|
||||
effectiveDataRetentionWithSource = noRetentionLifecycle.getEffectiveDataRetentionWithSource(
|
||||
// Complete global retention with data lifecycle
|
||||
effectiveDataRetentionWithSource = noDataRetentionLifecycle.getEffectiveDataRetentionWithSource(
|
||||
new DataStreamGlobalRetention(defaultRetention, maxRetention),
|
||||
false
|
||||
);
|
||||
assertThat(effectiveDataRetentionWithSource.v1(), equalTo(defaultRetention));
|
||||
assertThat(effectiveDataRetentionWithSource.v2(), equalTo(DEFAULT_GLOBAL_RETENTION));
|
||||
|
||||
// Complete global retention with failures lifecycle
|
||||
effectiveFailuresRetentionWithSource = noFailuresRetentionLifecycle.getEffectiveDataRetentionWithSource(
|
||||
new DataStreamGlobalRetention(defaultRetention, maxRetention),
|
||||
false
|
||||
);
|
||||
assertThat(effectiveFailuresRetentionWithSource.v1(), equalTo(defaultRetention));
|
||||
assertThat(effectiveFailuresRetentionWithSource.v2(), equalTo(DEFAULT_GLOBAL_RETENTION));
|
||||
}
|
||||
|
||||
// With retention in the data stream lifecycle
|
||||
{
|
||||
TimeValue dataStreamRetention = TimeValue.timeValueDays(randomIntBetween(5, 100));
|
||||
DataStreamLifecycle lifecycleRetention = DataStreamLifecycle.dataLifecycleBuilder()
|
||||
DataStreamLifecycle dataLifecycleRetention = DataStreamLifecycle.dataLifecycleBuilder()
|
||||
.dataRetention(dataStreamRetention)
|
||||
.downsampling(randomDownsampling())
|
||||
.build();
|
||||
DataStreamLifecycle failuresLifecycleRetention = DataStreamLifecycle.failuresLifecycleBuilder()
|
||||
.dataRetention(dataStreamRetention)
|
||||
.build();
|
||||
TimeValue defaultRetention = TimeValue.timeValueDays(randomIntBetween(1, (int) dataStreamRetention.getDays() - 1));
|
||||
|
||||
Tuple<TimeValue, DataStreamLifecycle.RetentionSource> effectiveDataRetentionWithSource = lifecycleRetention
|
||||
// No global retention
|
||||
Tuple<TimeValue, DataStreamLifecycle.RetentionSource> effectiveDataRetentionWithSource = dataLifecycleRetention
|
||||
.getEffectiveDataRetentionWithSource(null, randomBoolean());
|
||||
assertThat(effectiveDataRetentionWithSource.v1(), equalTo(dataStreamRetention));
|
||||
assertThat(effectiveDataRetentionWithSource.v2(), equalTo(DATA_STREAM_CONFIGURATION));
|
||||
Tuple<TimeValue, DataStreamLifecycle.RetentionSource> effectiveFailuresRetentionWithSource = failuresLifecycleRetention
|
||||
.getEffectiveDataRetentionWithSource(null, randomBoolean());
|
||||
assertThat(effectiveFailuresRetentionWithSource.v1(), equalTo(dataStreamRetention));
|
||||
assertThat(effectiveFailuresRetentionWithSource.v2(), equalTo(DATA_STREAM_CONFIGURATION));
|
||||
|
||||
effectiveDataRetentionWithSource = lifecycleRetention.getEffectiveDataRetentionWithSource(
|
||||
// only default & data lifecycle
|
||||
effectiveDataRetentionWithSource = dataLifecycleRetention.getEffectiveDataRetentionWithSource(
|
||||
new DataStreamGlobalRetention(defaultRetention, null),
|
||||
false
|
||||
);
|
||||
assertThat(effectiveDataRetentionWithSource.v1(), equalTo(dataStreamRetention));
|
||||
assertThat(effectiveDataRetentionWithSource.v2(), equalTo(DATA_STREAM_CONFIGURATION));
|
||||
|
||||
// only failures default & failures lifecycle
|
||||
effectiveFailuresRetentionWithSource = failuresLifecycleRetention.getEffectiveDataRetentionWithSource(
|
||||
new DataStreamGlobalRetention(defaultRetention, null),
|
||||
false
|
||||
);
|
||||
assertThat(effectiveFailuresRetentionWithSource.v1(), equalTo(dataStreamRetention));
|
||||
assertThat(effectiveFailuresRetentionWithSource.v2(), equalTo(DATA_STREAM_CONFIGURATION));
|
||||
|
||||
TimeValue maxGlobalRetention = randomBoolean() ? dataStreamRetention : TimeValue.timeValueDays(dataStreamRetention.days() + 1);
|
||||
effectiveDataRetentionWithSource = lifecycleRetention.getEffectiveDataRetentionWithSource(
|
||||
effectiveDataRetentionWithSource = dataLifecycleRetention.getEffectiveDataRetentionWithSource(
|
||||
new DataStreamGlobalRetention(defaultRetention, maxGlobalRetention),
|
||||
false
|
||||
);
|
||||
|
@ -345,7 +422,7 @@ public class DataStreamLifecycleTests extends AbstractXContentSerializingTestCas
|
|||
assertThat(effectiveDataRetentionWithSource.v2(), equalTo(DATA_STREAM_CONFIGURATION));
|
||||
|
||||
TimeValue maxRetentionLessThanDataStream = TimeValue.timeValueDays(dataStreamRetention.days() - 1);
|
||||
effectiveDataRetentionWithSource = lifecycleRetention.getEffectiveDataRetentionWithSource(
|
||||
effectiveDataRetentionWithSource = dataLifecycleRetention.getEffectiveDataRetentionWithSource(
|
||||
new DataStreamGlobalRetention(
|
||||
randomBoolean()
|
||||
? null
|
||||
|
@ -356,6 +433,17 @@ public class DataStreamLifecycleTests extends AbstractXContentSerializingTestCas
|
|||
);
|
||||
assertThat(effectiveDataRetentionWithSource.v1(), equalTo(maxRetentionLessThanDataStream));
|
||||
assertThat(effectiveDataRetentionWithSource.v2(), equalTo(MAX_GLOBAL_RETENTION));
|
||||
effectiveFailuresRetentionWithSource = failuresLifecycleRetention.getEffectiveDataRetentionWithSource(
|
||||
new DataStreamGlobalRetention(
|
||||
randomBoolean()
|
||||
? null
|
||||
: TimeValue.timeValueDays(randomIntBetween(1, (int) (maxRetentionLessThanDataStream.days() - 1))),
|
||||
maxRetentionLessThanDataStream
|
||||
),
|
||||
false
|
||||
);
|
||||
assertThat(effectiveFailuresRetentionWithSource.v1(), equalTo(maxRetentionLessThanDataStream));
|
||||
assertThat(effectiveFailuresRetentionWithSource.v2(), equalTo(MAX_GLOBAL_RETENTION));
|
||||
}
|
||||
|
||||
// Global retention does not apply to internal data streams
|
||||
|
@ -367,10 +455,13 @@ public class DataStreamLifecycleTests extends AbstractXContentSerializingTestCas
|
|||
TimeValue.timeValueDays(7),
|
||||
TimeValue.timeValueDays(90)
|
||||
);
|
||||
DataStreamLifecycle lifecycle = DataStreamLifecycle.dataLifecycleBuilder().dataRetention(dataStreamRetention).build();
|
||||
DataStreamLifecycle dataLifecycle = DataStreamLifecycle.dataLifecycleBuilder().dataRetention(dataStreamRetention).build();
|
||||
DataStreamLifecycle failuresLifecycle = DataStreamLifecycle.failuresLifecycleBuilder()
|
||||
.dataRetention(dataStreamRetention)
|
||||
.build();
|
||||
|
||||
// Verify that global retention should have kicked in
|
||||
var effectiveDataRetentionWithSource = lifecycle.getEffectiveDataRetentionWithSource(globalRetention, false);
|
||||
var effectiveDataRetentionWithSource = dataLifecycle.getEffectiveDataRetentionWithSource(globalRetention, false);
|
||||
if (dataStreamWithRetention) {
|
||||
assertThat(effectiveDataRetentionWithSource.v1(), equalTo(globalRetention.maxRetention()));
|
||||
assertThat(effectiveDataRetentionWithSource.v2(), equalTo(MAX_GLOBAL_RETENTION));
|
||||
|
@ -379,15 +470,32 @@ public class DataStreamLifecycleTests extends AbstractXContentSerializingTestCas
|
|||
assertThat(effectiveDataRetentionWithSource.v2(), equalTo(DEFAULT_GLOBAL_RETENTION));
|
||||
}
|
||||
|
||||
// Verify that global retention should have kicked in
|
||||
var effectiveFailuresRetentionWithSource = failuresLifecycle.getEffectiveDataRetentionWithSource(globalRetention, false);
|
||||
if (dataStreamWithRetention) {
|
||||
assertThat(effectiveFailuresRetentionWithSource.v1(), equalTo(globalRetention.maxRetention()));
|
||||
assertThat(effectiveFailuresRetentionWithSource.v2(), equalTo(MAX_GLOBAL_RETENTION));
|
||||
} else {
|
||||
assertThat(effectiveFailuresRetentionWithSource.v1(), equalTo(globalRetention.defaultRetention()));
|
||||
assertThat(effectiveFailuresRetentionWithSource.v2(), equalTo(DEFAULT_GLOBAL_RETENTION));
|
||||
}
|
||||
|
||||
// Now verify that internal data streams do not use global retention
|
||||
// Verify that global retention should have kicked in
|
||||
effectiveDataRetentionWithSource = lifecycle.getEffectiveDataRetentionWithSource(globalRetention, true);
|
||||
effectiveDataRetentionWithSource = dataLifecycle.getEffectiveDataRetentionWithSource(globalRetention, true);
|
||||
if (dataStreamWithRetention) {
|
||||
assertThat(effectiveDataRetentionWithSource.v1(), equalTo(dataStreamRetention));
|
||||
} else {
|
||||
assertThat(effectiveDataRetentionWithSource.v1(), nullValue());
|
||||
}
|
||||
assertThat(effectiveDataRetentionWithSource.v2(), equalTo(DATA_STREAM_CONFIGURATION));
|
||||
effectiveFailuresRetentionWithSource = failuresLifecycle.getEffectiveDataRetentionWithSource(globalRetention, true);
|
||||
if (dataStreamWithRetention) {
|
||||
assertThat(effectiveFailuresRetentionWithSource.v1(), equalTo(dataStreamRetention));
|
||||
} else {
|
||||
assertThat(effectiveFailuresRetentionWithSource.v1(), nullValue());
|
||||
}
|
||||
assertThat(effectiveFailuresRetentionWithSource.v2(), equalTo(DATA_STREAM_CONFIGURATION));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -408,6 +516,17 @@ public class DataStreamLifecycleTests extends AbstractXContentSerializingTestCas
|
|||
.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Failures lifecycle does not support downsampling, this is why we have a dedicated method
|
||||
* @return a random lifecycle without downsampling
|
||||
*/
|
||||
public static DataStreamLifecycle randomFailuresLifecycle() {
|
||||
return DataStreamLifecycle.failuresLifecycleBuilder()
|
||||
.dataRetention(randomBoolean() ? null : randomTimeValue(1, 365, TimeUnit.DAYS))
|
||||
.enabled(randomBoolean())
|
||||
.build();
|
||||
}
|
||||
|
||||
static List<DataStreamLifecycle.DownsamplingRound> randomDownsampling() {
|
||||
var count = randomIntBetween(0, 9);
|
||||
List<DataStreamLifecycle.DownsamplingRound> rounds = new ArrayList<>();
|
||||
|
@ -431,4 +550,23 @@ public class DataStreamLifecycleTests extends AbstractXContentSerializingTestCas
|
|||
);
|
||||
return new DataStreamLifecycle.DownsamplingRound(after, fixedInterval);
|
||||
}
|
||||
|
||||
public void testInvalidLifecycleConfiguration() {
|
||||
IllegalArgumentException exception = expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> new DataStreamFailureStore(
|
||||
randomBoolean(),
|
||||
new DataStreamLifecycle(
|
||||
DataStreamLifecycle.LifecycleType.FAILURES,
|
||||
null,
|
||||
null,
|
||||
DataStreamLifecycleTests.randomDownsampling()
|
||||
)
|
||||
)
|
||||
);
|
||||
assertThat(
|
||||
exception.getMessage(),
|
||||
containsString("Failure store lifecycle does not support downsampling, please remove the downsampling configuration.")
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -9,6 +9,7 @@
|
|||
|
||||
package org.elasticsearch.cluster.metadata;
|
||||
|
||||
import org.elasticsearch.TransportVersions;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.test.AbstractXContentSerializingTestCase;
|
||||
import org.elasticsearch.xcontent.XContentParser;
|
||||
|
@ -32,11 +33,12 @@ public class DataStreamOptionsTemplateTests extends AbstractXContentSerializingT
|
|||
}
|
||||
|
||||
public static DataStreamOptions.Template randomDataStreamOptions() {
|
||||
return switch (randomIntBetween(0, 3)) {
|
||||
return switch (randomIntBetween(0, 2)) {
|
||||
case 0 -> DataStreamOptions.Template.EMPTY;
|
||||
case 1 -> createTemplateWithFailureStoreConfig(true);
|
||||
case 2 -> createTemplateWithFailureStoreConfig(false);
|
||||
case 3 -> RESET;
|
||||
case 1 -> RESET;
|
||||
case 2 -> new DataStreamOptions.Template(
|
||||
ResettableValue.create(DataStreamFailureStoreTemplateTests.randomFailureStoreTemplate())
|
||||
);
|
||||
default -> throw new IllegalArgumentException("Illegal randomisation branch");
|
||||
};
|
||||
}
|
||||
|
@ -45,22 +47,24 @@ public class DataStreamOptionsTemplateTests extends AbstractXContentSerializingT
|
|||
protected DataStreamOptions.Template mutateInstance(DataStreamOptions.Template instance) {
|
||||
ResettableValue<DataStreamFailureStore.Template> failureStore = instance.failureStore();
|
||||
if (failureStore.isDefined() == false) {
|
||||
if (randomBoolean()) {
|
||||
return createTemplateWithFailureStoreConfig(randomBoolean());
|
||||
} else {
|
||||
return new DataStreamOptions.Template(ResettableValue.reset());
|
||||
}
|
||||
failureStore = randomBoolean()
|
||||
? ResettableValue.create(DataStreamFailureStoreTemplateTests.randomFailureStoreTemplate())
|
||||
: ResettableValue.reset();
|
||||
} else if (failureStore.shouldReset()) {
|
||||
failureStore = ResettableValue.create(
|
||||
randomBoolean() ? DataStreamFailureStoreTemplateTests.randomFailureStoreTemplate() : null
|
||||
);
|
||||
} else {
|
||||
failureStore = switch (randomIntBetween(0, 2)) {
|
||||
case 0 -> ResettableValue.undefined();
|
||||
case 1 -> ResettableValue.reset();
|
||||
case 2 -> ResettableValue.create(
|
||||
randomValueOtherThan(failureStore.get(), DataStreamFailureStoreTemplateTests::randomFailureStoreTemplate)
|
||||
);
|
||||
default -> throw new IllegalArgumentException("Illegal randomisation branch");
|
||||
};
|
||||
}
|
||||
if (failureStore.shouldReset()) {
|
||||
if (randomBoolean()) {
|
||||
return createTemplateWithFailureStoreConfig(randomBoolean());
|
||||
} else {
|
||||
return DataStreamOptions.Template.EMPTY;
|
||||
}
|
||||
}
|
||||
return new DataStreamOptions.Template(
|
||||
instance.failureStore().map(x -> new DataStreamFailureStore.Template(x.enabled().map(e -> e == false)))
|
||||
);
|
||||
return new DataStreamOptions.Template(failureStore);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -68,15 +72,15 @@ public class DataStreamOptionsTemplateTests extends AbstractXContentSerializingT
|
|||
return DataStreamOptions.Template.fromXContent(parser);
|
||||
}
|
||||
|
||||
private static DataStreamOptions.Template createTemplateWithFailureStoreConfig(boolean enabled) {
|
||||
return new DataStreamOptions.Template(ResettableValue.create(new DataStreamFailureStore.Template(ResettableValue.create(enabled))));
|
||||
}
|
||||
|
||||
public void testTemplateComposition() {
|
||||
DataStreamOptions.Template fullyConfigured = new DataStreamOptions.Template(new DataStreamFailureStore.Template(randomBoolean()));
|
||||
DataStreamOptions.Template negated = new DataStreamOptions.Template(
|
||||
new DataStreamFailureStore.Template(fullyConfigured.failureStore().get().enabled().get() == false)
|
||||
// we fully define the options to avoid having to check for normalised values in the assertion
|
||||
DataStreamOptions.Template fullyConfigured = new DataStreamOptions.Template(
|
||||
new DataStreamFailureStore.Template(
|
||||
randomBoolean(),
|
||||
DataStreamLifecycle.createFailuresLifecycleTemplate(randomBoolean(), randomTimeValue())
|
||||
)
|
||||
);
|
||||
|
||||
// No updates
|
||||
DataStreamOptions.Template result = DataStreamOptions.builder(DataStreamOptions.Template.EMPTY).buildTemplate();
|
||||
assertThat(result, equalTo(DataStreamOptions.Template.EMPTY));
|
||||
|
@ -92,11 +96,61 @@ public class DataStreamOptionsTemplateTests extends AbstractXContentSerializingT
|
|||
assertThat(result, equalTo(fullyConfigured));
|
||||
|
||||
// Override
|
||||
DataStreamOptions.Template negated = new DataStreamOptions.Template(
|
||||
fullyConfigured.failureStore()
|
||||
.map(
|
||||
failureStore -> DataStreamFailureStore.builder(failureStore)
|
||||
.enabled(failureStore.enabled().map(enabled -> enabled == false))
|
||||
.buildTemplate()
|
||||
)
|
||||
);
|
||||
result = DataStreamOptions.builder(fullyConfigured).composeTemplate(negated).buildTemplate();
|
||||
assertThat(result, equalTo(negated));
|
||||
|
||||
// Test merging
|
||||
DataStreamOptions.Template dataStreamOptionsWithoutLifecycle = new DataStreamOptions.Template(
|
||||
new DataStreamFailureStore.Template(true, null)
|
||||
);
|
||||
DataStreamOptions.Template dataStreamOptionsWithLifecycle = new DataStreamOptions.Template(
|
||||
new DataStreamFailureStore.Template(null, DataStreamLifecycle.createFailuresLifecycleTemplate(true, randomPositiveTimeValue()))
|
||||
);
|
||||
result = DataStreamOptions.builder(dataStreamOptionsWithLifecycle)
|
||||
.composeTemplate(dataStreamOptionsWithoutLifecycle)
|
||||
.buildTemplate();
|
||||
assertThat(result.failureStore().get().enabled(), equalTo(dataStreamOptionsWithoutLifecycle.failureStore().get().enabled()));
|
||||
assertThat(result.failureStore().get().lifecycle(), equalTo(dataStreamOptionsWithLifecycle.failureStore().get().lifecycle()));
|
||||
|
||||
// Reset
|
||||
result = DataStreamOptions.builder(fullyConfigured).composeTemplate(RESET).buildTemplate();
|
||||
assertThat(result, equalTo(DataStreamOptions.Template.EMPTY));
|
||||
}
|
||||
|
||||
public void testBackwardCompatibility() throws IOException {
|
||||
DataStreamOptions.Template result = copyInstance(DataStreamOptions.Template.EMPTY, TransportVersions.SETTINGS_IN_DATA_STREAMS);
|
||||
assertThat(result, equalTo(DataStreamOptions.Template.EMPTY));
|
||||
|
||||
DataStreamOptions.Template withEnabled = new DataStreamOptions.Template(
|
||||
new DataStreamFailureStore.Template(randomBoolean(), DataStreamLifecycleTemplateTests.randomFailuresLifecycleTemplate())
|
||||
);
|
||||
result = copyInstance(withEnabled, TransportVersions.SETTINGS_IN_DATA_STREAMS);
|
||||
assertThat(result.failureStore().get().enabled(), equalTo(withEnabled.failureStore().get().enabled()));
|
||||
assertThat(result.failureStore().get().lifecycle(), equalTo(ResettableValue.undefined()));
|
||||
|
||||
DataStreamOptions.Template withoutEnabled = new DataStreamOptions.Template(
|
||||
new DataStreamFailureStore.Template(
|
||||
ResettableValue.undefined(),
|
||||
randomBoolean()
|
||||
? ResettableValue.reset()
|
||||
: ResettableValue.create(DataStreamLifecycleTemplateTests.randomFailuresLifecycleTemplate())
|
||||
)
|
||||
);
|
||||
result = copyInstance(withoutEnabled, TransportVersions.SETTINGS_IN_DATA_STREAMS);
|
||||
assertThat(result, equalTo(DataStreamOptions.Template.EMPTY));
|
||||
|
||||
DataStreamOptions.Template withEnabledReset = new DataStreamOptions.Template(
|
||||
new DataStreamFailureStore.Template(ResettableValue.reset(), ResettableValue.undefined())
|
||||
);
|
||||
result = copyInstance(withEnabledReset, TransportVersions.SETTINGS_IN_DATA_STREAMS);
|
||||
assertThat(result, equalTo(new DataStreamOptions.Template(ResettableValue.reset())));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -9,12 +9,16 @@
|
|||
|
||||
package org.elasticsearch.cluster.metadata;
|
||||
|
||||
import org.elasticsearch.TransportVersions;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.test.AbstractXContentSerializingTestCase;
|
||||
import org.elasticsearch.xcontent.XContentParser;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
||||
public class DataStreamOptionsTests extends AbstractXContentSerializingTestCase<DataStreamOptions> {
|
||||
|
||||
@Override
|
||||
|
@ -51,4 +55,22 @@ public class DataStreamOptionsTests extends AbstractXContentSerializingTestCase<
|
|||
protected DataStreamOptions doParseInstance(XContentParser parser) throws IOException {
|
||||
return DataStreamOptions.fromXContent(parser);
|
||||
}
|
||||
|
||||
public void testBackwardCompatibility() throws IOException {
|
||||
DataStreamOptions result = copyInstance(DataStreamOptions.EMPTY, TransportVersions.SETTINGS_IN_DATA_STREAMS);
|
||||
assertThat(result, equalTo(DataStreamOptions.EMPTY));
|
||||
|
||||
DataStreamOptions withEnabled = new DataStreamOptions(
|
||||
new DataStreamFailureStore(randomBoolean(), DataStreamLifecycleTests.randomFailuresLifecycle())
|
||||
);
|
||||
result = copyInstance(withEnabled, TransportVersions.SETTINGS_IN_DATA_STREAMS);
|
||||
assertThat(result.failureStore().enabled(), equalTo(withEnabled.failureStore().enabled()));
|
||||
assertThat(result.failureStore().lifecycle(), nullValue());
|
||||
|
||||
DataStreamOptions withoutEnabled = new DataStreamOptions(
|
||||
new DataStreamFailureStore(null, DataStreamLifecycleTests.randomFailuresLifecycle())
|
||||
);
|
||||
result = copyInstance(withoutEnabled, TransportVersions.SETTINGS_IN_DATA_STREAMS);
|
||||
assertThat(result, equalTo(DataStreamOptions.EMPTY));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -49,15 +49,14 @@ import java.util.List;
|
|||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.elasticsearch.cluster.metadata.ComponentTemplateTests.randomMappings;
|
||||
import static org.elasticsearch.cluster.metadata.DataStream.getDefaultBackingIndexName;
|
||||
import static org.elasticsearch.cluster.metadata.DataStream.getDefaultFailureStoreName;
|
||||
import static org.elasticsearch.cluster.metadata.DataStreamTestHelper.newInstance;
|
||||
import static org.elasticsearch.cluster.metadata.DataStreamTestHelper.randomGlobalRetention;
|
||||
import static org.elasticsearch.cluster.metadata.DataStreamTestHelper.randomIndexInstances;
|
||||
import static org.elasticsearch.cluster.metadata.DataStreamTestHelper.randomNonEmptyIndexInstances;
|
||||
import static org.elasticsearch.cluster.metadata.DataStreamTestHelper.randomSettings;
|
||||
|
@ -72,7 +71,6 @@ import static org.hamcrest.Matchers.is;
|
|||
import static org.hamcrest.Matchers.not;
|
||||
import static org.hamcrest.Matchers.notNullValue;
|
||||
import static org.hamcrest.Matchers.nullValue;
|
||||
import static org.junit.Assert.assertThrows;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
|
@ -147,9 +145,12 @@ public class DataStreamTests extends AbstractXContentSerializingTestCase<DataStr
|
|||
? null
|
||||
: DataStreamLifecycle.dataLifecycleBuilder().dataRetention(randomPositiveTimeValue()).build();
|
||||
case 10 -> failureIndices = randomValueOtherThan(failureIndices, DataStreamTestHelper::randomIndexInstances);
|
||||
case 11 -> dataStreamOptions = dataStreamOptions.isEmpty() ? new DataStreamOptions(new DataStreamFailureStore(randomBoolean()))
|
||||
case 11 -> dataStreamOptions = dataStreamOptions.isEmpty()
|
||||
? new DataStreamOptions(DataStreamFailureStoreTests.randomFailureStore())
|
||||
: randomBoolean() ? DataStreamOptions.EMPTY
|
||||
: new DataStreamOptions(new DataStreamFailureStore(dataStreamOptions.failureStore().enabled() == false));
|
||||
: new DataStreamOptions(
|
||||
randomValueOtherThan(dataStreamOptions.failureStore(), DataStreamFailureStoreTests::randomFailureStore)
|
||||
);
|
||||
case 12 -> {
|
||||
rolloverOnWrite = rolloverOnWrite == false;
|
||||
isReplicated = rolloverOnWrite == false && isReplicated;
|
||||
|
@ -1390,156 +1391,14 @@ public class DataStreamTests extends AbstractXContentSerializingTestCase<DataStr
|
|||
}
|
||||
|
||||
public void testGetBackingIndicesPastRetention() {
|
||||
String dataStreamName = "metrics-foo";
|
||||
long now = System.currentTimeMillis();
|
||||
|
||||
List<DataStreamMetadata> creationAndRolloverTimes = List.of(
|
||||
DataStreamMetadata.dataStreamMetadata(now - 5000_000, now - 4000_000),
|
||||
DataStreamMetadata.dataStreamMetadata(now - 4000_000, now - 3000_000),
|
||||
DataStreamMetadata.dataStreamMetadata(now - 3000_000, now - 2000_000),
|
||||
DataStreamMetadata.dataStreamMetadata(now - 2000_000, now - 1000_000),
|
||||
DataStreamMetadata.dataStreamMetadata(now, null)
|
||||
);
|
||||
|
||||
{
|
||||
{
|
||||
// no lifecycle configured so we expect an empty list
|
||||
Metadata.Builder builder = Metadata.builder();
|
||||
DataStream dataStream = createDataStream(
|
||||
builder,
|
||||
dataStreamName,
|
||||
creationAndRolloverTimes,
|
||||
settings(IndexVersion.current()),
|
||||
null
|
||||
);
|
||||
Metadata metadata = builder.build();
|
||||
|
||||
assertThat(
|
||||
dataStream.getBackingIndicesPastRetention(metadata.getProject()::index, () -> now, randomGlobalRetention()).isEmpty(),
|
||||
is(true)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
Metadata.Builder builder = Metadata.builder();
|
||||
AtomicReference<TimeValue> retention = new AtomicReference<>();
|
||||
DataStream dataStream = createDataStream(
|
||||
builder,
|
||||
dataStreamName,
|
||||
creationAndRolloverTimes,
|
||||
settings(IndexVersion.current()),
|
||||
new DataStreamLifecycle(null, null, null) {
|
||||
public TimeValue dataRetention() {
|
||||
return retention.get();
|
||||
}
|
||||
}
|
||||
);
|
||||
Metadata metadata = builder.build();
|
||||
|
||||
{
|
||||
// Mix of indices younger and older than retention, data stream retention is effective retention
|
||||
retention.set(TimeValue.timeValueSeconds(2500));
|
||||
List<Index> backingIndices = dataStream.getBackingIndicesPastRetention(
|
||||
metadata.getProject()::index,
|
||||
() -> now,
|
||||
randomBoolean() ? randomGlobalRetention() : null
|
||||
);
|
||||
assertThat(backingIndices.size(), is(2));
|
||||
for (int i = 0; i < backingIndices.size(); i++) {
|
||||
assertThat(backingIndices.get(i).getName(), is(dataStream.getIndices().get(i).getName()));
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
// All indices past retention, but we keep the write index
|
||||
retention.set(TimeValue.timeValueSeconds(0));
|
||||
List<Index> backingIndices = dataStream.getBackingIndicesPastRetention(metadata.getProject()::index, () -> now, null);
|
||||
assertThat(backingIndices.size(), is(4));
|
||||
for (int i = 0; i < backingIndices.size(); i++) {
|
||||
assertThat(backingIndices.get(i).getName(), is(dataStream.getIndices().get(i).getName()));
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
// All indices younger than retention
|
||||
retention.set(TimeValue.timeValueSeconds(6000));
|
||||
List<Index> backingIndices = dataStream.getBackingIndicesPastRetention(metadata.getProject()::index, () -> now, null);
|
||||
assertThat(backingIndices.isEmpty(), is(true));
|
||||
}
|
||||
|
||||
{
|
||||
// Test predicate that influences which indices are candidates for a retention check
|
||||
Function<String, IndexMetadata> indexMetadataWithSomeLifecycleSupplier = indexName -> {
|
||||
IndexMetadata indexMetadata = metadata.getProject().index(indexName);
|
||||
if (indexName.endsWith("00003") || indexName.endsWith("00005")) {
|
||||
return indexMetadata;
|
||||
}
|
||||
return IndexMetadata.builder(indexMetadata)
|
||||
.settings(Settings.builder().put(indexMetadata.getSettings()).put(IndexMetadata.LIFECYCLE_NAME, "some-policy").build())
|
||||
.build();
|
||||
};
|
||||
retention.set(TimeValue.timeValueSeconds(0));
|
||||
List<Index> backingIndices = dataStream.getBackingIndicesPastRetention(indexMetadataWithSomeLifecycleSupplier, () -> now, null);
|
||||
assertThat(backingIndices.size(), is(1));
|
||||
assertThat(backingIndices.get(0).getName(), is(dataStream.getIndices().get(2).getName()));
|
||||
}
|
||||
|
||||
{
|
||||
// no retention configured but we have default retention
|
||||
DataStreamGlobalRetention globalRetention = new DataStreamGlobalRetention(
|
||||
TimeValue.timeValueSeconds(2500),
|
||||
randomBoolean() ? TimeValue.timeValueSeconds(randomIntBetween(2500, 5000)) : null
|
||||
);
|
||||
retention.set(null);
|
||||
|
||||
List<Index> backingIndices = dataStream.getBackingIndicesPastRetention(
|
||||
metadata.getProject()::index,
|
||||
() -> now,
|
||||
globalRetention
|
||||
);
|
||||
assertThat(backingIndices.size(), is(2));
|
||||
for (int i = 0; i < backingIndices.size(); i++) {
|
||||
assertThat(backingIndices.get(i).getName(), is(dataStream.getIndices().get(i).getName()));
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
// no retention or too large retention configured and we have max retention
|
||||
DataStreamGlobalRetention globalRetention = new DataStreamGlobalRetention(null, TimeValue.timeValueSeconds(2500));
|
||||
retention.set(randomBoolean() ? TimeValue.timeValueDays(6000) : null);
|
||||
List<Index> backingIndices = dataStream.getBackingIndicesPastRetention(
|
||||
metadata.getProject()::index,
|
||||
() -> now,
|
||||
globalRetention
|
||||
);
|
||||
assertThat(backingIndices.size(), is(2));
|
||||
for (int i = 0; i < backingIndices.size(); i++) {
|
||||
assertThat(backingIndices.get(i).getName(), is(dataStream.getIndices().get(i).getName()));
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
// no indices are returned as even though all pass retention age none are managed by data stream lifecycle
|
||||
Metadata.Builder builderWithIlm = Metadata.builder();
|
||||
DataStream dataStreamWithIlm = createDataStream(
|
||||
builderWithIlm,
|
||||
dataStreamName,
|
||||
creationAndRolloverTimes,
|
||||
settings(IndexVersion.current()).put(IndexMetadata.LIFECYCLE_NAME, "ILM_policy"),
|
||||
DataStreamLifecycle.dataLifecycleBuilder().dataRetention(TimeValue.ZERO).build()
|
||||
);
|
||||
Metadata metadataWithIlm = builderWithIlm.build();
|
||||
|
||||
List<Index> backingIndices = dataStreamWithIlm.getBackingIndicesPastRetention(
|
||||
metadataWithIlm.getProject()::index,
|
||||
() -> now,
|
||||
randomGlobalRetention()
|
||||
);
|
||||
assertThat(backingIndices.isEmpty(), is(true));
|
||||
}
|
||||
testIndicesPastRetention(false);
|
||||
}
|
||||
|
||||
public void testGetFailureIndicesPastRetention() {
|
||||
testIndicesPastRetention(true);
|
||||
}
|
||||
|
||||
private void testIndicesPastRetention(boolean failureStore) {
|
||||
String dataStreamName = "metrics-foo";
|
||||
long now = System.currentTimeMillis();
|
||||
|
||||
|
@ -1553,72 +1412,78 @@ public class DataStreamTests extends AbstractXContentSerializingTestCase<DataStr
|
|||
|
||||
{
|
||||
{
|
||||
// no lifecycle configured so we expect an empty list
|
||||
// No lifecycle or disabled for data and only disabled lifecycle for failures should result in empty list.
|
||||
Metadata.Builder builder = Metadata.builder();
|
||||
var disabledLifecycle = DataStreamLifecycle.failuresLifecycleBuilder().enabled(false).build();
|
||||
DataStream dataStream = createDataStream(
|
||||
builder,
|
||||
dataStreamName,
|
||||
creationAndRolloverTimes,
|
||||
settings(IndexVersion.current()),
|
||||
null
|
||||
randomBoolean() ? DataStreamLifecycle.dataLifecycleBuilder().enabled(false).build() : null,
|
||||
new DataStreamOptions(new DataStreamFailureStore(randomBoolean(), disabledLifecycle))
|
||||
);
|
||||
Metadata metadata = builder.build();
|
||||
|
||||
assertThat(
|
||||
dataStream.getFailureIndicesPastRetention(metadata.getProject()::index, () -> now, randomGlobalRetention()).isEmpty(),
|
||||
dataStream.getIndicesPastRetention(metadata.getProject()::index, () -> now, TimeValue.ZERO, failureStore).isEmpty(),
|
||||
is(true)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
Metadata.Builder builder = Metadata.builder();
|
||||
AtomicReference<TimeValue> retention = new AtomicReference<>();
|
||||
DataStream dataStream = createDataStream(
|
||||
builder,
|
||||
dataStreamName,
|
||||
creationAndRolloverTimes,
|
||||
settings(IndexVersion.current()),
|
||||
new DataStreamLifecycle(null, null, null) {
|
||||
public TimeValue dataRetention() {
|
||||
return retention.get();
|
||||
}
|
||||
}
|
||||
DataStreamLifecycle.DEFAULT_DATA_LIFECYCLE
|
||||
);
|
||||
Metadata metadata = builder.build();
|
||||
Supplier<List<Index>> indicesSupplier = () -> failureStore ? dataStream.getFailureIndices() : dataStream.getIndices();
|
||||
|
||||
{
|
||||
// Mix of indices younger and older than retention, data stream retention is effective retention
|
||||
retention.set(TimeValue.timeValueSeconds(2500));
|
||||
List<Index> failureIndices = dataStream.getFailureIndicesPastRetention(
|
||||
// Mix of indices younger and older than retention
|
||||
List<Index> indicesPastRetention = dataStream.getIndicesPastRetention(
|
||||
metadata.getProject()::index,
|
||||
() -> now,
|
||||
randomBoolean() ? randomGlobalRetention() : null
|
||||
TimeValue.timeValueSeconds(2500),
|
||||
failureStore
|
||||
);
|
||||
assertThat(failureIndices.size(), is(2));
|
||||
for (int i = 0; i < failureIndices.size(); i++) {
|
||||
assertThat(failureIndices.get(i).getName(), is(dataStream.getFailureIndices().get(i).getName()));
|
||||
assertThat(indicesPastRetention.size(), is(2));
|
||||
for (int i = 0; i < indicesPastRetention.size(); i++) {
|
||||
assertThat(indicesPastRetention.get(i).getName(), is(indicesSupplier.get().get(i).getName()));
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
// All indices past retention, but we keep the write index
|
||||
retention.set(TimeValue.timeValueSeconds(0));
|
||||
List<Index> failureIndices = dataStream.getFailureIndicesPastRetention(metadata.getProject()::index, () -> now, null);
|
||||
assertThat(failureIndices.size(), is(4));
|
||||
for (int i = 0; i < failureIndices.size(); i++) {
|
||||
assertThat(failureIndices.get(i).getName(), is(dataStream.getFailureIndices().get(i).getName()));
|
||||
List<Index> indicesPastRetention = dataStream.getIndicesPastRetention(
|
||||
metadata.getProject()::index,
|
||||
() -> now,
|
||||
TimeValue.ZERO,
|
||||
failureStore
|
||||
);
|
||||
assertThat(indicesPastRetention.size(), is(4));
|
||||
for (int i = 0; i < indicesPastRetention.size(); i++) {
|
||||
assertThat(indicesPastRetention.get(i).getName(), is(indicesSupplier.get().get(i).getName()));
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
// All indices younger than retention
|
||||
retention.set(TimeValue.timeValueSeconds(6000));
|
||||
List<Index> failureIndices = dataStream.getFailureIndicesPastRetention(metadata.getProject()::index, () -> now, null);
|
||||
assertThat(failureIndices.isEmpty(), is(true));
|
||||
List<Index> indicesPastRetention = dataStream.getIndicesPastRetention(
|
||||
metadata.getProject()::index,
|
||||
() -> now,
|
||||
TimeValue.timeValueSeconds(6000),
|
||||
failureStore
|
||||
);
|
||||
assertThat(indicesPastRetention.isEmpty(), is(true));
|
||||
}
|
||||
|
||||
{
|
||||
// Test predicate that influences which indices are candidates for a retention check
|
||||
// Test ILM managed indices are excluded
|
||||
Function<String, IndexMetadata> indexMetadataWithSomeLifecycleSupplier = indexName -> {
|
||||
IndexMetadata indexMetadata = metadata.getProject().index(indexName);
|
||||
if (indexName.endsWith("00003") || indexName.endsWith("00005")) {
|
||||
|
@ -1628,99 +1493,26 @@ public class DataStreamTests extends AbstractXContentSerializingTestCase<DataStr
|
|||
.settings(Settings.builder().put(indexMetadata.getSettings()).put(IndexMetadata.LIFECYCLE_NAME, "some-policy").build())
|
||||
.build();
|
||||
};
|
||||
retention.set(TimeValue.timeValueSeconds(0));
|
||||
List<Index> failureIndices = dataStream.getFailureIndicesPastRetention(indexMetadataWithSomeLifecycleSupplier, () -> now, null);
|
||||
assertThat(failureIndices.size(), is(1));
|
||||
assertThat(failureIndices.get(0).getName(), is(dataStream.getFailureIndices().get(2).getName()));
|
||||
}
|
||||
|
||||
{
|
||||
// no retention configured but we have default retention
|
||||
DataStreamGlobalRetention globalRetention = new DataStreamGlobalRetention(
|
||||
TimeValue.timeValueSeconds(2500),
|
||||
randomBoolean() ? TimeValue.timeValueSeconds(randomIntBetween(2500, 5000)) : null
|
||||
);
|
||||
retention.set(null);
|
||||
|
||||
List<Index> failureIndices = dataStream.getFailureIndicesPastRetention(
|
||||
metadata.getProject()::index,
|
||||
List<Index> indicesPastRetention = dataStream.getIndicesPastRetention(
|
||||
indexMetadataWithSomeLifecycleSupplier,
|
||||
() -> now,
|
||||
globalRetention
|
||||
TimeValue.ZERO,
|
||||
failureStore
|
||||
);
|
||||
assertThat(failureIndices.size(), is(2));
|
||||
for (int i = 0; i < failureIndices.size(); i++) {
|
||||
assertThat(failureIndices.get(i).getName(), is(dataStream.getFailureIndices().get(i).getName()));
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
// no retention or too large retention configured and we have max retention
|
||||
DataStreamGlobalRetention globalRetention = new DataStreamGlobalRetention(null, TimeValue.timeValueSeconds(2500));
|
||||
retention.set(randomBoolean() ? TimeValue.timeValueDays(6000) : null);
|
||||
List<Index> failureIndices = dataStream.getFailureIndicesPastRetention(
|
||||
metadata.getProject()::index,
|
||||
() -> now,
|
||||
globalRetention
|
||||
);
|
||||
assertThat(failureIndices.size(), is(2));
|
||||
for (int i = 0; i < failureIndices.size(); i++) {
|
||||
assertThat(failureIndices.get(i).getName(), is(dataStream.getFailureIndices().get(i).getName()));
|
||||
}
|
||||
assertThat(indicesPastRetention.size(), is(1));
|
||||
assertThat(indicesPastRetention.get(0).getName(), is(indicesSupplier.get().get(2).getName()));
|
||||
}
|
||||
}
|
||||
|
||||
public void testBackingIndicesPastRetentionWithOriginationDate() {
|
||||
// First, build an ordinary data stream:
|
||||
String dataStreamName = "metrics-foo";
|
||||
long now = System.currentTimeMillis();
|
||||
List<DataStreamMetadata> creationAndRolloverTimes = List.of(
|
||||
DataStreamMetadata.dataStreamMetadata(now - 5000, now - 4000),
|
||||
DataStreamMetadata.dataStreamMetadata(now - 4000, now - 3000),
|
||||
DataStreamMetadata.dataStreamMetadata(now - 3000, now - 2000),
|
||||
DataStreamMetadata.dataStreamMetadata(now - 2000, now - 1000),
|
||||
DataStreamMetadata.dataStreamMetadata(now, null, now - 8000), // origination date older than retention
|
||||
DataStreamMetadata.dataStreamMetadata(now, null, now - 1000), // origination date within retention
|
||||
DataStreamMetadata.dataStreamMetadata(now, null)
|
||||
);
|
||||
Metadata.Builder metadataBuilder = Metadata.builder();
|
||||
AtomicReference<TimeValue> testRetentionReference = new AtomicReference<>(null);
|
||||
DataStream dataStream = createDataStream(
|
||||
metadataBuilder,
|
||||
dataStreamName,
|
||||
creationAndRolloverTimes,
|
||||
settings(IndexVersion.current()),
|
||||
new DataStreamLifecycle(null, null, null) {
|
||||
public TimeValue dataRetention() {
|
||||
return testRetentionReference.get();
|
||||
}
|
||||
}
|
||||
);
|
||||
Metadata metadata = metadataBuilder.build();
|
||||
{
|
||||
// no retention configured so we expect an empty list
|
||||
testRetentionReference.set(null);
|
||||
assertThat(dataStream.getBackingIndicesPastRetention(metadata.getProject()::index, () -> now, null).isEmpty(), is(true));
|
||||
}
|
||||
|
||||
{
|
||||
// retention period where first and second index is too old, and 5th has old origination date.
|
||||
testRetentionReference.set(TimeValue.timeValueMillis(2500));
|
||||
List<Index> backingIndices = dataStream.getBackingIndicesPastRetention(metadata.getProject()::index, () -> now, null);
|
||||
assertThat(backingIndices.size(), is(3));
|
||||
assertThat(backingIndices.get(0).getName(), is(dataStream.getIndices().get(0).getName()));
|
||||
assertThat(backingIndices.get(1).getName(), is(dataStream.getIndices().get(1).getName()));
|
||||
assertThat(backingIndices.get(2).getName(), is(dataStream.getIndices().get(5).getName()));
|
||||
}
|
||||
|
||||
{
|
||||
// no index matches the retention age
|
||||
testRetentionReference.set(TimeValue.timeValueMillis(9000));
|
||||
List<Index> backingIndices = dataStream.getBackingIndicesPastRetention(metadata.getProject()::index, () -> now, null);
|
||||
assertThat(backingIndices.isEmpty(), is(true));
|
||||
}
|
||||
testIndicesPastRetentionWithOriginationDate(false);
|
||||
}
|
||||
|
||||
public void testFailureIndicesPastRetentionWithOriginationDate() {
|
||||
testIndicesPastRetentionWithOriginationDate(true);
|
||||
}
|
||||
|
||||
private void testIndicesPastRetentionWithOriginationDate(boolean failureStore) {
|
||||
// First, build an ordinary data stream:
|
||||
String dataStreamName = "metrics-foo";
|
||||
long now = System.currentTimeMillis();
|
||||
|
@ -1734,40 +1526,43 @@ public class DataStreamTests extends AbstractXContentSerializingTestCase<DataStr
|
|||
DataStreamMetadata.dataStreamMetadata(now, null)
|
||||
);
|
||||
Metadata.Builder metadataBuilder = Metadata.builder();
|
||||
AtomicReference<TimeValue> testRetentionReference = new AtomicReference<>(null);
|
||||
DataStream dataStream = createDataStream(
|
||||
metadataBuilder,
|
||||
dataStreamName,
|
||||
creationAndRolloverTimes,
|
||||
settings(IndexVersion.current()),
|
||||
new DataStreamLifecycle(null, null, null) {
|
||||
public TimeValue dataRetention() {
|
||||
return testRetentionReference.get();
|
||||
}
|
||||
}
|
||||
DataStreamLifecycle.DEFAULT_DATA_LIFECYCLE
|
||||
);
|
||||
Metadata metadata = metadataBuilder.build();
|
||||
Supplier<List<Index>> indicesSupplier = () -> failureStore ? dataStream.getFailureIndices() : dataStream.getIndices();
|
||||
{
|
||||
// no retention configured so we expect an empty list
|
||||
testRetentionReference.set(null);
|
||||
assertThat(dataStream.getFailureIndicesPastRetention(metadata.getProject()::index, () -> now, null).isEmpty(), is(true));
|
||||
assertThat(dataStream.getIndicesPastRetention(metadata.getProject()::index, () -> now, null, failureStore).isEmpty(), is(true));
|
||||
}
|
||||
|
||||
{
|
||||
// retention period where first and second index is too old, and 5th has old origination date.
|
||||
testRetentionReference.set(TimeValue.timeValueMillis(2500));
|
||||
List<Index> failureIndices = dataStream.getFailureIndicesPastRetention(metadata.getProject()::index, () -> now, null);
|
||||
assertThat(failureIndices.size(), is(3));
|
||||
assertThat(failureIndices.get(0).getName(), is(dataStream.getFailureIndices().get(0).getName()));
|
||||
assertThat(failureIndices.get(1).getName(), is(dataStream.getFailureIndices().get(1).getName()));
|
||||
assertThat(failureIndices.get(2).getName(), is(dataStream.getFailureIndices().get(5).getName()));
|
||||
List<Index> indicesPastRetention = dataStream.getIndicesPastRetention(
|
||||
metadata.getProject()::index,
|
||||
() -> now,
|
||||
TimeValue.timeValueMillis(2500),
|
||||
failureStore
|
||||
);
|
||||
assertThat(indicesPastRetention.size(), is(3));
|
||||
assertThat(indicesPastRetention.get(0).getName(), is(indicesSupplier.get().get(0).getName()));
|
||||
assertThat(indicesPastRetention.get(1).getName(), is(indicesSupplier.get().get(1).getName()));
|
||||
assertThat(indicesPastRetention.get(2).getName(), is(indicesSupplier.get().get(5).getName()));
|
||||
}
|
||||
|
||||
{
|
||||
// no index matches the retention age
|
||||
testRetentionReference.set(TimeValue.timeValueMillis(9000));
|
||||
List<Index> failureIndices = dataStream.getFailureIndicesPastRetention(metadata.getProject()::index, () -> now, null);
|
||||
assertThat(failureIndices.isEmpty(), is(true));
|
||||
List<Index> indicesPastRetention = dataStream.getIndicesPastRetention(
|
||||
metadata.getProject()::index,
|
||||
() -> now,
|
||||
TimeValue.timeValueMillis(9000),
|
||||
failureStore
|
||||
);
|
||||
assertThat(indicesPastRetention.isEmpty(), is(true));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -2034,12 +1829,44 @@ public class DataStreamTests extends AbstractXContentSerializingTestCase<DataStr
|
|||
}
|
||||
}
|
||||
|
||||
public void testFailuresLifecycle() {
|
||||
DataStream noFailureStoreDs = DataStream.builder("no-fs", List.of(new Index(randomAlphaOfLength(10), randomUUID()))).build();
|
||||
assertThat(noFailureStoreDs.getFailuresLifecycle(), nullValue());
|
||||
|
||||
assertThat(noFailureStoreDs.getFailuresLifecycle(true), equalTo(DataStreamLifecycle.DEFAULT_FAILURE_LIFECYCLE));
|
||||
assertThat(noFailureStoreDs.getFailuresLifecycle(randomBoolean() ? false : null), nullValue());
|
||||
|
||||
DataStream withFailureIndices = DataStream.builder("with-fs-indices", List.of(new Index(randomAlphaOfLength(10), randomUUID())))
|
||||
.setFailureIndices(
|
||||
DataStream.DataStreamIndices.failureIndicesBuilder(List.of(new Index(randomAlphaOfLength(10), randomUUID()))).build()
|
||||
)
|
||||
.build();
|
||||
assertThat(withFailureIndices.getFailuresLifecycle(), equalTo(DataStreamLifecycle.DEFAULT_FAILURE_LIFECYCLE));
|
||||
|
||||
DataStreamLifecycle lifecycle = DataStreamLifecycleTests.randomFailuresLifecycle();
|
||||
DataStream withFailuresLifecycle = DataStream.builder("with-fs", List.of(new Index(randomAlphaOfLength(10), randomUUID())))
|
||||
.setDataStreamOptions(new DataStreamOptions(new DataStreamFailureStore(randomBoolean(), lifecycle)))
|
||||
.build();
|
||||
assertThat(withFailuresLifecycle.getFailuresLifecycle(), equalTo(lifecycle));
|
||||
}
|
||||
|
||||
private DataStream createDataStream(
|
||||
Metadata.Builder builder,
|
||||
String dataStreamName,
|
||||
List<DataStreamMetadata> creationAndRolloverTimes,
|
||||
Settings.Builder backingIndicesSettings,
|
||||
@Nullable DataStreamLifecycle lifecycle
|
||||
) {
|
||||
return createDataStream(builder, dataStreamName, creationAndRolloverTimes, backingIndicesSettings, lifecycle, null);
|
||||
}
|
||||
|
||||
private DataStream createDataStream(
|
||||
Metadata.Builder builder,
|
||||
String dataStreamName,
|
||||
List<DataStreamMetadata> creationAndRolloverTimes,
|
||||
Settings.Builder backingIndicesSettings,
|
||||
@Nullable DataStreamLifecycle lifecycle,
|
||||
@Nullable DataStreamOptions dataStreamOptions
|
||||
) {
|
||||
int backingIndicesCount = creationAndRolloverTimes.size();
|
||||
final List<Index> backingIndices = createDataStreamIndices(
|
||||
|
@ -2058,7 +1885,7 @@ public class DataStreamTests extends AbstractXContentSerializingTestCase<DataStr
|
|||
backingIndicesCount,
|
||||
true
|
||||
);
|
||||
return newInstance(dataStreamName, backingIndices, backingIndicesCount, null, false, lifecycle, failureIndices);
|
||||
return newInstance(dataStreamName, backingIndices, backingIndicesCount, null, false, lifecycle, failureIndices, dataStreamOptions);
|
||||
}
|
||||
|
||||
private static List<Index> createDataStreamIndices(
|
||||
|
@ -2116,6 +1943,7 @@ public class DataStreamTests extends AbstractXContentSerializingTestCase<DataStr
|
|||
if (failureStore) {
|
||||
failureIndices = randomNonEmptyIndexInstances();
|
||||
}
|
||||
var failuresLifecycle = randomBoolean() ? null : DataStreamLifecycleTests.randomFailuresLifecycle();
|
||||
|
||||
DataStreamLifecycle lifecycle = DataStreamLifecycle.DEFAULT_DATA_LIFECYCLE;
|
||||
boolean isSystem = randomBoolean();
|
||||
|
@ -2131,7 +1959,7 @@ public class DataStreamTests extends AbstractXContentSerializingTestCase<DataStr
|
|||
randomBoolean(),
|
||||
randomBoolean() ? IndexMode.STANDARD : null, // IndexMode.TIME_SERIES triggers validation that many unit tests doesn't pass
|
||||
lifecycle,
|
||||
new DataStreamOptions(new DataStreamFailureStore(failureStore)),
|
||||
new DataStreamOptions(new DataStreamFailureStore(failureStore, failuresLifecycle)),
|
||||
failureIndices,
|
||||
false,
|
||||
null
|
||||
|
@ -2151,8 +1979,6 @@ public class DataStreamTests extends AbstractXContentSerializingTestCase<DataStr
|
|||
).getConditions().keySet()) {
|
||||
assertThat(serialized, containsString(label));
|
||||
}
|
||||
// We check that even if there was no retention provided by the user, the global retention applies
|
||||
assertThat(serialized, not(containsString("data_retention")));
|
||||
if (dataStream.isInternal() == false
|
||||
&& (globalRetention.defaultRetention() != null || globalRetention.maxRetention() != null)) {
|
||||
assertThat(serialized, containsString("effective_retention"));
|
||||
|
|
|
@ -825,13 +825,11 @@ public final class DataStreamTestHelper {
|
|||
return indicesService;
|
||||
}
|
||||
|
||||
public static DataStreamOptions.Template createDataStreamOptionsTemplate(Boolean failureStore) {
|
||||
if (failureStore == null) {
|
||||
public static DataStreamOptions.Template createDataStreamOptionsTemplate(Boolean failureStoreEnabled) {
|
||||
if (failureStoreEnabled == null) {
|
||||
return DataStreamOptions.Template.EMPTY;
|
||||
}
|
||||
return new DataStreamOptions.Template(
|
||||
ResettableValue.create(new DataStreamFailureStore.Template(ResettableValue.create(failureStore)))
|
||||
);
|
||||
return new DataStreamOptions.Template(DataStreamFailureStore.builder().enabled(failureStoreEnabled).buildTemplate());
|
||||
}
|
||||
|
||||
static Settings randomSettings() {
|
||||
|
|
|
@ -0,0 +1,328 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License
|
||||
* 2.0; you may not use this file except in compliance with the Elastic License
|
||||
* 2.0.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.xpack.core.action;
|
||||
|
||||
import org.elasticsearch.action.support.PlainActionFuture;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.ClusterStateUpdateTask;
|
||||
import org.elasticsearch.cluster.metadata.DataStream;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamFailureStore;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamFailureStoreSettings;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionSettings;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamLifecycle;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamOptions;
|
||||
import org.elasticsearch.cluster.metadata.Metadata;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.XContentHelper;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.core.Tuple;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.IndexMode;
|
||||
import org.elasticsearch.plugins.ActionPlugin;
|
||||
import org.elasticsearch.plugins.Plugin;
|
||||
import org.elasticsearch.protocol.xpack.XPackUsageRequest;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
import org.elasticsearch.xcontent.ToXContent;
|
||||
import org.elasticsearch.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.xcontent.XContentFactory;
|
||||
import org.elasticsearch.xcontent.XContentType;
|
||||
import org.elasticsearch.xpack.core.XPackClientPlugin;
|
||||
import org.junit.After;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
import static org.elasticsearch.xpack.core.action.XPackUsageFeatureAction.DATA_STREAMS;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
||||
public class DataStreamUsageTransportActionIT extends ESIntegTestCase {
|
||||
/*
|
||||
* The DataStreamUsageTransportAction is not exposed in the xpack core plugin, so we have a special test plugin to do this
|
||||
*/
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> nodePlugins() {
|
||||
return List.of(TestDataStreamUsagePlugin.class);
|
||||
}
|
||||
|
||||
@After
|
||||
private void cleanup() throws Exception {
|
||||
updateClusterState(clusterState -> {
|
||||
ClusterState.Builder clusterStateBuilder = new ClusterState.Builder(clusterState);
|
||||
Metadata.Builder metadataBuilder = Metadata.builder(clusterState.metadata());
|
||||
metadataBuilder.dataStreams(Map.of(), Map.of());
|
||||
clusterStateBuilder.metadata(metadataBuilder);
|
||||
return clusterStateBuilder.build();
|
||||
});
|
||||
updateClusterSettings(
|
||||
Settings.builder()
|
||||
.putNull(DataStreamGlobalRetentionSettings.DATA_STREAMS_DEFAULT_RETENTION_SETTING.getKey())
|
||||
.putNull(DataStreamFailureStoreSettings.DATA_STREAM_FAILURE_STORED_ENABLED_SETTING.getKey())
|
||||
);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testAction() throws Exception {
|
||||
// test empty results
|
||||
{
|
||||
Map<String, Object> map = getDataStreamUsage();
|
||||
assertThat(map.get("available"), equalTo(true));
|
||||
assertThat(map.get("enabled"), equalTo(true));
|
||||
assertThat(map.get("data_streams"), equalTo(0));
|
||||
assertThat(map.get("indices_count"), equalTo(0));
|
||||
|
||||
Map<String, Object> failureStoreMap = (Map<String, Object>) map.get("failure_store");
|
||||
assertThat(failureStoreMap.get("explicitly_enabled_count"), equalTo(0));
|
||||
assertThat(failureStoreMap.get("effectively_enabled_count"), equalTo(0));
|
||||
assertThat(failureStoreMap.get("failure_indices_count"), equalTo(0));
|
||||
|
||||
Map<String, Object> failuresLifecycleMap = (Map<String, Object>) failureStoreMap.get("lifecycle");
|
||||
assertThat(failuresLifecycleMap.get("explicitly_enabled_count"), equalTo(0));
|
||||
assertThat(failuresLifecycleMap.get("effectively_enabled_count"), equalTo(0));
|
||||
Map<String, Object> dataRetentionMap = (Map<String, Object>) failuresLifecycleMap.get("data_retention");
|
||||
assertThat(dataRetentionMap.size(), equalTo(1));
|
||||
assertThat(dataRetentionMap.get("configured_data_streams"), equalTo(0));
|
||||
|
||||
Map<String, Object> effectiveRetentionMap = (Map<String, Object>) failuresLifecycleMap.get("effective_retention");
|
||||
assertThat(effectiveRetentionMap.size(), equalTo(1));
|
||||
assertThat(effectiveRetentionMap.get("retained_data_streams"), equalTo(0));
|
||||
|
||||
Map<String, Object> globalRetentionMap = (Map<String, Object>) failuresLifecycleMap.get("global_retention");
|
||||
assertThat(globalRetentionMap.get("max"), equalTo(Map.of("defined", false)));
|
||||
assertThat(globalRetentionMap.get("default"), equalTo(Map.of("defined", false)));
|
||||
}
|
||||
|
||||
// Keep track of the data streams created
|
||||
int dataStreamsCount = randomIntBetween(1, 200);
|
||||
AtomicInteger backingIndicesCount = new AtomicInteger(0);
|
||||
AtomicInteger failureIndicesCount = new AtomicInteger(0);
|
||||
AtomicInteger explicitlyEnabledFailureStoreCount = new AtomicInteger(0);
|
||||
AtomicInteger effectivelyEnabledFailureStoreCount = new AtomicInteger(0);
|
||||
AtomicInteger explicitlyEnabledFailuresLifecycleCount = new AtomicInteger(0);
|
||||
AtomicInteger effectivelyEnabledFailuresLifecycleCount = new AtomicInteger(0);
|
||||
AtomicInteger failuresLifecycleWithRetention = new AtomicInteger(0);
|
||||
AtomicInteger failuresLifecycleWithDefaultRetention = new AtomicInteger(0);
|
||||
|
||||
AtomicLong totalRetentionTimes = new AtomicLong(0);
|
||||
AtomicLong minRetention = new AtomicLong(Long.MAX_VALUE);
|
||||
AtomicLong maxRetention = new AtomicLong(Long.MIN_VALUE);
|
||||
|
||||
TimeValue defaultRetention = TimeValue.timeValueDays(10);
|
||||
Settings.Builder settingsBuilder = Settings.builder()
|
||||
.put(DataStreamFailureStoreSettings.DATA_STREAM_FAILURE_STORED_ENABLED_SETTING.getKey(), "mis-*");
|
||||
boolean useDefaultRetention = randomBoolean();
|
||||
if (useDefaultRetention) {
|
||||
settingsBuilder.put(
|
||||
DataStreamGlobalRetentionSettings.DATA_STREAMS_DEFAULT_RETENTION_SETTING.getKey(),
|
||||
defaultRetention.getStringRep()
|
||||
);
|
||||
}
|
||||
updateClusterSettings(settingsBuilder);
|
||||
|
||||
/*
|
||||
* We now add a number of simulated data streams to the cluster state. Some have failure store, some don't. The ones with failure
|
||||
* store may or may not have lifecycle with varying retention periods. After adding them, we make sure the numbers add up.
|
||||
*/
|
||||
updateClusterState(clusterState -> {
|
||||
Metadata.Builder metadataBuilder = Metadata.builder(clusterState.metadata());
|
||||
Map<String, DataStream> dataStreamMap = new HashMap<>();
|
||||
for (int i = 0; i < dataStreamsCount; i++) {
|
||||
boolean replicated = randomBoolean();
|
||||
boolean systemDataStream = rarely();
|
||||
List<Index> backingIndices = IntStream.range(0, randomIntBetween(1, 10))
|
||||
.mapToObj(ignore -> new Index(randomAlphaOfLength(60), randomAlphaOfLength(60)))
|
||||
.toList();
|
||||
backingIndicesCount.addAndGet(backingIndices.size());
|
||||
List<Index> failureIndices = IntStream.range(0, randomIntBetween(0, 10))
|
||||
.mapToObj(ignore -> new Index(randomAlphaOfLength(60), randomAlphaOfLength(60)))
|
||||
.toList();
|
||||
failureIndicesCount.addAndGet(failureIndices.size());
|
||||
Boolean failureStoreEnabled = randomBoolean() ? null : randomBoolean();
|
||||
boolean enabledBySetting = failureStoreEnabled == null && randomBoolean() && systemDataStream == false;
|
||||
if (failureStoreEnabled == null) {
|
||||
if (enabledBySetting) {
|
||||
effectivelyEnabledFailureStoreCount.incrementAndGet();
|
||||
}
|
||||
} else if (failureStoreEnabled) {
|
||||
explicitlyEnabledFailureStoreCount.incrementAndGet();
|
||||
effectivelyEnabledFailureStoreCount.incrementAndGet();
|
||||
}
|
||||
DataStreamLifecycle lifecycle = randomBoolean()
|
||||
? null
|
||||
: DataStreamLifecycle.createFailuresLifecycle(randomBoolean(), TimeValue.timeValueDays(randomIntBetween(1, 10)));
|
||||
if (lifecycle != null && lifecycle.enabled()) {
|
||||
explicitlyEnabledFailuresLifecycleCount.incrementAndGet();
|
||||
effectivelyEnabledFailuresLifecycleCount.incrementAndGet();
|
||||
if (lifecycle.dataRetention() != null) {
|
||||
long retentionMillis = lifecycle.dataRetention().getMillis();
|
||||
totalRetentionTimes.addAndGet(retentionMillis);
|
||||
failuresLifecycleWithRetention.incrementAndGet();
|
||||
if (retentionMillis < minRetention.get()) {
|
||||
minRetention.set(retentionMillis);
|
||||
}
|
||||
if (retentionMillis > maxRetention.get()) {
|
||||
maxRetention.set(retentionMillis);
|
||||
}
|
||||
}
|
||||
}
|
||||
if (lifecycle == null
|
||||
&& (enabledBySetting || Boolean.TRUE.equals(failureStoreEnabled) || failureIndices.isEmpty() == false)) {
|
||||
effectivelyEnabledFailuresLifecycleCount.incrementAndGet();
|
||||
if (systemDataStream == false && useDefaultRetention) {
|
||||
failuresLifecycleWithDefaultRetention.incrementAndGet();
|
||||
}
|
||||
}
|
||||
DataStream dataStream = new DataStream(
|
||||
enabledBySetting ? "mis-" + randomAlphaOfLength(10) : randomAlphaOfLength(50),
|
||||
backingIndices,
|
||||
randomLongBetween(0, 1000),
|
||||
Map.of(),
|
||||
systemDataStream || randomBoolean(),
|
||||
replicated,
|
||||
systemDataStream,
|
||||
randomBoolean(),
|
||||
IndexMode.STANDARD,
|
||||
null,
|
||||
failureStoreEnabled == null && lifecycle == null
|
||||
? DataStreamOptions.EMPTY
|
||||
: new DataStreamOptions(new DataStreamFailureStore(failureStoreEnabled, lifecycle)),
|
||||
failureIndices,
|
||||
replicated == false && randomBoolean(),
|
||||
null
|
||||
);
|
||||
dataStreamMap.put(dataStream.getName(), dataStream);
|
||||
}
|
||||
metadataBuilder.dataStreams(dataStreamMap, Map.of());
|
||||
ClusterState.Builder clusterStateBuilder = new ClusterState.Builder(clusterState);
|
||||
clusterStateBuilder.metadata(metadataBuilder);
|
||||
return clusterStateBuilder.build();
|
||||
});
|
||||
|
||||
int retainedDataStreams = failuresLifecycleWithRetention.get() + failuresLifecycleWithDefaultRetention.get();
|
||||
|
||||
int expectedMinimumDataRetention = minRetention.get() == Long.MAX_VALUE ? 0 : minRetention.intValue();
|
||||
int expectedMinimumEffectiveRetention = failuresLifecycleWithDefaultRetention.get() > 0
|
||||
? (int) Math.min(minRetention.get(), defaultRetention.getMillis())
|
||||
: expectedMinimumDataRetention;
|
||||
|
||||
int expectedMaximumDataRetention = maxRetention.get() == Long.MIN_VALUE ? 0 : maxRetention.intValue();
|
||||
int expectedMaximumEffectiveRetention = failuresLifecycleWithDefaultRetention.get() > 0
|
||||
? (int) Math.max(maxRetention.get(), defaultRetention.getMillis())
|
||||
: expectedMaximumDataRetention;
|
||||
|
||||
double expectedAverageDataRetention = failuresLifecycleWithRetention.get() == 0
|
||||
? 0.0
|
||||
: totalRetentionTimes.doubleValue() / failuresLifecycleWithRetention.get();
|
||||
double expectedAverageEffectiveRetention = failuresLifecycleWithDefaultRetention.get() > 0
|
||||
? (totalRetentionTimes.doubleValue() + failuresLifecycleWithDefaultRetention.get() * defaultRetention.getMillis())
|
||||
/ retainedDataStreams
|
||||
: expectedAverageDataRetention;
|
||||
|
||||
Map<String, Object> map = getDataStreamUsage();
|
||||
assertThat(map.get("available"), equalTo(true));
|
||||
assertThat(map.get("enabled"), equalTo(true));
|
||||
assertThat(map.get("data_streams"), equalTo(dataStreamsCount));
|
||||
assertThat(map.get("indices_count"), equalTo(backingIndicesCount.get()));
|
||||
|
||||
Map<String, Object> failureStoreMap = (Map<String, Object>) map.get("failure_store");
|
||||
assertThat(failureStoreMap.get("explicitly_enabled_count"), equalTo(explicitlyEnabledFailureStoreCount.get()));
|
||||
assertThat(failureStoreMap.get("effectively_enabled_count"), equalTo(effectivelyEnabledFailureStoreCount.get()));
|
||||
assertThat(failureStoreMap.get("failure_indices_count"), equalTo(failureIndicesCount.get()));
|
||||
|
||||
Map<String, Object> failuresLifecycleMap = (Map<String, Object>) failureStoreMap.get("lifecycle");
|
||||
assertThat(failuresLifecycleMap.get("explicitly_enabled_count"), equalTo(explicitlyEnabledFailuresLifecycleCount.get()));
|
||||
assertThat(failuresLifecycleMap.get("effectively_enabled_count"), equalTo(effectivelyEnabledFailuresLifecycleCount.get()));
|
||||
|
||||
Map<String, Object> dataRetentionMap = (Map<String, Object>) failuresLifecycleMap.get("data_retention");
|
||||
assertThat(dataRetentionMap.get("configured_data_streams"), equalTo(failuresLifecycleWithRetention.get()));
|
||||
if (failuresLifecycleWithRetention.get() > 0) {
|
||||
assertThat(dataRetentionMap.get("minimum_millis"), equalTo(expectedMinimumDataRetention));
|
||||
assertThat(dataRetentionMap.get("maximum_millis"), equalTo(expectedMaximumDataRetention));
|
||||
assertThat(dataRetentionMap.get("average_millis"), equalTo(expectedAverageDataRetention));
|
||||
}
|
||||
|
||||
Map<String, Object> effectiveRetentionMap = (Map<String, Object>) failuresLifecycleMap.get("effective_retention");
|
||||
assertThat(effectiveRetentionMap.get("retained_data_streams"), equalTo(retainedDataStreams));
|
||||
if (retainedDataStreams > 0) {
|
||||
assertThat(effectiveRetentionMap.get("minimum_millis"), equalTo(expectedMinimumEffectiveRetention));
|
||||
assertThat(effectiveRetentionMap.get("maximum_millis"), equalTo(expectedMaximumEffectiveRetention));
|
||||
assertThat(effectiveRetentionMap.get("average_millis"), equalTo(expectedAverageEffectiveRetention));
|
||||
}
|
||||
|
||||
Map<String, Map<String, Object>> globalRetentionMap = (Map<String, Map<String, Object>>) failuresLifecycleMap.get(
|
||||
"global_retention"
|
||||
);
|
||||
assertThat(globalRetentionMap.get("max").get("defined"), equalTo(false));
|
||||
assertThat(globalRetentionMap.get("default").get("defined"), equalTo(useDefaultRetention));
|
||||
if (useDefaultRetention) {
|
||||
assertThat(
|
||||
globalRetentionMap.get("default").get("affected_data_streams"),
|
||||
equalTo(failuresLifecycleWithDefaultRetention.get())
|
||||
);
|
||||
assertThat(globalRetentionMap.get("default").get("retention_millis"), equalTo((int) defaultRetention.getMillis()));
|
||||
}
|
||||
}
|
||||
|
||||
private Map<String, Object> getDataStreamUsage() throws IOException {
|
||||
XPackUsageFeatureResponse response = safeGet(client().execute(DATA_STREAMS, new XPackUsageRequest(SAFE_AWAIT_TIMEOUT)));
|
||||
XContentBuilder builder = XContentFactory.jsonBuilder();
|
||||
builder = response.getUsage().toXContent(builder, ToXContent.EMPTY_PARAMS);
|
||||
Tuple<XContentType, Map<String, Object>> tuple = XContentHelper.convertToMap(
|
||||
BytesReference.bytes(builder),
|
||||
true,
|
||||
XContentType.JSON
|
||||
);
|
||||
return tuple.v2();
|
||||
}
|
||||
|
||||
/*
|
||||
* Updates the cluster state in the internal cluster using the provided function
|
||||
*/
|
||||
protected static void updateClusterState(final Function<ClusterState, ClusterState> updater) throws Exception {
|
||||
final PlainActionFuture<Void> future = new PlainActionFuture<>();
|
||||
final ClusterService clusterService = internalCluster().getCurrentMasterNodeInstance(ClusterService.class);
|
||||
clusterService.submitUnbatchedStateUpdateTask("test", new ClusterStateUpdateTask() {
|
||||
@Override
|
||||
public ClusterState execute(ClusterState currentState) {
|
||||
return updater.apply(currentState);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Exception e) {
|
||||
future.onFailure(e);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clusterStateProcessed(ClusterState oldState, ClusterState newState) {
|
||||
future.onResponse(null);
|
||||
}
|
||||
});
|
||||
future.get();
|
||||
}
|
||||
|
||||
/*
|
||||
* This plugin exposes the DataStreamUsageTransportAction.
|
||||
*/
|
||||
public static final class TestDataStreamUsagePlugin extends XPackClientPlugin {
|
||||
@Override
|
||||
public List<ActionHandler> getActions() {
|
||||
List<ActionHandler> actions = new ArrayList<>();
|
||||
actions.add(new ActionPlugin.ActionHandler(DATA_STREAMS, DataStreamUsageTransportAction.class));
|
||||
return actions;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -26,7 +26,6 @@ import org.elasticsearch.transport.TransportService;
|
|||
import org.elasticsearch.xpack.core.datastreams.DataStreamLifecycleFeatureSetUsage;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.LongSummaryStatistics;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -102,11 +101,12 @@ public class DataStreamLifecycleUsageTransportAction extends XPackUsageFeatureTr
|
|||
}
|
||||
}
|
||||
}
|
||||
Map<String, DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats> globalRetentionStats = getGlobalRetentionStats(
|
||||
globalRetention,
|
||||
dataStreamsWithDefaultRetention,
|
||||
dataStreamsWithMaxRetention
|
||||
);
|
||||
Map<String, DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats> globalRetentionStats =
|
||||
DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats.getGlobalRetentionStats(
|
||||
globalRetention,
|
||||
dataStreamsWithDefaultRetention,
|
||||
dataStreamsWithMaxRetention
|
||||
);
|
||||
return new DataStreamLifecycleFeatureSetUsage.LifecycleStats(
|
||||
dataStreamsWithLifecycles,
|
||||
DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING.getDefault(null).equals(rolloverConfiguration),
|
||||
|
@ -115,31 +115,4 @@ public class DataStreamLifecycleUsageTransportAction extends XPackUsageFeatureTr
|
|||
globalRetentionStats
|
||||
);
|
||||
}
|
||||
|
||||
private static Map<String, DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats> getGlobalRetentionStats(
|
||||
DataStreamGlobalRetention globalRetention,
|
||||
long dataStreamsWithDefaultRetention,
|
||||
long dataStreamsWithMaxRetention
|
||||
) {
|
||||
if (globalRetention == null) {
|
||||
return Map.of();
|
||||
}
|
||||
Map<String, DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats> globalRetentionStats = new HashMap<>();
|
||||
if (globalRetention.defaultRetention() != null) {
|
||||
globalRetentionStats.put(
|
||||
DataStreamLifecycleFeatureSetUsage.LifecycleStats.DEFAULT_RETENTION_FIELD_NAME,
|
||||
new DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats(
|
||||
dataStreamsWithDefaultRetention,
|
||||
globalRetention.defaultRetention()
|
||||
)
|
||||
);
|
||||
}
|
||||
if (globalRetention.maxRetention() != null) {
|
||||
globalRetentionStats.put(
|
||||
DataStreamLifecycleFeatureSetUsage.LifecycleStats.MAX_RETENTION_FIELD_NAME,
|
||||
new DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats(dataStreamsWithMaxRetention, globalRetention.maxRetention())
|
||||
);
|
||||
}
|
||||
return globalRetentionStats;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -12,19 +12,27 @@ import org.elasticsearch.action.support.ActionFilters;
|
|||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.metadata.DataStream;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamFailureStoreSettings;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionSettings;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamLifecycle;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.core.Tuple;
|
||||
import org.elasticsearch.injection.guice.Inject;
|
||||
import org.elasticsearch.protocol.xpack.XPackUsageRequest;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.core.datastreams.DataStreamFeatureSetUsage;
|
||||
import org.elasticsearch.xpack.core.datastreams.DataStreamLifecycleFeatureSetUsage;
|
||||
|
||||
import java.util.LongSummaryStatistics;
|
||||
import java.util.Map;
|
||||
|
||||
public class DataStreamUsageTransportAction extends XPackUsageFeatureTransportAction {
|
||||
|
||||
private final DataStreamFailureStoreSettings dataStreamFailureStoreSettings;
|
||||
private final DataStreamGlobalRetentionSettings globalRetentionSettings;
|
||||
|
||||
@Inject
|
||||
public DataStreamUsageTransportAction(
|
||||
|
@ -32,10 +40,12 @@ public class DataStreamUsageTransportAction extends XPackUsageFeatureTransportAc
|
|||
ClusterService clusterService,
|
||||
ThreadPool threadPool,
|
||||
ActionFilters actionFilters,
|
||||
DataStreamFailureStoreSettings dataStreamFailureStoreSettings
|
||||
DataStreamFailureStoreSettings dataStreamFailureStoreSettings,
|
||||
DataStreamGlobalRetentionSettings globalRetentionSettings
|
||||
) {
|
||||
super(XPackUsageFeatureAction.DATA_STREAMS.name(), transportService, clusterService, threadPool, actionFilters);
|
||||
this.dataStreamFailureStoreSettings = dataStreamFailureStoreSettings;
|
||||
this.globalRetentionSettings = globalRetentionSettings;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -50,24 +60,74 @@ public class DataStreamUsageTransportAction extends XPackUsageFeatureTransportAc
|
|||
long failureStoreExplicitlyEnabledCounter = 0;
|
||||
long failureStoreEffectivelyEnabledCounter = 0;
|
||||
long failureIndicesCounter = 0;
|
||||
long failuresLifecycleExplicitlyEnabledCounter = 0;
|
||||
long failuresLifecycleEffectivelyEnabledCounter = 0;
|
||||
LongSummaryStatistics dataRetentionStats = new LongSummaryStatistics();
|
||||
LongSummaryStatistics effectiveRetentionStats = new LongSummaryStatistics();
|
||||
long affectedByMaxRetentionCounter = 0;
|
||||
long affectedByDefaultRetentionCounter = 0;
|
||||
DataStreamGlobalRetention globalRetention = globalRetentionSettings.get();
|
||||
for (DataStream ds : dataStreams.values()) {
|
||||
backingIndicesCounter += ds.getIndices().size();
|
||||
if (ds.isFailureStoreExplicitlyEnabled()) {
|
||||
failureStoreExplicitlyEnabledCounter++;
|
||||
}
|
||||
if (ds.isFailureStoreEffectivelyEnabled(dataStreamFailureStoreSettings)) {
|
||||
boolean failureStoreEffectivelyEnabled = ds.isFailureStoreEffectivelyEnabled(dataStreamFailureStoreSettings);
|
||||
if (failureStoreEffectivelyEnabled) {
|
||||
failureStoreEffectivelyEnabledCounter++;
|
||||
}
|
||||
if (ds.getFailureIndices().isEmpty() == false) {
|
||||
failureIndicesCounter += ds.getFailureIndices().size();
|
||||
}
|
||||
|
||||
// Track explicitly enabled failures lifecycle configuration
|
||||
DataStreamLifecycle configuredFailuresLifecycle = ds.getDataStreamOptions() != null
|
||||
&& ds.getDataStreamOptions().failureStore() != null
|
||||
&& ds.getDataStreamOptions().failureStore().lifecycle() != null
|
||||
? ds.getDataStreamOptions().failureStore().lifecycle()
|
||||
: null;
|
||||
if (configuredFailuresLifecycle != null && configuredFailuresLifecycle.enabled()) {
|
||||
failuresLifecycleExplicitlyEnabledCounter++;
|
||||
if (configuredFailuresLifecycle.dataRetention() != null) {
|
||||
dataRetentionStats.accept(configuredFailuresLifecycle.dataRetention().getMillis());
|
||||
}
|
||||
}
|
||||
|
||||
// Track effective failure lifecycle
|
||||
DataStreamLifecycle effectiveFailuresLifecycle = ds.getFailuresLifecycle(failureStoreEffectivelyEnabled);
|
||||
if (effectiveFailuresLifecycle != null && effectiveFailuresLifecycle.enabled()) {
|
||||
failuresLifecycleEffectivelyEnabledCounter++;
|
||||
// Track effective retention
|
||||
Tuple<TimeValue, DataStreamLifecycle.RetentionSource> effectiveDataRetentionWithSource = effectiveFailuresLifecycle
|
||||
.getEffectiveDataRetentionWithSource(globalRetention, ds.isInternal());
|
||||
|
||||
// Track global retention usage
|
||||
if (effectiveDataRetentionWithSource.v1() != null) {
|
||||
effectiveRetentionStats.accept(effectiveDataRetentionWithSource.v1().getMillis());
|
||||
if (effectiveDataRetentionWithSource.v2().equals(DataStreamLifecycle.RetentionSource.MAX_GLOBAL_RETENTION)) {
|
||||
affectedByMaxRetentionCounter++;
|
||||
}
|
||||
if (effectiveDataRetentionWithSource.v2().equals(DataStreamLifecycle.RetentionSource.DEFAULT_GLOBAL_RETENTION)) {
|
||||
affectedByDefaultRetentionCounter++;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
final DataStreamFeatureSetUsage.DataStreamStats stats = new DataStreamFeatureSetUsage.DataStreamStats(
|
||||
dataStreams.size(),
|
||||
backingIndicesCounter,
|
||||
failureStoreExplicitlyEnabledCounter,
|
||||
failureStoreEffectivelyEnabledCounter,
|
||||
failureIndicesCounter
|
||||
failureIndicesCounter,
|
||||
failuresLifecycleExplicitlyEnabledCounter,
|
||||
failuresLifecycleEffectivelyEnabledCounter,
|
||||
DataStreamLifecycleFeatureSetUsage.RetentionStats.create(dataRetentionStats),
|
||||
DataStreamLifecycleFeatureSetUsage.RetentionStats.create(effectiveRetentionStats),
|
||||
DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats.getGlobalRetentionStats(
|
||||
globalRetention,
|
||||
affectedByDefaultRetentionCounter,
|
||||
affectedByMaxRetentionCounter
|
||||
)
|
||||
);
|
||||
final DataStreamFeatureSetUsage usage = new DataStreamFeatureSetUsage(stats);
|
||||
listener.onResponse(new XPackUsageFeatureResponse(usage));
|
||||
|
|
|
@ -16,8 +16,10 @@ import org.elasticsearch.common.io.stream.Writeable;
|
|||
import org.elasticsearch.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.xpack.core.XPackFeatureUsage;
|
||||
import org.elasticsearch.xpack.core.XPackField;
|
||||
import org.elasticsearch.xpack.core.datastreams.DataStreamLifecycleFeatureSetUsage.LifecycleStats;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
public class DataStreamFeatureSetUsage extends XPackFeatureUsage {
|
||||
|
@ -44,6 +46,10 @@ public class DataStreamFeatureSetUsage extends XPackFeatureUsage {
|
|||
return TransportVersions.ZERO;
|
||||
}
|
||||
|
||||
public DataStreamStats getStats() {
|
||||
return streamStats;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
super.innerXContent(builder, params);
|
||||
|
@ -53,6 +59,36 @@ public class DataStreamFeatureSetUsage extends XPackFeatureUsage {
|
|||
builder.field("explicitly_enabled_count", streamStats.failureStoreExplicitlyEnabledDataStreamCount);
|
||||
builder.field("effectively_enabled_count", streamStats.failureStoreEffectivelyEnabledDataStreamCount);
|
||||
builder.field("failure_indices_count", streamStats.failureStoreIndicesCount);
|
||||
|
||||
// Failures lifecycle
|
||||
builder.startObject("lifecycle");
|
||||
builder.field("explicitly_enabled_count", streamStats.failuresLifecycleExplicitlyEnabledCount);
|
||||
builder.field("effectively_enabled_count", streamStats.failuresLifecycleEffectivelyEnabledCount);
|
||||
|
||||
// Retention
|
||||
DataStreamLifecycleFeatureSetUsage.RetentionStats.toXContentFragment(
|
||||
builder,
|
||||
streamStats.failuresLifecycleDataRetentionStats,
|
||||
false
|
||||
);
|
||||
DataStreamLifecycleFeatureSetUsage.RetentionStats.toXContentFragment(
|
||||
builder,
|
||||
streamStats.failuresLifecycleEffectiveRetentionStats,
|
||||
true
|
||||
);
|
||||
builder.startObject("global_retention");
|
||||
DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats.toXContentFragment(
|
||||
builder,
|
||||
LifecycleStats.DEFAULT_RETENTION_FIELD_NAME,
|
||||
streamStats.globalRetentionStats.get(LifecycleStats.DEFAULT_RETENTION_FIELD_NAME)
|
||||
);
|
||||
DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats.toXContentFragment(
|
||||
builder,
|
||||
LifecycleStats.MAX_RETENTION_FIELD_NAME,
|
||||
streamStats.globalRetentionStats.get(LifecycleStats.MAX_RETENTION_FIELD_NAME)
|
||||
);
|
||||
builder.endObject();
|
||||
builder.endObject();
|
||||
builder.endObject();
|
||||
}
|
||||
|
||||
|
@ -83,7 +119,12 @@ public class DataStreamFeatureSetUsage extends XPackFeatureUsage {
|
|||
long indicesBehindDataStream,
|
||||
long failureStoreExplicitlyEnabledDataStreamCount,
|
||||
long failureStoreEffectivelyEnabledDataStreamCount,
|
||||
long failureStoreIndicesCount
|
||||
long failureStoreIndicesCount,
|
||||
long failuresLifecycleExplicitlyEnabledCount,
|
||||
long failuresLifecycleEffectivelyEnabledCount,
|
||||
DataStreamLifecycleFeatureSetUsage.RetentionStats failuresLifecycleDataRetentionStats,
|
||||
DataStreamLifecycleFeatureSetUsage.RetentionStats failuresLifecycleEffectiveRetentionStats,
|
||||
Map<String, DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats> globalRetentionStats
|
||||
) implements Writeable {
|
||||
|
||||
public DataStreamStats(StreamInput in) throws IOException {
|
||||
|
@ -92,7 +133,18 @@ public class DataStreamFeatureSetUsage extends XPackFeatureUsage {
|
|||
in.readVLong(),
|
||||
in.getTransportVersion().onOrAfter(TransportVersions.V_8_15_0) ? in.readVLong() : 0,
|
||||
in.getTransportVersion().onOrAfter(TransportVersions.FAILURE_STORE_ENABLED_BY_CLUSTER_SETTING) ? in.readVLong() : 0,
|
||||
in.getTransportVersion().onOrAfter(TransportVersions.V_8_15_0) ? in.readVLong() : 0
|
||||
in.getTransportVersion().onOrAfter(TransportVersions.V_8_15_0) ? in.readVLong() : 0,
|
||||
in.getTransportVersion().onOrAfter(TransportVersions.INTRODUCE_FAILURES_LIFECYCLE) ? in.readVLong() : 0,
|
||||
in.getTransportVersion().onOrAfter(TransportVersions.INTRODUCE_FAILURES_LIFECYCLE) ? in.readVLong() : 0,
|
||||
in.getTransportVersion().onOrAfter(TransportVersions.INTRODUCE_FAILURES_LIFECYCLE)
|
||||
? DataStreamLifecycleFeatureSetUsage.RetentionStats.read(in)
|
||||
: DataStreamLifecycleFeatureSetUsage.RetentionStats.NO_DATA,
|
||||
in.getTransportVersion().onOrAfter(TransportVersions.INTRODUCE_FAILURES_LIFECYCLE)
|
||||
? DataStreamLifecycleFeatureSetUsage.RetentionStats.read(in)
|
||||
: DataStreamLifecycleFeatureSetUsage.RetentionStats.NO_DATA,
|
||||
in.getTransportVersion().onOrAfter(TransportVersions.INTRODUCE_FAILURES_LIFECYCLE)
|
||||
? in.readMap(DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats::new)
|
||||
: Map.of()
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -107,6 +159,13 @@ public class DataStreamFeatureSetUsage extends XPackFeatureUsage {
|
|||
}
|
||||
out.writeVLong(this.failureStoreIndicesCount);
|
||||
}
|
||||
if (out.getTransportVersion().onOrAfter(TransportVersions.INTRODUCE_FAILURES_LIFECYCLE)) {
|
||||
out.writeVLong(failuresLifecycleExplicitlyEnabledCount);
|
||||
out.writeVLong(failuresLifecycleEffectivelyEnabledCount);
|
||||
failuresLifecycleDataRetentionStats.writeTo(out);
|
||||
failuresLifecycleEffectiveRetentionStats.writeTo(out);
|
||||
out.writeMap(globalRetentionStats, (o, v) -> v.writeTo(o));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -9,6 +9,7 @@ package org.elasticsearch.xpack.core.datastreams;
|
|||
|
||||
import org.elasticsearch.TransportVersion;
|
||||
import org.elasticsearch.TransportVersions;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
|
@ -20,6 +21,7 @@ import org.elasticsearch.xpack.core.XPackFeatureUsage;
|
|||
import org.elasticsearch.xpack.core.XPackField;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.LongSummaryStatistics;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
@ -183,41 +185,23 @@ public class DataStreamLifecycleFeatureSetUsage extends XPackFeatureUsage {
|
|||
builder.field("count", dataStreamsWithLifecyclesCount);
|
||||
builder.field("default_rollover_used", defaultRolloverUsed);
|
||||
|
||||
builder.startObject("data_retention");
|
||||
builder.field("configured_data_streams", dataRetentionStats.dataStreamCount());
|
||||
if (dataRetentionStats.dataStreamCount() > 0) {
|
||||
builder.field("minimum_millis", dataRetentionStats.minMillis);
|
||||
builder.field("maximum_millis", dataRetentionStats.maxMillis);
|
||||
builder.field("average_millis", dataRetentionStats.avgMillis);
|
||||
}
|
||||
builder.endObject();
|
||||
|
||||
builder.startObject("effective_retention");
|
||||
builder.field("retained_data_streams", effectiveRetentionStats.dataStreamCount());
|
||||
if (effectiveRetentionStats.dataStreamCount() > 0) {
|
||||
builder.field("minimum_millis", effectiveRetentionStats.minMillis);
|
||||
builder.field("maximum_millis", effectiveRetentionStats.maxMillis);
|
||||
builder.field("average_millis", effectiveRetentionStats.avgMillis);
|
||||
}
|
||||
builder.endObject();
|
||||
RetentionStats.toXContentFragment(builder, dataRetentionStats, false);
|
||||
RetentionStats.toXContentFragment(builder, effectiveRetentionStats, true);
|
||||
|
||||
builder.startObject("global_retention");
|
||||
globalRetentionStatsToXContent(builder, params, LifecycleStats.DEFAULT_RETENTION_FIELD_NAME);
|
||||
globalRetentionStatsToXContent(builder, params, LifecycleStats.MAX_RETENTION_FIELD_NAME);
|
||||
GlobalRetentionStats.toXContentFragment(
|
||||
builder,
|
||||
LifecycleStats.DEFAULT_RETENTION_FIELD_NAME,
|
||||
globalRetentionStats.get(LifecycleStats.DEFAULT_RETENTION_FIELD_NAME)
|
||||
);
|
||||
GlobalRetentionStats.toXContentFragment(
|
||||
builder,
|
||||
LifecycleStats.MAX_RETENTION_FIELD_NAME,
|
||||
globalRetentionStats.get(LifecycleStats.MAX_RETENTION_FIELD_NAME)
|
||||
);
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
private void globalRetentionStatsToXContent(XContentBuilder builder, Params params, String retentionType) throws IOException {
|
||||
builder.startObject(retentionType);
|
||||
GlobalRetentionStats stats = globalRetentionStats.get(retentionType);
|
||||
builder.field("defined", stats != null);
|
||||
if (stats != null) {
|
||||
builder.field("affected_data_streams", stats.dataStreamCount());
|
||||
builder.field("retention_millis", stats.retention());
|
||||
}
|
||||
builder.endObject();
|
||||
}
|
||||
}
|
||||
|
||||
public record RetentionStats(long dataStreamCount, Double avgMillis, Long minMillis, Long maxMillis) implements Writeable {
|
||||
|
@ -251,6 +235,17 @@ public class DataStreamLifecycleFeatureSetUsage extends XPackFeatureUsage {
|
|||
out.writeVLong(maxMillis);
|
||||
}
|
||||
}
|
||||
|
||||
static void toXContentFragment(XContentBuilder builder, RetentionStats stats, boolean isEffectiveRetention) throws IOException {
|
||||
builder.startObject(isEffectiveRetention ? "effective_retention" : "data_retention");
|
||||
builder.field(isEffectiveRetention ? "retained_data_streams" : "configured_data_streams", stats.dataStreamCount());
|
||||
if (stats.dataStreamCount() > 0) {
|
||||
builder.field("minimum_millis", stats.minMillis);
|
||||
builder.field("maximum_millis", stats.maxMillis);
|
||||
builder.field("average_millis", stats.avgMillis);
|
||||
}
|
||||
builder.endObject();
|
||||
}
|
||||
}
|
||||
|
||||
public record GlobalRetentionStats(long dataStreamCount, long retention) implements Writeable {
|
||||
|
@ -263,10 +258,44 @@ public class DataStreamLifecycleFeatureSetUsage extends XPackFeatureUsage {
|
|||
this(in.readVLong(), in.readVLong());
|
||||
}
|
||||
|
||||
public static Map<String, GlobalRetentionStats> getGlobalRetentionStats(
|
||||
DataStreamGlobalRetention globalRetention,
|
||||
long dataStreamsWithDefaultRetention,
|
||||
long dataStreamsWithMaxRetention
|
||||
) {
|
||||
if (globalRetention == null) {
|
||||
return Map.of();
|
||||
}
|
||||
Map<String, GlobalRetentionStats> globalRetentionStats = new HashMap<>();
|
||||
if (globalRetention.defaultRetention() != null) {
|
||||
globalRetentionStats.put(
|
||||
LifecycleStats.DEFAULT_RETENTION_FIELD_NAME,
|
||||
new GlobalRetentionStats(dataStreamsWithDefaultRetention, globalRetention.defaultRetention())
|
||||
);
|
||||
}
|
||||
if (globalRetention.maxRetention() != null) {
|
||||
globalRetentionStats.put(
|
||||
LifecycleStats.MAX_RETENTION_FIELD_NAME,
|
||||
new GlobalRetentionStats(dataStreamsWithMaxRetention, globalRetention.maxRetention())
|
||||
);
|
||||
}
|
||||
return globalRetentionStats;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeVLong(dataStreamCount);
|
||||
out.writeVLong(retention);
|
||||
}
|
||||
|
||||
static void toXContentFragment(XContentBuilder builder, String retentionType, GlobalRetentionStats stats) throws IOException {
|
||||
builder.startObject(retentionType);
|
||||
builder.field("defined", stats != null);
|
||||
if (stats != null) {
|
||||
builder.field("affected_data_streams", stats.dataStreamCount());
|
||||
builder.field("retention_millis", stats.retention());
|
||||
}
|
||||
builder.endObject();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,116 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License
|
||||
* 2.0; you may not use this file except in compliance with the Elastic License
|
||||
* 2.0.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.xpack.core.datastreams;
|
||||
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.test.AbstractWireSerializingTestCase;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.elasticsearch.xpack.core.datastreams.DataStreamLifecycleFeatureSetUsageTests.generateGlobalRetention;
|
||||
import static org.elasticsearch.xpack.core.datastreams.DataStreamLifecycleFeatureSetUsageTests.generateRetentionStats;
|
||||
|
||||
public class DataStreamFeatureSetUsageTests extends AbstractWireSerializingTestCase<DataStreamFeatureSetUsage> {
|
||||
|
||||
@Override
|
||||
protected DataStreamFeatureSetUsage createTestInstance() {
|
||||
return new DataStreamFeatureSetUsage(
|
||||
new DataStreamFeatureSetUsage.DataStreamStats(
|
||||
randomNonNegativeLong(),
|
||||
randomNonNegativeLong(),
|
||||
randomNonNegativeLong(),
|
||||
randomNonNegativeLong(),
|
||||
randomNonNegativeLong(),
|
||||
randomNonNegativeLong(),
|
||||
randomNonNegativeLong(),
|
||||
generateRetentionStats(),
|
||||
generateRetentionStats(),
|
||||
randomBoolean() ? Map.of() : Map.of("default", generateGlobalRetention(), "max", generateGlobalRetention())
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DataStreamFeatureSetUsage mutateInstance(DataStreamFeatureSetUsage instance) {
|
||||
var totalDataStreamCount = instance.getStats().totalDataStreamCount();
|
||||
var indicesBehindDataStream = instance.getStats().indicesBehindDataStream();
|
||||
long failureStoreExplicitlyEnabledDataStreamCount = instance.getStats().failureStoreExplicitlyEnabledDataStreamCount();
|
||||
long failureStoreEffectivelyEnabledDataStreamCount = instance.getStats().failureStoreEffectivelyEnabledDataStreamCount();
|
||||
long failureStoreIndicesCount = instance.getStats().failureStoreIndicesCount();
|
||||
long failuresLifecycleExplicitlyEnabledDataStreamCount = instance.getStats().failuresLifecycleExplicitlyEnabledCount();
|
||||
long failuresLifecycleEffectivelyEnabledDataStreamCount = instance.getStats().failuresLifecycleEffectivelyEnabledCount();
|
||||
var dataRetentionStats = instance.getStats().failuresLifecycleDataRetentionStats();
|
||||
var effectiveRetentionStats = instance.getStats().failuresLifecycleEffectiveRetentionStats();
|
||||
var defaultRetention = instance.getStats().globalRetentionStats().get("default");
|
||||
var maxRetention = instance.getStats().globalRetentionStats().get("max");
|
||||
|
||||
switch (between(0, 10)) {
|
||||
case 0 -> totalDataStreamCount = randomValueOtherThan(totalDataStreamCount, ESTestCase::randomNonNegativeLong);
|
||||
case 1 -> indicesBehindDataStream = randomValueOtherThan(indicesBehindDataStream, ESTestCase::randomNonNegativeLong);
|
||||
case 2 -> failureStoreExplicitlyEnabledDataStreamCount = randomValueOtherThan(
|
||||
failureStoreExplicitlyEnabledDataStreamCount,
|
||||
ESTestCase::randomNonNegativeLong
|
||||
);
|
||||
case 3 -> failureStoreEffectivelyEnabledDataStreamCount = randomValueOtherThan(
|
||||
failureStoreEffectivelyEnabledDataStreamCount,
|
||||
ESTestCase::randomNonNegativeLong
|
||||
);
|
||||
case 4 -> failureStoreIndicesCount = randomValueOtherThan(failureStoreIndicesCount, ESTestCase::randomNonNegativeLong);
|
||||
case 5 -> failuresLifecycleExplicitlyEnabledDataStreamCount = randomValueOtherThan(
|
||||
failuresLifecycleExplicitlyEnabledDataStreamCount,
|
||||
ESTestCase::randomNonNegativeLong
|
||||
);
|
||||
case 6 -> failuresLifecycleEffectivelyEnabledDataStreamCount = randomValueOtherThan(
|
||||
failuresLifecycleEffectivelyEnabledDataStreamCount,
|
||||
ESTestCase::randomNonNegativeLong
|
||||
);
|
||||
case 7 -> dataRetentionStats = randomValueOtherThan(
|
||||
dataRetentionStats,
|
||||
DataStreamLifecycleFeatureSetUsageTests::generateRetentionStats
|
||||
);
|
||||
case 8 -> effectiveRetentionStats = randomValueOtherThan(
|
||||
effectiveRetentionStats,
|
||||
DataStreamLifecycleFeatureSetUsageTests::generateRetentionStats
|
||||
);
|
||||
case 9 -> maxRetention = randomValueOtherThan(maxRetention, DataStreamLifecycleFeatureSetUsageTests::generateGlobalRetention);
|
||||
case 10 -> defaultRetention = randomValueOtherThan(
|
||||
defaultRetention,
|
||||
DataStreamLifecycleFeatureSetUsageTests::generateGlobalRetention
|
||||
);
|
||||
default -> throw new IllegalStateException("Unexpected randomisation branch");
|
||||
}
|
||||
Map<String, DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats> map = new HashMap<>();
|
||||
if (defaultRetention != null) {
|
||||
map.put("default", defaultRetention);
|
||||
}
|
||||
if (maxRetention != null) {
|
||||
map.put("max", maxRetention);
|
||||
}
|
||||
return new DataStreamFeatureSetUsage(
|
||||
new DataStreamFeatureSetUsage.DataStreamStats(
|
||||
totalDataStreamCount,
|
||||
indicesBehindDataStream,
|
||||
failureStoreExplicitlyEnabledDataStreamCount,
|
||||
failureStoreEffectivelyEnabledDataStreamCount,
|
||||
failureStoreIndicesCount,
|
||||
failuresLifecycleExplicitlyEnabledDataStreamCount,
|
||||
failuresLifecycleEffectivelyEnabledDataStreamCount,
|
||||
dataRetentionStats,
|
||||
effectiveRetentionStats,
|
||||
map
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Writeable.Reader<DataStreamFeatureSetUsage> instanceReader() {
|
||||
return DataStreamFeatureSetUsage::new;
|
||||
}
|
||||
}
|
|
@ -45,11 +45,11 @@ public class DataStreamLifecycleFeatureSetUsageTests extends AbstractWireSeriali
|
|||
: DataStreamLifecycleFeatureSetUsage.DISABLED;
|
||||
}
|
||||
|
||||
private static DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats generateGlobalRetention() {
|
||||
static DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats generateGlobalRetention() {
|
||||
return new DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats(randomNonNegativeLong(), randomNonNegativeLong());
|
||||
}
|
||||
|
||||
private static DataStreamLifecycleFeatureSetUsage.RetentionStats generateRetentionStats() {
|
||||
static DataStreamLifecycleFeatureSetUsage.RetentionStats generateRetentionStats() {
|
||||
return randomBoolean()
|
||||
? DataStreamLifecycleFeatureSetUsage.RetentionStats.NO_DATA
|
||||
: new DataStreamLifecycleFeatureSetUsage.RetentionStats(
|
||||
|
|
|
@ -1,39 +0,0 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License
|
||||
* 2.0; you may not use this file except in compliance with the Elastic License
|
||||
* 2.0.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.xpack.core.searchablesnapshots;
|
||||
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.test.AbstractWireSerializingTestCase;
|
||||
import org.elasticsearch.xpack.core.datastreams.DataStreamFeatureSetUsage;
|
||||
|
||||
public class DataStreamFeatureSetUsageTests extends AbstractWireSerializingTestCase<DataStreamFeatureSetUsage> {
|
||||
|
||||
@Override
|
||||
protected DataStreamFeatureSetUsage createTestInstance() {
|
||||
return new DataStreamFeatureSetUsage(
|
||||
new DataStreamFeatureSetUsage.DataStreamStats(
|
||||
randomNonNegativeLong(),
|
||||
randomNonNegativeLong(),
|
||||
randomNonNegativeLong(),
|
||||
randomNonNegativeLong(),
|
||||
randomNonNegativeLong()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DataStreamFeatureSetUsage mutateInstance(DataStreamFeatureSetUsage instance) {
|
||||
return randomValueOtherThan(instance, this::createTestInstance);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Writeable.Reader<DataStreamFeatureSetUsage> instanceReader() {
|
||||
return DataStreamFeatureSetUsage::new;
|
||||
}
|
||||
|
||||
}
|
|
@ -25,7 +25,6 @@ import org.elasticsearch.cluster.metadata.ComposableIndexTemplate;
|
|||
import org.elasticsearch.cluster.metadata.DataStreamFailureStore;
|
||||
import org.elasticsearch.cluster.metadata.DataStreamOptions;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetadata;
|
||||
import org.elasticsearch.cluster.metadata.ResettableValue;
|
||||
import org.elasticsearch.cluster.metadata.Template;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.common.collect.Iterators;
|
||||
|
@ -1133,13 +1132,8 @@ public class EsqlActionIT extends AbstractEsqlIntegTestCase {
|
|||
}
|
||||
}"""))
|
||||
.dataStreamOptions(
|
||||
ResettableValue.create(
|
||||
new DataStreamOptions.Template(
|
||||
ResettableValue.create(new DataStreamFailureStore.Template(ResettableValue.create(true)))
|
||||
)
|
||||
)
|
||||
new DataStreamOptions.Template(DataStreamFailureStore.builder().enabled(true).buildTemplate())
|
||||
)
|
||||
.build()
|
||||
)
|
||||
.build()
|
||||
)
|
||||
|
|
|
@ -223,6 +223,7 @@ public class DataStreamLifecycleServiceRuntimeSecurityIT extends SecurityIntegTe
|
|||
var dataLifecycle = retention == null
|
||||
? DataStreamLifecycle.Template.DATA_DEFAULT
|
||||
: DataStreamLifecycle.dataLifecycleBuilder().enabled(true).dataRetention(retention).buildTemplate();
|
||||
var failuresLifecycle = retention == null ? null : DataStreamLifecycle.createFailuresLifecycleTemplate(true, retention);
|
||||
putComposableIndexTemplate("id1", """
|
||||
{
|
||||
"properties": {
|
||||
|
@ -233,7 +234,7 @@ public class DataStreamLifecycleServiceRuntimeSecurityIT extends SecurityIntegTe
|
|||
"type": "long"
|
||||
}
|
||||
}
|
||||
}""", List.of(dataStreamName + "*"), null, null, dataLifecycle);
|
||||
}""", List.of(dataStreamName + "*"), null, null, dataLifecycle, failuresLifecycle);
|
||||
CreateDataStreamAction.Request createDataStreamRequest = new CreateDataStreamAction.Request(
|
||||
TEST_REQUEST_TIMEOUT,
|
||||
TEST_REQUEST_TIMEOUT,
|
||||
|
@ -272,7 +273,8 @@ public class DataStreamLifecycleServiceRuntimeSecurityIT extends SecurityIntegTe
|
|||
List<String> patterns,
|
||||
@Nullable Settings settings,
|
||||
@Nullable Map<String, Object> metadata,
|
||||
@Nullable DataStreamLifecycle.Template dataLifecycle
|
||||
@Nullable DataStreamLifecycle.Template dataLifecycle,
|
||||
@Nullable DataStreamLifecycle.Template failuresLifecycle
|
||||
) throws IOException {
|
||||
TransportPutComposableIndexTemplateAction.Request request = new TransportPutComposableIndexTemplateAction.Request(id);
|
||||
request.indexTemplate(
|
||||
|
@ -283,7 +285,7 @@ public class DataStreamLifecycleServiceRuntimeSecurityIT extends SecurityIntegTe
|
|||
.settings(settings)
|
||||
.mappings(mappings == null ? null : CompressedXContent.fromJSON(mappings))
|
||||
.lifecycle(dataLifecycle)
|
||||
.dataStreamOptions(new DataStreamOptions.Template(new DataStreamFailureStore.Template(true)))
|
||||
.dataStreamOptions(new DataStreamOptions.Template(new DataStreamFailureStore.Template(true, failuresLifecycle)))
|
||||
)
|
||||
.metadata(metadata)
|
||||
.dataStreamTemplate(new ComposableIndexTemplate.DataStreamTemplate())
|
||||
|
@ -359,7 +361,14 @@ public class DataStreamLifecycleServiceRuntimeSecurityIT extends SecurityIntegTe
|
|||
}
|
||||
}"""))
|
||||
.lifecycle(DataStreamLifecycle.dataLifecycleBuilder().dataRetention(TimeValue.ZERO))
|
||||
.dataStreamOptions(new DataStreamOptions.Template(new DataStreamFailureStore.Template(true)))
|
||||
.dataStreamOptions(
|
||||
new DataStreamOptions.Template(
|
||||
new DataStreamFailureStore.Template(
|
||||
true,
|
||||
DataStreamLifecycle.failuresLifecycleBuilder().dataRetention(TimeValue.ZERO).buildTemplate()
|
||||
)
|
||||
)
|
||||
)
|
||||
)
|
||||
.dataStreamTemplate(new ComposableIndexTemplate.DataStreamTemplate())
|
||||
.build(),
|
||||
|
|
Loading…
Reference in New Issue