|
@@ -12,15 +12,20 @@ import org.apache.logging.log4j.LogManager;
|
|
|
import org.apache.logging.log4j.Logger;
|
|
|
import org.apache.lucene.util.SetOnce;
|
|
|
import org.elasticsearch.ElasticsearchException;
|
|
|
+import org.elasticsearch.ResourceAlreadyExistsException;
|
|
|
import org.elasticsearch.action.ActionListener;
|
|
|
import org.elasticsearch.action.ResultDeduplicator;
|
|
|
import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest;
|
|
|
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest;
|
|
|
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse;
|
|
|
+import org.elasticsearch.action.admin.indices.readonly.AddIndexBlockRequest;
|
|
|
+import org.elasticsearch.action.admin.indices.readonly.AddIndexBlockResponse;
|
|
|
import org.elasticsearch.action.admin.indices.rollover.RolloverConfiguration;
|
|
|
import org.elasticsearch.action.admin.indices.rollover.RolloverRequest;
|
|
|
import org.elasticsearch.action.admin.indices.rollover.RolloverResponse;
|
|
|
import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest;
|
|
|
+import org.elasticsearch.action.downsample.DownsampleAction;
|
|
|
+import org.elasticsearch.action.downsample.DownsampleConfig;
|
|
|
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
|
|
|
import org.elasticsearch.action.support.master.AcknowledgedResponse;
|
|
|
import org.elasticsearch.client.internal.Client;
|
|
@@ -29,6 +34,7 @@ import org.elasticsearch.cluster.ClusterState;
|
|
|
import org.elasticsearch.cluster.ClusterStateListener;
|
|
|
import org.elasticsearch.cluster.ClusterStateTaskListener;
|
|
|
import org.elasticsearch.cluster.SimpleBatchedExecutor;
|
|
|
+import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
|
|
import org.elasticsearch.cluster.metadata.DataStream;
|
|
|
import org.elasticsearch.cluster.metadata.DataStreamLifecycle;
|
|
|
import org.elasticsearch.cluster.metadata.IndexMetadata;
|
|
@@ -46,6 +52,8 @@ import org.elasticsearch.core.Nullable;
|
|
|
import org.elasticsearch.core.Strings;
|
|
|
import org.elasticsearch.core.TimeValue;
|
|
|
import org.elasticsearch.core.Tuple;
|
|
|
+import org.elasticsearch.datastreams.lifecycle.downsampling.ReplaceBackingWithDownsampleIndexExecutor;
|
|
|
+import org.elasticsearch.datastreams.lifecycle.downsampling.ReplaceSourceWithDownsampleIndexTask;
|
|
|
import org.elasticsearch.gateway.GatewayService;
|
|
|
import org.elasticsearch.index.Index;
|
|
|
import org.elasticsearch.index.IndexNotFoundException;
|
|
@@ -56,6 +64,7 @@ import org.elasticsearch.transport.TransportRequest;
|
|
|
|
|
|
import java.io.Closeable;
|
|
|
import java.time.Clock;
|
|
|
+import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.HashSet;
|
|
@@ -63,10 +72,19 @@ import java.util.List;
|
|
|
import java.util.Locale;
|
|
|
import java.util.Map;
|
|
|
import java.util.Objects;
|
|
|
+import java.util.Optional;
|
|
|
import java.util.Set;
|
|
|
+import java.util.function.Function;
|
|
|
import java.util.function.LongSupplier;
|
|
|
import java.util.stream.Collectors;
|
|
|
|
|
|
+import static org.elasticsearch.cluster.metadata.IndexMetadata.APIBlock.WRITE;
|
|
|
+import static org.elasticsearch.cluster.metadata.IndexMetadata.DownsampleTaskStatus.STARTED;
|
|
|
+import static org.elasticsearch.cluster.metadata.IndexMetadata.DownsampleTaskStatus.SUCCESS;
|
|
|
+import static org.elasticsearch.cluster.metadata.IndexMetadata.INDEX_DOWNSAMPLE_STATUS;
|
|
|
+import static org.elasticsearch.datastreams.DataStreamsPlugin.LIFECYCLE_CUSTOM_INDEX_METADATA_KEY;
|
|
|
+import static org.elasticsearch.datastreams.lifecycle.downsampling.ReplaceSourceWithDownsampleIndexTask.REPLACEMENT_SOURCE_INDEX;
|
|
|
+
|
|
|
/**
|
|
|
* This service will implement the needed actions (e.g. rollover, retention) to manage the data streams with a data stream lifecycle
|
|
|
* configured. It runs on the master node and it schedules a job according to the configured
|
|
@@ -100,6 +118,7 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
|
|
|
Setting.Property.Dynamic,
|
|
|
Setting.Property.NodeScope
|
|
|
);
|
|
|
+ public static final String DOWNSAMPLED_INDEX_PREFIX = "downsample-";
|
|
|
|
|
|
private static final Logger logger = LogManager.getLogger(DataStreamLifecycleService.class);
|
|
|
/**
|
|
@@ -109,13 +128,13 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
|
|
|
/*
|
|
|
* This is the key for data stream lifecycle related custom index metadata.
|
|
|
*/
|
|
|
- static final String LIFECYCLE_CUSTOM_INDEX_METADATA_KEY = "data_stream_lifecycle";
|
|
|
static final String FORCE_MERGE_COMPLETED_TIMESTAMP_METADATA_KEY = "force_merge_completed_timestamp";
|
|
|
private final Settings settings;
|
|
|
private final Client client;
|
|
|
private final ClusterService clusterService;
|
|
|
private final ThreadPool threadPool;
|
|
|
final ResultDeduplicator<TransportRequest, Void> transportActionsDeduplicator;
|
|
|
+ final ResultDeduplicator<ClusterStateTaskListener, Void> clusterStateChangesDeduplicator;
|
|
|
private final LongSupplier nowSupplier;
|
|
|
private final Clock clock;
|
|
|
private final DataStreamLifecycleErrorStore errorStore;
|
|
@@ -125,6 +144,7 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
|
|
|
private SchedulerEngine.Job scheduledJob;
|
|
|
private final SetOnce<SchedulerEngine> scheduler = new SetOnce<>();
|
|
|
private final MasterServiceTaskQueue<UpdateForceMergeCompleteTask> forceMergeClusterStateUpdateTaskQueue;
|
|
|
+ private final MasterServiceTaskQueue<ReplaceSourceWithDownsampleIndexTask> swapSourceWithDownsampleIndexQueue;
|
|
|
private volatile ByteSizeValue targetMergePolicyFloorSegment;
|
|
|
private volatile int targetMergePolicyFactor;
|
|
|
|
|
@@ -157,6 +177,7 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
|
|
|
this.clock = clock;
|
|
|
this.threadPool = threadPool;
|
|
|
this.transportActionsDeduplicator = new ResultDeduplicator<>(threadPool.getThreadContext());
|
|
|
+ this.clusterStateChangesDeduplicator = new ResultDeduplicator<>(threadPool.getThreadContext());
|
|
|
this.nowSupplier = nowSupplier;
|
|
|
this.errorStore = errorStore;
|
|
|
this.scheduledJob = null;
|
|
@@ -170,6 +191,11 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
|
|
|
Priority.LOW,
|
|
|
FORCE_MERGE_STATE_UPDATE_TASK_EXECUTOR
|
|
|
);
|
|
|
+ this.swapSourceWithDownsampleIndexQueue = clusterService.createTaskQueue(
|
|
|
+ "data-stream-lifecycle-swap-source-with-downsample",
|
|
|
+ Priority.NORMAL,
|
|
|
+ new ReplaceBackingWithDownsampleIndexExecutor(client)
|
|
|
+ );
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -240,6 +266,8 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
|
|
|
*/
|
|
|
// default visibility for testing purposes
|
|
|
void run(ClusterState state) {
|
|
|
+ int affectedIndices = 0;
|
|
|
+ int affectedDataStreams = 0;
|
|
|
for (DataStream dataStream : state.metadata().dataStreams().values()) {
|
|
|
clearErrorStoreForUnmanagedIndices(dataStream);
|
|
|
if (dataStream.getLifecycle() == null) {
|
|
@@ -285,21 +313,17 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
|
|
|
);
|
|
|
}
|
|
|
|
|
|
+ // the following indices should not be considered for the remainder of this service run:
|
|
|
+ // 1) the write index as it's still getting writes and we'll have to roll it over when the conditions are met
|
|
|
+ // 2) we exclude any indices that we're in the process of deleting because they'll be gone soon anyway
|
|
|
+ Set<Index> indicesToExcludeForRemainingRun = new HashSet<>();
|
|
|
+ indicesToExcludeForRemainingRun.add(currentRunWriteIndex);
|
|
|
+ indicesToExcludeForRemainingRun.addAll(indicesBeingRemoved);
|
|
|
+
|
|
|
try {
|
|
|
- /*
|
|
|
- * When considering indices for force merge, we want to exclude several indices: (1) We exclude the current write index
|
|
|
- * because obviously it is still likely to get writes, (2) we exclude the most recent previous write index because since
|
|
|
- * we just switched over it might still be getting some writes, and (3) we exclude any indices that we're in the process
|
|
|
- * of deleting because they'll be gone soon anyway.
|
|
|
- */
|
|
|
- Set<Index> indicesToExclude = new HashSet<>();
|
|
|
- indicesToExclude.add(currentRunWriteIndex);
|
|
|
- indicesToExclude.addAll(indicesBeingRemoved);
|
|
|
- List<Index> potentialForceMergeIndices = dataStream.getIndices()
|
|
|
- .stream()
|
|
|
- .filter(index -> indicesToExclude.contains(index) == false)
|
|
|
- .toList();
|
|
|
- maybeExecuteForceMerge(state, dataStream, potentialForceMergeIndices);
|
|
|
+ indicesToExcludeForRemainingRun.addAll(
|
|
|
+ maybeExecuteForceMerge(state, getTargetIndices(dataStream, indicesToExcludeForRemainingRun, state.metadata()::index))
|
|
|
+ );
|
|
|
} catch (Exception e) {
|
|
|
logger.error(
|
|
|
() -> String.format(
|
|
@@ -310,7 +334,322 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
|
|
|
e
|
|
|
);
|
|
|
}
|
|
|
+
|
|
|
+ try {
|
|
|
+ indicesToExcludeForRemainingRun.addAll(
|
|
|
+ maybeExecuteDownsampling(
|
|
|
+ state,
|
|
|
+ dataStream,
|
|
|
+ getTargetIndices(dataStream, indicesToExcludeForRemainingRun, state.metadata()::index)
|
|
|
+ )
|
|
|
+ );
|
|
|
+ } catch (Exception e) {
|
|
|
+ logger.error(
|
|
|
+ () -> String.format(
|
|
|
+ Locale.ROOT,
|
|
|
+ "Data stream lifecycle failed to execute downsampling for data stream [%s]",
|
|
|
+ dataStream.getName()
|
|
|
+ ),
|
|
|
+ e
|
|
|
+ );
|
|
|
+ }
|
|
|
+
|
|
|
+ affectedIndices += indicesToExcludeForRemainingRun.size();
|
|
|
+ affectedDataStreams++;
|
|
|
+ }
|
|
|
+ logger.trace(
|
|
|
+ "Data stream lifecycle service performed operations on [{}] indices, part of [{}] data streams",
|
|
|
+ affectedIndices,
|
|
|
+ affectedDataStreams
|
|
|
+ );
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Data stream lifecycle supports configuring multiple rounds of downsampling for each managed index. When attempting to execute
|
|
|
+ * downsampling we iterate through the ordered rounds of downsampling that match an index (ordered ascending according to the `after`
|
|
|
+ * configuration) and try to figure out:
|
|
|
+ * - if we started downsampling for an earlier round and is in progress, in which case we need to wait for it to complete
|
|
|
+ * - if we started downsampling for an earlier round and it's finished but the downsampling index is not part of the data stream, in
|
|
|
+ * which case we need to replace the backing index with the downsampling index and delete the backing index
|
|
|
+ * - if we don't have any early rounds started or to add to the data stream, start downsampling the last matching round
|
|
|
+ *
|
|
|
+ * Note that the first time an index has a matching downsampling round we first mark it as read-only.
|
|
|
+ *
|
|
|
+ * Returns a set of indices that now have in-flight operations triggered by downsampling (it could be marking them as read-only,
|
|
|
+ * replacing an index in the data stream, deleting a source index, or downsampling itself) so these indices can be skipped in case
|
|
|
+ * there are other operations to be executed by the data stream lifecycle after downsampling.
|
|
|
+ */
|
|
|
+ Set<Index> maybeExecuteDownsampling(ClusterState state, DataStream dataStream, List<Index> targetIndices) {
|
|
|
+ Set<Index> affectedIndices = new HashSet<>();
|
|
|
+ Metadata metadata = state.metadata();
|
|
|
+ for (Index index : targetIndices) {
|
|
|
+ IndexMetadata backingIndexMeta = metadata.index(index);
|
|
|
+ assert backingIndexMeta != null : "the data stream backing indices must exist";
|
|
|
+ List<DataStreamLifecycle.Downsampling.Round> downsamplingRounds = dataStream.getDownsamplingRoundsFor(
|
|
|
+ index,
|
|
|
+ metadata::index,
|
|
|
+ nowSupplier
|
|
|
+ );
|
|
|
+ if (downsamplingRounds.isEmpty()) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+
|
|
|
+ String indexName = index.getName();
|
|
|
+ IndexMetadata.DownsampleTaskStatus backingIndexDownsamplingStatus = INDEX_DOWNSAMPLE_STATUS.get(backingIndexMeta.getSettings());
|
|
|
+ String backingIndexDownsamplingSource = IndexMetadata.INDEX_DOWNSAMPLE_SOURCE_NAME.get(backingIndexMeta.getSettings());
|
|
|
+
|
|
|
+ // if the current index is not a downsample we want to mark the index as read-only before proceeding with downsampling
|
|
|
+ if (org.elasticsearch.common.Strings.hasText(backingIndexDownsamplingSource) == false
|
|
|
+ && state.blocks().indexBlocked(ClusterBlockLevel.WRITE, indexName) == false) {
|
|
|
+ affectedIndices.add(index);
|
|
|
+ addIndexBlockOnce(indexName);
|
|
|
+ } else if (org.elasticsearch.common.Strings.hasText(backingIndexDownsamplingSource)
|
|
|
+ && backingIndexDownsamplingStatus.equals(SUCCESS)) {
|
|
|
+ // if the backing index is a downsample index itself, let's check if its source index still exists as we must delete it
|
|
|
+ Map<String, String> lifecycleMetadata = backingIndexMeta.getCustomData(LIFECYCLE_CUSTOM_INDEX_METADATA_KEY);
|
|
|
+
|
|
|
+ // TODO document that we don't handle downsample indices that were added to the data stream manually (because we
|
|
|
+ // TODO currently can't reliably identify the source index to delete when multiple rounds of donwsampling are
|
|
|
+ // TODO involved unless DSL stores the needed metadata in the index metadata)
|
|
|
+ if (lifecycleMetadata != null && lifecycleMetadata.containsKey(REPLACEMENT_SOURCE_INDEX)) {
|
|
|
+ String actualDownsamplingSource = lifecycleMetadata.get(REPLACEMENT_SOURCE_INDEX);
|
|
|
+ IndexMetadata downsampleSourceIndex = metadata.index(actualDownsamplingSource);
|
|
|
+ if (downsampleSourceIndex != null) {
|
|
|
+ // we mark the backing index as affected as we don't want subsequent operations that might change its state to
|
|
|
+ // be performed, as we might lose the way to identify that we must delete its replacement source index
|
|
|
+ affectedIndices.add(index);
|
|
|
+ // delete downsampling source index (that's not part of the data stream anymore) before doing any more
|
|
|
+ // downsampling
|
|
|
+ deleteIndexOnce(backingIndexDownsamplingSource, "replacement with its downsampled index in the data stream");
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ logger.trace(
|
|
|
+ "Data stream lifecycle encountered managed index [{}] as part of data stream [{}] which was "
|
|
|
+ + "downsampled from source [{} ]. This index was manually downsampled but data stream lifecycle service "
|
|
|
+ + "only supports downsampled indices through the data stream lifecycle. This index will be ignored from "
|
|
|
+ + "lifecycle donwsampling",
|
|
|
+ indexName,
|
|
|
+ dataStream,
|
|
|
+ backingIndexDownsamplingSource
|
|
|
+ );
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ if (affectedIndices.contains(index) == false) {
|
|
|
+ // we're not performing any operation for this index which means that it:
|
|
|
+ // - has matching downsample rounds
|
|
|
+ // - is read-only
|
|
|
+ // So let's wait for an in-progress downsampling operation to succeed or trigger the last matching round
|
|
|
+ affectedIndices.addAll(waitForInProgressOrTriggerDownsampling(dataStream, backingIndexMeta, downsamplingRounds, metadata));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ return affectedIndices;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Iterate over the matching downsampling rounds for the backing index (if any) and either wait for an early round to complete,
|
|
|
+ * add an early completed downsampling round to the data stream, or otherwise trigger the last matching downsampling round.
|
|
|
+ *
|
|
|
+ * Returns the indices for which we triggered an action/operation.
|
|
|
+ */
|
|
|
+ private Set<Index> waitForInProgressOrTriggerDownsampling(
|
|
|
+ DataStream dataStream,
|
|
|
+ IndexMetadata backingIndex,
|
|
|
+ List<DataStreamLifecycle.Downsampling.Round> downsamplingRounds,
|
|
|
+ Metadata metadata
|
|
|
+ ) {
|
|
|
+ assert dataStream.getIndices().contains(backingIndex.getIndex())
|
|
|
+ : "the provided backing index must be part of data stream:" + dataStream.getName();
|
|
|
+ assert downsamplingRounds.isEmpty() == false : "the index should be managed and have matching downsampling rounds";
|
|
|
+ Set<Index> affectedIndices = new HashSet<>();
|
|
|
+ DataStreamLifecycle.Downsampling.Round lastRound = downsamplingRounds.get(downsamplingRounds.size() - 1);
|
|
|
+
|
|
|
+ Index index = backingIndex.getIndex();
|
|
|
+ String indexName = index.getName();
|
|
|
+ for (DataStreamLifecycle.Downsampling.Round round : downsamplingRounds) {
|
|
|
+ // the downsample index name for each round is deterministic
|
|
|
+ String downsampleIndexName = DownsampleConfig.generateDownsampleIndexName(
|
|
|
+ DOWNSAMPLED_INDEX_PREFIX,
|
|
|
+ backingIndex,
|
|
|
+ round.config().getFixedInterval()
|
|
|
+ );
|
|
|
+ IndexMetadata targetDownsampleIndexMeta = metadata.index(downsampleIndexName);
|
|
|
+ boolean targetDownsampleIndexExists = targetDownsampleIndexMeta != null;
|
|
|
+
|
|
|
+ if (targetDownsampleIndexExists) {
|
|
|
+ Set<Index> downsamplingNotComplete = evaluateDownsampleStatus(
|
|
|
+ dataStream,
|
|
|
+ INDEX_DOWNSAMPLE_STATUS.get(targetDownsampleIndexMeta.getSettings()),
|
|
|
+ round,
|
|
|
+ lastRound,
|
|
|
+ index,
|
|
|
+ targetDownsampleIndexMeta.getIndex()
|
|
|
+ );
|
|
|
+ if (downsamplingNotComplete.isEmpty() == false) {
|
|
|
+ affectedIndices.addAll(downsamplingNotComplete);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ if (round.equals(lastRound)) {
|
|
|
+ // no maintenance needed for previously started downsampling actions and we are on the last matching round so it's time
|
|
|
+ // to kick off downsampling
|
|
|
+ affectedIndices.add(index);
|
|
|
+ downsampleIndexOnce(round, indexName, downsampleIndexName);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
+ return affectedIndices;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Issues a request downsample the source index to the downsample index for the specified round.
|
|
|
+ */
|
|
|
+ private void downsampleIndexOnce(DataStreamLifecycle.Downsampling.Round round, String sourceIndex, String downsampleIndexName) {
|
|
|
+ DownsampleAction.Request request = new DownsampleAction.Request(sourceIndex, downsampleIndexName, null, round.config());
|
|
|
+ transportActionsDeduplicator.executeOnce(
|
|
|
+ request,
|
|
|
+ new ErrorRecordingActionListener(sourceIndex, errorStore),
|
|
|
+ (req, reqListener) -> downsampleIndex(request, reqListener)
|
|
|
+ );
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Checks the status of the downsampling operations for the provided backing index and its corresponding downsample index.
|
|
|
+ * Depending on the status, we'll either error (if it's UNKNOWN and we've reached the last round), wait for it to complete (if it's
|
|
|
+ * STARTED), or replace the backing index with the downsample index in the data stream (if the status is SUCCESS).
|
|
|
+ */
|
|
|
+ private Set<Index> evaluateDownsampleStatus(
|
|
|
+ DataStream dataStream,
|
|
|
+ IndexMetadata.DownsampleTaskStatus downsampleStatus,
|
|
|
+ DataStreamLifecycle.Downsampling.Round currentRound,
|
|
|
+ DataStreamLifecycle.Downsampling.Round lastRound,
|
|
|
+ Index backingIndex,
|
|
|
+ Index downsampleIndex
|
|
|
+ ) {
|
|
|
+ Set<Index> affectedIndices = new HashSet<>();
|
|
|
+ String indexName = backingIndex.getName();
|
|
|
+ String downsampleIndexName = downsampleIndex.getName();
|
|
|
+ return switch (downsampleStatus) {
|
|
|
+ case UNKNOWN -> {
|
|
|
+ if (currentRound.equals(lastRound)) {
|
|
|
+ // target downsampling index exists and is not a downsampling index (name clash?)
|
|
|
+ // we fail now but perhaps we should just randomise the name?
|
|
|
+ String previousError = errorStore.getError(indexName);
|
|
|
+
|
|
|
+ errorStore.recordError(indexName, new ResourceAlreadyExistsException(downsampleIndexName));
|
|
|
+ // To avoid spamming our logs, we only want to log the error once.
|
|
|
+ if (previousError == null || previousError.equals(errorStore.getError(indexName)) == false) {
|
|
|
+ logger.error(
|
|
|
+ "Data stream lifecycle service is unable to downsample backing index [{}] for data stream [{}] and "
|
|
|
+ + "donwsampling round [{}] because the target downsample index [{}] already exists",
|
|
|
+ indexName,
|
|
|
+ dataStream.getName(),
|
|
|
+ currentRound,
|
|
|
+ downsampleIndexName
|
|
|
+ );
|
|
|
+ }
|
|
|
+ }
|
|
|
+ yield affectedIndices;
|
|
|
+ }
|
|
|
+ case STARTED -> {
|
|
|
+ // we'll wait for this round to complete
|
|
|
+ // TODO add support for cancelling a current in-progress operation if another, later, round matches
|
|
|
+ logger.trace(
|
|
|
+ "Data stream lifecycle service waits for index [{}] to be downsampled. Current status is [{}] and the "
|
|
|
+ + "downsample index name is [{}]",
|
|
|
+ indexName,
|
|
|
+ STARTED,
|
|
|
+ downsampleIndexName
|
|
|
+ );
|
|
|
+ // this request here might seem weird, but hear me out:
|
|
|
+ // if we triggered a downsample operation, and then had a master failover (so DSL starts from scratch)
|
|
|
+ // we can't really find out if the downsampling persistent task failed (if it was successful, no worries, the next case
|
|
|
+ // SUCCESS branch will catch it and we will cruise forward)
|
|
|
+ // if the downsampling persistent task failed, we will find out only via re-issuing the downsample request (and we will
|
|
|
+ // continue to re-issue the request until we get SUCCESS)
|
|
|
+
|
|
|
+ // NOTE that the downsample request is made through the deduplicator so it will only really be executed if
|
|
|
+ // there isn't one already in-flight. This can happen if a previous request timed-out, failed, or there was a
|
|
|
+ // master failover and data stream lifecycle needed to restart
|
|
|
+ downsampleIndexOnce(currentRound, indexName, downsampleIndexName);
|
|
|
+ affectedIndices.add(backingIndex);
|
|
|
+ yield affectedIndices;
|
|
|
+ }
|
|
|
+ case SUCCESS -> {
|
|
|
+ if (dataStream.getIndices().contains(downsampleIndex) == false) {
|
|
|
+ // at this point the source index is part of the data stream and the downsample index is complete but not
|
|
|
+ // part of the data stream. we need to replace the source index with the downsample index in the data stream
|
|
|
+ affectedIndices.add(backingIndex);
|
|
|
+ replaceBackingIndexWithDownsampleIndexOnce(dataStream, indexName, downsampleIndexName);
|
|
|
+ }
|
|
|
+ yield affectedIndices;
|
|
|
+ }
|
|
|
+ };
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Issues a request to replace the backing index with the downsample index through the cluster state changes deduplicator.
|
|
|
+ */
|
|
|
+ private void replaceBackingIndexWithDownsampleIndexOnce(DataStream dataStream, String backingIndexName, String downsampleIndexName) {
|
|
|
+ clusterStateChangesDeduplicator.executeOnce(
|
|
|
+ new ReplaceSourceWithDownsampleIndexTask(dataStream.getName(), backingIndexName, downsampleIndexName, null),
|
|
|
+ new ErrorRecordingActionListener(backingIndexName, errorStore),
|
|
|
+ (req, reqListener) -> {
|
|
|
+ logger.trace(
|
|
|
+ "Data stream lifecycle issues request to replace index [{}] with index [{}] in data stream [{}]",
|
|
|
+ backingIndexName,
|
|
|
+ downsampleIndexName,
|
|
|
+ dataStream
|
|
|
+ );
|
|
|
+ swapSourceWithDownsampleIndexQueue.submitTask(
|
|
|
+ "data-stream-lifecycle-replace-source[" + backingIndexName + "]-with-[" + downsampleIndexName + "]",
|
|
|
+ new ReplaceSourceWithDownsampleIndexTask(dataStream.getName(), backingIndexName, downsampleIndexName, reqListener),
|
|
|
+ null
|
|
|
+ );
|
|
|
+ }
|
|
|
+ );
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Issues a request to delete the provided index through the transport action deduplicator.
|
|
|
+ */
|
|
|
+ private void deleteIndexOnce(String indexName, String reason) {
|
|
|
+ DeleteIndexRequest deleteIndexRequest = new DeleteIndexRequest(indexName).masterNodeTimeout(TimeValue.MAX_VALUE);
|
|
|
+ transportActionsDeduplicator.executeOnce(
|
|
|
+ deleteIndexRequest,
|
|
|
+ new ErrorRecordingActionListener(indexName, errorStore),
|
|
|
+ (req, reqListener) -> deleteIndex(deleteIndexRequest, reason, reqListener)
|
|
|
+ );
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Issues a request to add a WRITE index block for the provided index through the transport action deduplicator.
|
|
|
+ */
|
|
|
+ private void addIndexBlockOnce(String indexName) {
|
|
|
+ AddIndexBlockRequest addIndexBlockRequest = new AddIndexBlockRequest(WRITE, indexName).masterNodeTimeout(TimeValue.MAX_VALUE);
|
|
|
+ transportActionsDeduplicator.executeOnce(
|
|
|
+ addIndexBlockRequest,
|
|
|
+ new ErrorRecordingActionListener(indexName, errorStore),
|
|
|
+ (req, reqListener) -> addIndexBlock(addIndexBlockRequest, reqListener)
|
|
|
+ );
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Returns the data stream lifecycle managed indices that are not part of the set of indices to exclude.
|
|
|
+ */
|
|
|
+ private static List<Index> getTargetIndices(
|
|
|
+ DataStream dataStream,
|
|
|
+ Set<Index> indicesToExcludeForRemainingRun,
|
|
|
+ Function<String, IndexMetadata> indexMetadataSupplier
|
|
|
+ ) {
|
|
|
+ return dataStream.getIndices()
|
|
|
+ .stream()
|
|
|
+ .filter(
|
|
|
+ index -> dataStream.isIndexManagedByDataStreamLifecycle(index, indexMetadataSupplier)
|
|
|
+ && indicesToExcludeForRemainingRun.contains(index) == false
|
|
|
+ )
|
|
|
+ .toList();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -367,14 +706,7 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
|
|
|
// there's an opportunity here to batch the delete requests (i.e. delete 100 indices / request)
|
|
|
// let's start simple and reevaluate
|
|
|
String indexName = backingIndex.getIndex().getName();
|
|
|
- DeleteIndexRequest deleteRequest = new DeleteIndexRequest(indexName).masterNodeTimeout(TimeValue.MAX_VALUE);
|
|
|
-
|
|
|
- // time to delete the index
|
|
|
- transportActionsDeduplicator.executeOnce(
|
|
|
- deleteRequest,
|
|
|
- new ErrorRecordingActionListener(indexName, errorStore),
|
|
|
- (req, reqListener) -> deleteIndex(deleteRequest, retention, reqListener)
|
|
|
- );
|
|
|
+ deleteIndexOnce(indexName, "the lapsed [" + retention + "] retention period");
|
|
|
}
|
|
|
}
|
|
|
return indicesToBeRemoved;
|
|
@@ -384,49 +716,51 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
|
|
|
* This method force merges the given indices in the datastream. It writes a timestamp in the cluster state upon completion of the
|
|
|
* force merge.
|
|
|
*/
|
|
|
- private void maybeExecuteForceMerge(ClusterState state, DataStream dataStream, List<Index> indices) {
|
|
|
+ private Set<Index> maybeExecuteForceMerge(ClusterState state, List<Index> indices) {
|
|
|
Metadata metadata = state.metadata();
|
|
|
+ Set<Index> affectedIndices = new HashSet<>();
|
|
|
for (Index index : indices) {
|
|
|
- if (dataStream.isIndexManagedByDataStreamLifecycle(index, state.metadata()::index)) {
|
|
|
- IndexMetadata backingIndex = metadata.index(index);
|
|
|
- assert backingIndex != null : "the data stream backing indices must exist";
|
|
|
- String indexName = index.getName();
|
|
|
- boolean alreadyForceMerged = isForceMergeComplete(backingIndex);
|
|
|
- if (alreadyForceMerged) {
|
|
|
- logger.trace("Already force merged {}", indexName);
|
|
|
- continue;
|
|
|
- }
|
|
|
+ IndexMetadata backingIndex = metadata.index(index);
|
|
|
+ assert backingIndex != null : "the data stream backing indices must exist";
|
|
|
+ String indexName = index.getName();
|
|
|
+ boolean alreadyForceMerged = isForceMergeComplete(backingIndex);
|
|
|
+ if (alreadyForceMerged) {
|
|
|
+ logger.trace("Already force merged {}", indexName);
|
|
|
+ continue;
|
|
|
+ }
|
|
|
|
|
|
- ByteSizeValue configuredFloorSegmentMerge = MergePolicyConfig.INDEX_MERGE_POLICY_FLOOR_SEGMENT_SETTING.get(
|
|
|
- backingIndex.getSettings()
|
|
|
+ ByteSizeValue configuredFloorSegmentMerge = MergePolicyConfig.INDEX_MERGE_POLICY_FLOOR_SEGMENT_SETTING.get(
|
|
|
+ backingIndex.getSettings()
|
|
|
+ );
|
|
|
+ Integer configuredMergeFactor = MergePolicyConfig.INDEX_MERGE_POLICY_MERGE_FACTOR_SETTING.get(backingIndex.getSettings());
|
|
|
+ if ((configuredFloorSegmentMerge == null || configuredFloorSegmentMerge.equals(targetMergePolicyFloorSegment) == false)
|
|
|
+ || (configuredMergeFactor == null || configuredMergeFactor.equals(targetMergePolicyFactor) == false)) {
|
|
|
+ UpdateSettingsRequest updateMergePolicySettingsRequest = new UpdateSettingsRequest();
|
|
|
+ updateMergePolicySettingsRequest.indices(indexName);
|
|
|
+ updateMergePolicySettingsRequest.settings(
|
|
|
+ Settings.builder()
|
|
|
+ .put(MergePolicyConfig.INDEX_MERGE_POLICY_FLOOR_SEGMENT_SETTING.getKey(), targetMergePolicyFloorSegment)
|
|
|
+ .put(MergePolicyConfig.INDEX_MERGE_POLICY_MERGE_FACTOR_SETTING.getKey(), targetMergePolicyFactor)
|
|
|
+ );
|
|
|
+ updateMergePolicySettingsRequest.masterNodeTimeout(TimeValue.MAX_VALUE);
|
|
|
+ affectedIndices.add(index);
|
|
|
+ transportActionsDeduplicator.executeOnce(
|
|
|
+ updateMergePolicySettingsRequest,
|
|
|
+ new ErrorRecordingActionListener(indexName, errorStore),
|
|
|
+ (req, reqListener) -> updateIndexSetting(updateMergePolicySettingsRequest, reqListener)
|
|
|
+ );
|
|
|
+ } else {
|
|
|
+ affectedIndices.add(index);
|
|
|
+ ForceMergeRequest forceMergeRequest = new ForceMergeRequest(indexName);
|
|
|
+ // time to force merge the index
|
|
|
+ transportActionsDeduplicator.executeOnce(
|
|
|
+ new ForceMergeRequestWrapper(forceMergeRequest),
|
|
|
+ new ErrorRecordingActionListener(indexName, errorStore),
|
|
|
+ (req, reqListener) -> forceMergeIndex(forceMergeRequest, reqListener)
|
|
|
);
|
|
|
- Integer configuredMergeFactor = MergePolicyConfig.INDEX_MERGE_POLICY_MERGE_FACTOR_SETTING.get(backingIndex.getSettings());
|
|
|
- if ((configuredFloorSegmentMerge == null || configuredFloorSegmentMerge.equals(targetMergePolicyFloorSegment) == false)
|
|
|
- || (configuredMergeFactor == null || configuredMergeFactor.equals(targetMergePolicyFactor) == false)) {
|
|
|
- UpdateSettingsRequest updateMergePolicySettingsRequest = new UpdateSettingsRequest();
|
|
|
- updateMergePolicySettingsRequest.indices(indexName);
|
|
|
- updateMergePolicySettingsRequest.settings(
|
|
|
- Settings.builder()
|
|
|
- .put(MergePolicyConfig.INDEX_MERGE_POLICY_FLOOR_SEGMENT_SETTING.getKey(), targetMergePolicyFloorSegment)
|
|
|
- .put(MergePolicyConfig.INDEX_MERGE_POLICY_MERGE_FACTOR_SETTING.getKey(), targetMergePolicyFactor)
|
|
|
- );
|
|
|
- updateMergePolicySettingsRequest.masterNodeTimeout(TimeValue.MAX_VALUE);
|
|
|
- transportActionsDeduplicator.executeOnce(
|
|
|
- updateMergePolicySettingsRequest,
|
|
|
- new ErrorRecordingActionListener(indexName, errorStore),
|
|
|
- (req, reqListener) -> updateIndexSetting(updateMergePolicySettingsRequest, reqListener)
|
|
|
- );
|
|
|
- } else {
|
|
|
- ForceMergeRequest forceMergeRequest = new ForceMergeRequest(indexName);
|
|
|
- // time to force merge the index
|
|
|
- transportActionsDeduplicator.executeOnce(
|
|
|
- new ForceMergeRequestWrapper(forceMergeRequest),
|
|
|
- new ErrorRecordingActionListener(indexName, errorStore),
|
|
|
- (req, reqListener) -> forceMergeIndex(forceMergeRequest, reqListener)
|
|
|
- );
|
|
|
- }
|
|
|
}
|
|
|
}
|
|
|
+ return affectedIndices;
|
|
|
}
|
|
|
|
|
|
private void rolloverDataStream(String writeIndexName, RolloverRequest rolloverRequest, ActionListener<Void> listener) {
|
|
@@ -512,7 +846,97 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
|
|
|
});
|
|
|
}
|
|
|
|
|
|
- private void deleteIndex(DeleteIndexRequest deleteIndexRequest, TimeValue retention, ActionListener<Void> listener) {
|
|
|
+ private void addIndexBlock(AddIndexBlockRequest addIndexBlockRequest, ActionListener<Void> listener) {
|
|
|
+ assert addIndexBlockRequest.indices() != null && addIndexBlockRequest.indices().length == 1
|
|
|
+ : "Data stream lifecycle service updates the index block for one index at a time";
|
|
|
+ // "saving" the index name here so we don't capture the entire request
|
|
|
+ String targetIndex = addIndexBlockRequest.indices()[0];
|
|
|
+ logger.trace(
|
|
|
+ "Data stream lifecycle service issues request to add block [{}] for index [{}]",
|
|
|
+ addIndexBlockRequest.getBlock(),
|
|
|
+ targetIndex
|
|
|
+ );
|
|
|
+ client.admin().indices().addBlock(addIndexBlockRequest, new ActionListener<>() {
|
|
|
+ @Override
|
|
|
+ public void onResponse(AddIndexBlockResponse addIndexBlockResponse) {
|
|
|
+ if (addIndexBlockResponse.isAcknowledged()) {
|
|
|
+ logger.info(
|
|
|
+ "Data stream lifecycle service successfully added block [{}] for index index [{}]",
|
|
|
+ addIndexBlockRequest.getBlock(),
|
|
|
+ targetIndex
|
|
|
+ );
|
|
|
+ listener.onResponse(null);
|
|
|
+ } else {
|
|
|
+ Optional<AddIndexBlockResponse.AddBlockResult> resultForTargetIndex = addIndexBlockResponse.getIndices()
|
|
|
+ .stream()
|
|
|
+ .filter(blockResult -> blockResult.getIndex().getName().equals(targetIndex))
|
|
|
+ .findAny();
|
|
|
+ if (resultForTargetIndex.isEmpty()) {
|
|
|
+ // blimey
|
|
|
+ // this is weird, we don't have a result for our index, so let's treat this as a success and the next DSL run will
|
|
|
+ // check if we need to retry adding the block for this index
|
|
|
+ logger.trace(
|
|
|
+ "Data stream lifecycle service received an unacknowledged response when attempting to add the "
|
|
|
+ + "read-only block to index [{}], but the response didn't contain an explicit result for the index.",
|
|
|
+ targetIndex
|
|
|
+ );
|
|
|
+ logger.error(
|
|
|
+ "Data stream lifecycle service request to mark index [{}] as readonly was not acknowledged",
|
|
|
+ targetIndex
|
|
|
+ );
|
|
|
+ listener.onFailure(
|
|
|
+ new ElasticsearchException("request to mark index [" + targetIndex + "] as read-only was not acknowledged")
|
|
|
+ );
|
|
|
+ } else if (resultForTargetIndex.get().hasFailures()) {
|
|
|
+ AddIndexBlockResponse.AddBlockResult blockResult = resultForTargetIndex.get();
|
|
|
+ if (blockResult.getException() != null) {
|
|
|
+ listener.onFailure(blockResult.getException());
|
|
|
+ } else {
|
|
|
+ List<AddIndexBlockResponse.AddBlockShardResult.Failure> shardFailures = new ArrayList<>(
|
|
|
+ blockResult.getShards().length
|
|
|
+ );
|
|
|
+ for (AddIndexBlockResponse.AddBlockShardResult shard : blockResult.getShards()) {
|
|
|
+ if (shard.hasFailures()) {
|
|
|
+ shardFailures.addAll(Arrays.asList(shard.getFailures()));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ assert shardFailures.isEmpty() == false
|
|
|
+ : "The block response must have shard failures as the global "
|
|
|
+ + "exception is null. The block result is: "
|
|
|
+ + blockResult;
|
|
|
+ String errorMessage = org.elasticsearch.common.Strings.collectionToDelimitedString(
|
|
|
+ shardFailures.stream().map(org.elasticsearch.common.Strings::toString).collect(Collectors.toList()),
|
|
|
+ ","
|
|
|
+ );
|
|
|
+ listener.onFailure(new ElasticsearchException(errorMessage));
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ logger.error(
|
|
|
+ "Data stream lifecycle service request to mark index [{}] as readonly was not acknowledged",
|
|
|
+ targetIndex
|
|
|
+ );
|
|
|
+ listener.onFailure(
|
|
|
+ new ElasticsearchException("request to mark index [" + targetIndex + "] as read-only was not acknowledged")
|
|
|
+ );
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void onFailure(Exception e) {
|
|
|
+ if (e instanceof IndexNotFoundException) {
|
|
|
+ // index was already deleted, treat this as a success
|
|
|
+ errorStore.clearRecordedError(targetIndex);
|
|
|
+ listener.onResponse(null);
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
+ listener.onFailure(e);
|
|
|
+ }
|
|
|
+ });
|
|
|
+ }
|
|
|
+
|
|
|
+ private void deleteIndex(DeleteIndexRequest deleteIndexRequest, String reason, ActionListener<Void> listener) {
|
|
|
assert deleteIndexRequest.indices() != null && deleteIndexRequest.indices().length == 1
|
|
|
: "Data stream lifecycle deletes one index at a time";
|
|
|
// "saving" the index name here so we don't capture the entire request
|
|
@@ -521,11 +945,15 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
|
|
|
client.admin().indices().delete(deleteIndexRequest, new ActionListener<>() {
|
|
|
@Override
|
|
|
public void onResponse(AcknowledgedResponse acknowledgedResponse) {
|
|
|
- logger.info(
|
|
|
- "Data stream lifecycle successfully deleted index [{}] due to the lapsed [{}] retention period",
|
|
|
- targetIndex,
|
|
|
- retention
|
|
|
- );
|
|
|
+ if (acknowledgedResponse.isAcknowledged()) {
|
|
|
+ logger.info("Data stream lifecycle successfully deleted index [{}] due to {}", targetIndex, reason);
|
|
|
+ } else {
|
|
|
+ logger.trace(
|
|
|
+ "The delete request for index [{}] was not acknowledged. Data stream lifecycle service will retry on the"
|
|
|
+ + " next run if the index still exists",
|
|
|
+ targetIndex
|
|
|
+ );
|
|
|
+ }
|
|
|
listener.onResponse(null);
|
|
|
}
|
|
|
|
|
@@ -555,6 +983,38 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
|
|
|
});
|
|
|
}
|
|
|
|
|
|
+ private void downsampleIndex(DownsampleAction.Request request, ActionListener<Void> listener) {
|
|
|
+ String sourceIndex = request.getSourceIndex();
|
|
|
+ String downsampleIndex = request.getTargetIndex();
|
|
|
+ logger.info("Data stream lifecycle issuing request to downsample index [{}] to index [{}]", sourceIndex, downsampleIndex);
|
|
|
+ client.execute(DownsampleAction.INSTANCE, request, new ActionListener<>() {
|
|
|
+ @Override
|
|
|
+ public void onResponse(AcknowledgedResponse acknowledgedResponse) {
|
|
|
+ assert acknowledgedResponse.isAcknowledged() : "the downsample response is always acknowledged";
|
|
|
+ logger.info("Data stream lifecycle successfully downsampled index [{}] to index [{}]", sourceIndex, downsampleIndex);
|
|
|
+ listener.onResponse(null);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void onFailure(Exception e) {
|
|
|
+ String previousError = errorStore.getError(sourceIndex);
|
|
|
+
|
|
|
+ listener.onFailure(e);
|
|
|
+ // To avoid spamming our logs, we only want to log the error once.
|
|
|
+ if (previousError == null || previousError.equals(errorStore.getError(sourceIndex)) == false) {
|
|
|
+ logger.error(
|
|
|
+ () -> Strings.format(
|
|
|
+ "Data stream lifecycle encountered an error trying to downsample index [%s]. Data stream lifecycle will "
|
|
|
+ + "attempt to downsample the index on its next run.",
|
|
|
+ sourceIndex
|
|
|
+ ),
|
|
|
+ e
|
|
|
+ );
|
|
|
+ }
|
|
|
+ }
|
|
|
+ });
|
|
|
+ }
|
|
|
+
|
|
|
/*
|
|
|
* This method executes the given force merge request. Once the request has completed successfully it writes a timestamp as custom
|
|
|
* metadata in the cluster state indicating when the force merge has completed. The listener is notified after the cluster state
|