浏览代码

Run downsampling using persistent tasks (#97557)

Add support for downsampling persistent tasks. We would like to have the ability
to resume downsampling tasks, instead of starting from scratch in case of
("retriable") failures.

Instead of keeping track of the state of the task we just try to query the downsampling
target index before starting the actual downsampling task, getting the latest document
indexed and its tsid. If we find any, we just restart from that tsid, maybe
overwriting a subset of the documents already indexed (documents with the same
tsid whose timestamp is smaller then the one of the latest document).

Querying the downsampling target index is possible after introducing a predictable
naming scheme for the target index. This was not the case before, a part of the target
index name was random. That is required to be able to start from where the task left.

Note that since the ordering of documents is based on (tsid, timestamp) we do not
include the timestamp in the query to avoid skipping downsampling of documents
with a larger tsid but smaller timestamp.

Note also that we are not removing the TransportDownsampleIndexerAction for
backward compatibility. An older master node running a previous version would
not use persistent tasks and an API call would result in something like "missing
downsample action".

We are also adding an additional downsample rest query parameter, timeout.
It allows a user to set the maximum time taken while waiting for a downsampling
task to complete before returning with an timeout error. Note that when the timeout
triggers the task will still run under control of the executor, just waiting for it to finish
will result in a timeout. The default value for this timeout is 1 day.

Closes #93582

---------

Co-authored-by: Martijn van Groningen <martijn.v.groningen@gmail.com>
Salvatore Campagna 2 年之前
父节点
当前提交
a52ab89ae6
共有 30 个文件被更改,包括 2709 次插入633 次删除
  1. 6 0
      docs/changelog/97557.yaml
  2. 3 1
      server/src/main/java/org/elasticsearch/TransportVersion.java
  3. 2 2
      server/src/main/java/org/elasticsearch/common/IndexNameGenerator.java
  4. 30 2
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/downsample/DownsampleAction.java
  5. 50 34
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/DownsampleAction.java
  6. 26 62
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/DownsampleStep.java
  7. 28 2
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/GenerateUniqueIndexNameStep.java
  8. 17 2
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/RollupShardIndexerStatus.java
  9. 126 0
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/RollupShardPersistentTaskState.java
  10. 37 11
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/RollupShardTask.java
  11. 12 8
      x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/DownsampleActionTests.java
  12. 34 79
      x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/DownsampleStepTests.java
  13. 12 10
      x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/TimeseriesLifecycleTypeTests.java
  14. 67 0
      x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/RollupShardPersistentTaskStateTests.java
  15. 32 16
      x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/DownsampleActionIT.java
  16. 33 0
      x-pack/plugin/ilm/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/ilm/70_downsampling.yml
  17. 3 0
      x-pack/plugin/rollup/build.gradle
  18. 442 0
      x-pack/plugin/rollup/src/internalClusterTest/java/org/elasticsearch/xpack/downsample/DownsampleClusterDisruptionIT.java
  19. 7 43
      x-pack/plugin/rollup/src/internalClusterTest/java/org/elasticsearch/xpack/downsample/DownsampleTransportFailureIT.java
  20. 331 0
      x-pack/plugin/rollup/src/internalClusterTest/java/org/elasticsearch/xpack/downsample/ILMDownsampleDisruptionIT.java
  21. 211 0
      x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/downsample/DownsampleShardTaskParams.java
  22. 8 1
      x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/downsample/RestDownsampleAction.java
  23. 93 38
      x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/downsample/RollupShardIndexer.java
  24. 28 0
      x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/downsample/RollupShardIndexerException.java
  25. 187 0
      x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/downsample/RollupShardPersistentTaskExecutor.java
  26. 405 254
      x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/downsample/TransportDownsampleAction.java
  27. 4 1
      x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/downsample/TransportDownsampleIndexerAction.java
  28. 98 2
      x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/Rollup.java
  29. 372 65
      x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/downsample/DownsampleActionSingleNodeTests.java
  30. 5 0
      x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/downsample/DownsampleDataStreamTests.java

+ 6 - 0
docs/changelog/97557.yaml

@@ -0,0 +1,6 @@
+pr: 97557
+summary: Run downsampling using persistent tasks
+area: Downsampling
+type: enhancement
+issues:
+ - 93582

+ 3 - 1
server/src/main/java/org/elasticsearch/TransportVersion.java

@@ -156,6 +156,7 @@ public record TransportVersion(int id) implements VersionId<TransportVersion> {
     public static final TransportVersion V_8_500_032 = registerTransportVersion(8_500_032, "a9a14bc6-c3f2-41d9-a3d8-c686bf2c901d");
     public static final TransportVersion V_8_500_033 = registerTransportVersion(8_500_033, "193ab7c4-a751-4cbd-a66a-2d7d56ccbc10");
     public static final TransportVersion V_8_500_034 = registerTransportVersion(8_500_034, "16871c8b-88ba-4432-980a-10fd9ecad2dc");
+
     public static final TransportVersion V_8_500_035 = registerTransportVersion(8_500_035, "664dd6ce-3487-4fbd-81a9-af778b28be45");
     // Introduced for stateless plugin
     public static final TransportVersion V_8_500_036 = registerTransportVersion(8_500_036, "3343c64f-d7ac-4f02-9262-3e1acfc56f89");
@@ -182,9 +183,10 @@ public record TransportVersion(int id) implements VersionId<TransportVersion> {
     public static final TransportVersion V_8_500_057 = registerTransportVersion(8_500_057, "80c088c6-358d-43b2-8d9c-1ea3c6c2b9fd");
     public static final TransportVersion V_8_500_058 = registerTransportVersion(8_500_058, "41d9c98a-1de2-4dc1-86f1-abd4cc1bef57");
     public static final TransportVersion V_8_500_059 = registerTransportVersion(8_500_059, "2f2090c0-7cd0-4a10-8f02-63d26073604f");
+    public static final TransportVersion V_8_500_060 = registerTransportVersion(8_500_060, "ec065a44-b468-4f8a-aded-7b90ca8d792b");
 
     private static class CurrentHolder {
-        private static final TransportVersion CURRENT = findCurrent(V_8_500_059);
+        private static final TransportVersion CURRENT = findCurrent(V_8_500_060);
 
         // finds the pluggable current version, or uses the given fallback
         private static TransportVersion findCurrent(TransportVersion fallback) {

+ 2 - 2
server/src/main/java/org/elasticsearch/common/IndexNameGenerator.java

@@ -22,8 +22,8 @@ import java.util.function.Supplier;
  */
 public final class IndexNameGenerator {
 
-    static final String ILLEGAL_INDEXNAME_CHARS_REGEX = "[/:\"*?<>|# ,\\\\]+";
-    static final int MAX_GENERATED_UUID_LENGTH = 4;
+    public static final String ILLEGAL_INDEXNAME_CHARS_REGEX = "[/:\"*?<>|# ,\\\\]+";
+    public static final int MAX_GENERATED_UUID_LENGTH = 4;
 
     private IndexNameGenerator() {}
 

+ 30 - 2
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/downsample/DownsampleAction.java

@@ -6,6 +6,7 @@
  */
 package org.elasticsearch.xpack.core.downsample;
 
+import org.elasticsearch.TransportVersion;
 import org.elasticsearch.action.ActionRequestBuilder;
 import org.elasticsearch.action.ActionRequestValidationException;
 import org.elasticsearch.action.ActionType;
@@ -17,6 +18,7 @@ import org.elasticsearch.action.support.master.MasterNodeRequest;
 import org.elasticsearch.client.internal.ElasticsearchClient;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.tasks.Task;
 import org.elasticsearch.tasks.TaskId;
 import org.elasticsearch.xcontent.ToXContentObject;
@@ -25,12 +27,14 @@ import org.elasticsearch.xcontent.XContentBuilder;
 import java.io.IOException;
 import java.util.Map;
 import java.util.Objects;
+import java.util.concurrent.TimeUnit;
 
 import static org.elasticsearch.action.ValidateActions.addValidationError;
 
 public class DownsampleAction extends ActionType<AcknowledgedResponse> {
     public static final DownsampleAction INSTANCE = new DownsampleAction();
     public static final String NAME = "indices:admin/xpack/downsample";
+    public static final TimeValue DEFAULT_WAIT_TIMEOUT = new TimeValue(1, TimeUnit.DAYS);
 
     private DownsampleAction() {
         super(NAME, AcknowledgedResponse::readFrom);
@@ -39,11 +43,18 @@ public class DownsampleAction extends ActionType<AcknowledgedResponse> {
     public static class Request extends MasterNodeRequest<Request> implements IndicesRequest, ToXContentObject {
         private String sourceIndex;
         private String targetIndex;
+        private TimeValue waitTimeout;
         private DownsampleConfig downsampleConfig;
 
-        public Request(String sourceIndex, String targetIndex, DownsampleConfig downsampleConfig) {
+        public Request(
+            final String sourceIndex,
+            final String targetIndex,
+            final TimeValue waitTimeout,
+            final DownsampleConfig downsampleConfig
+        ) {
             this.sourceIndex = sourceIndex;
             this.targetIndex = targetIndex;
+            this.waitTimeout = waitTimeout == null ? DEFAULT_WAIT_TIMEOUT : waitTimeout;
             this.downsampleConfig = downsampleConfig;
         }
 
@@ -53,6 +64,9 @@ public class DownsampleAction extends ActionType<AcknowledgedResponse> {
             super(in);
             sourceIndex = in.readString();
             targetIndex = in.readString();
+            waitTimeout = in.getTransportVersion().onOrAfter(TransportVersion.V_8_500_054)
+                ? TimeValue.parseTimeValue(in.readString(), "timeout")
+                : DEFAULT_WAIT_TIMEOUT;
             downsampleConfig = new DownsampleConfig(in);
         }
 
@@ -76,6 +90,11 @@ public class DownsampleAction extends ActionType<AcknowledgedResponse> {
             super.writeTo(out);
             out.writeString(sourceIndex);
             out.writeString(targetIndex);
+            out.writeString(
+                out.getTransportVersion().onOrAfter(TransportVersion.V_8_500_054)
+                    ? waitTimeout.getStringRep()
+                    : DEFAULT_WAIT_TIMEOUT.getStringRep()
+            );
             downsampleConfig.writeTo(out);
         }
 
@@ -87,6 +106,13 @@ public class DownsampleAction extends ActionType<AcknowledgedResponse> {
             return targetIndex;
         }
 
+        /**
+         * @return the time to wait for the persistent tasks the complete downsampling
+         */
+        public TimeValue getWaitTimeout() {
+            return waitTimeout;
+        }
+
         public DownsampleConfig getDownsampleConfig() {
             return downsampleConfig;
         }
@@ -111,6 +137,7 @@ public class DownsampleAction extends ActionType<AcknowledgedResponse> {
             builder.startObject();
             builder.field("source_index", sourceIndex);
             builder.field("target_index", targetIndex);
+            builder.field("wait_timeout", waitTimeout);
             downsampleConfig.toXContent(builder, params);
             builder.endObject();
             return builder;
@@ -118,7 +145,7 @@ public class DownsampleAction extends ActionType<AcknowledgedResponse> {
 
         @Override
         public int hashCode() {
-            return Objects.hash(sourceIndex, targetIndex, downsampleConfig);
+            return Objects.hash(sourceIndex, targetIndex, waitTimeout, downsampleConfig);
         }
 
         @Override
@@ -132,6 +159,7 @@ public class DownsampleAction extends ActionType<AcknowledgedResponse> {
             Request other = (Request) obj;
             return Objects.equals(sourceIndex, other.sourceIndex)
                 && Objects.equals(targetIndex, other.targetIndex)
+                && Objects.equals(waitTimeout, other.waitTimeout)
                 && Objects.equals(downsampleConfig, other.downsampleConfig);
         }
     }

+ 50 - 34
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/DownsampleAction.java

@@ -6,6 +6,7 @@
  */
 package org.elasticsearch.xpack.core.ilm;
 
+import org.elasticsearch.TransportVersion;
 import org.elasticsearch.action.downsample.DownsampleConfig;
 import org.elasticsearch.client.internal.Client;
 import org.elasticsearch.cluster.health.ClusterHealthStatus;
@@ -14,6 +15,7 @@ import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.index.IndexMode;
 import org.elasticsearch.index.IndexSettings;
 import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
@@ -27,8 +29,11 @@ import org.elasticsearch.xpack.core.ilm.Step.StepKey;
 import java.io.IOException;
 import java.util.List;
 import java.util.Objects;
+import java.util.concurrent.TimeUnit;
 
 import static org.elasticsearch.xcontent.ConstructingObjectParser.constructorArg;
+import static org.elasticsearch.xcontent.ConstructingObjectParser.optionalConstructorArg;
+import static org.elasticsearch.xpack.core.ilm.DownsampleStep.generateDownsampleIndexName;
 
 /**
  * A {@link LifecycleAction} which calls {@link org.elasticsearch.xpack.core.downsample.DownsampleAction} on an index
@@ -40,11 +45,13 @@ public class DownsampleAction implements LifecycleAction {
     public static final String CONDITIONAL_TIME_SERIES_CHECK_KEY = BranchingStep.NAME + "-on-timeseries-check";
     public static final String CONDITIONAL_DATASTREAM_CHECK_KEY = BranchingStep.NAME + "-on-datastream-check";
     public static final String GENERATE_DOWNSAMPLE_STEP_NAME = "generate-downsampled-index-name";
+    public static final TimeValue DEFAULT_WAIT_TIMEOUT = new TimeValue(1, TimeUnit.DAYS);
     private static final ParseField FIXED_INTERVAL_FIELD = new ParseField(DownsampleConfig.FIXED_INTERVAL);
+    private static final ParseField WAIT_TIMEOUT_FIELD = new ParseField("wait_timeout");
 
     private static final ConstructingObjectParser<DownsampleAction, Void> PARSER = new ConstructingObjectParser<>(
         NAME,
-        a -> new DownsampleAction((DateHistogramInterval) a[0])
+        a -> new DownsampleAction((DateHistogramInterval) a[0], (TimeValue) a[1])
     );
 
     static {
@@ -54,34 +61,53 @@ public class DownsampleAction implements LifecycleAction {
             FIXED_INTERVAL_FIELD,
             ObjectParser.ValueType.STRING
         );
+        PARSER.declareField(
+            optionalConstructorArg(),
+            p -> TimeValue.parseTimeValue(p.textOrNull(), WAIT_TIMEOUT_FIELD.getPreferredName()),
+            WAIT_TIMEOUT_FIELD,
+            ObjectParser.ValueType.STRING
+        );
     }
 
     private final DateHistogramInterval fixedInterval;
+    private final TimeValue waitTimeout;
 
     public static DownsampleAction parse(XContentParser parser) {
         return PARSER.apply(parser, null);
     }
 
-    public DownsampleAction(DateHistogramInterval fixedInterval) {
+    public DownsampleAction(final DateHistogramInterval fixedInterval, final TimeValue waitTimeout) {
         if (fixedInterval == null) {
             throw new IllegalArgumentException("Parameter [" + FIXED_INTERVAL_FIELD.getPreferredName() + "] is required.");
         }
         this.fixedInterval = fixedInterval;
+        this.waitTimeout = waitTimeout == null ? DEFAULT_WAIT_TIMEOUT : waitTimeout;
     }
 
     public DownsampleAction(StreamInput in) throws IOException {
-        this(new DateHistogramInterval(in));
+        this(
+            new DateHistogramInterval(in),
+            in.getTransportVersion().onOrAfter(TransportVersion.V_8_500_054)
+                ? TimeValue.parseTimeValue(in.readString(), WAIT_TIMEOUT_FIELD.getPreferredName())
+                : DEFAULT_WAIT_TIMEOUT
+        );
     }
 
     @Override
     public void writeTo(StreamOutput out) throws IOException {
         fixedInterval.writeTo(out);
+        if (out.getTransportVersion().onOrAfter(TransportVersion.V_8_500_054)) {
+            out.writeString(waitTimeout.getStringRep());
+        } else {
+            out.writeString(DEFAULT_WAIT_TIMEOUT.getStringRep());
+        }
     }
 
     @Override
     public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
         builder.startObject();
         builder.field(FIXED_INTERVAL_FIELD.getPreferredName(), fixedInterval.toString());
+        builder.field(WAIT_TIMEOUT_FIELD.getPreferredName(), waitTimeout.getStringRep());
         builder.endObject();
         return builder;
     }
@@ -95,6 +121,10 @@ public class DownsampleAction implements LifecycleAction {
         return fixedInterval;
     }
 
+    public TimeValue waitTimeout() {
+        return waitTimeout;
+    }
+
     @Override
     public boolean isSafeAction() {
         return false;
@@ -136,35 +166,21 @@ public class DownsampleAction implements LifecycleAction {
         WaitForNoFollowersStep waitForNoFollowersStep = new WaitForNoFollowersStep(waitForNoFollowerStepKey, readOnlyKey, client);
 
         // Mark source index as read-only
-        ReadOnlyStep readOnlyStep = new ReadOnlyStep(readOnlyKey, cleanupDownsampleIndexKey, client);
-
-        // We generate a unique downsample index name, but we also retry if the allocation of the downsample index
-        // is not possible, so we want to delete the "previously generated" downsample index (this is a no-op if it's
-        // the first run of the action, and we haven't generated a downsample index name)
-        CleanupTargetIndexStep cleanupDownsampleIndexStep = new CleanupTargetIndexStep(
-            cleanupDownsampleIndexKey,
-            generateDownsampleIndexNameKey,
-            client,
-            (indexMetadata) -> IndexMetadata.INDEX_DOWNSAMPLE_SOURCE_NAME.get(indexMetadata.getSettings()),
-            (indexMetadata) -> indexMetadata.getLifecycleExecutionState().downsampleIndexName()
-        );
+        ReadOnlyStep readOnlyStep = new ReadOnlyStep(readOnlyKey, downsampleKey, client);
 
-        // Generate a unique downsample index name and store it in the ILM execution state
-        GenerateUniqueIndexNameStep generateDownsampleIndexNameStep = new GenerateUniqueIndexNameStep(
-            generateDownsampleIndexNameKey,
-            downsampleKey,
-            DOWNSAMPLED_INDEX_PREFIX,
-            (downsampleIndexName, lifecycleStateBuilder) -> lifecycleStateBuilder.setDownsampleIndexName(downsampleIndexName)
-        );
+        // Before the downsample action was retry-able, we used to generate a unique downsample index name and delete the previous index in
+        // case a failure occurred. The downsample action can now retry execution in case of failure and start where it left off, so no
+        // unique name needs to be generated and the target index is now predictable and generated in the downsample step.
+        // (This noop step exists so deployments that are in this step (that has been converted to a noop) when the Elasticsearch
+        // upgrade was performed resume the ILM execution and complete the downsample action after upgrade.)
+        NoopStep cleanupDownsampleIndexStep = new NoopStep(cleanupDownsampleIndexKey, downsampleKey);
+
+        // Before a random downsample index name was generated.
+        // (this noop step allows ILM to resume after a rollover without failing)
+        NoopStep generateDownsampleIndexNameStep = new NoopStep(generateDownsampleIndexNameKey, downsampleKey);
 
         // Here is where the actual downsample action takes place
-        DownsampleStep downsampleStep = new DownsampleStep(
-            downsampleKey,
-            waitForDownsampleIndexKey,
-            cleanupDownsampleIndexKey,
-            client,
-            fixedInterval
-        );
+        DownsampleStep downsampleStep = new DownsampleStep(downsampleKey, waitForDownsampleIndexKey, client, fixedInterval, waitTimeout);
 
         // Wait until the downsampled index is recovered. We again wait until the configured threshold is breached and
         // if the downsampled index has not successfully recovered until then, we rewind to the "cleanup-downsample-index"
@@ -175,7 +191,7 @@ public class DownsampleAction implements LifecycleAction {
                 waitForDownsampleIndexKey,
                 copyMetadataKey,
                 ClusterHealthStatus.YELLOW,
-                (indexName, lifecycleState) -> lifecycleState.downsampleIndexName()
+                (indexName, lifecycleState) -> generateDownsampleIndexName(indexName, fixedInterval)
             ),
             cleanupDownsampleIndexKey
         );
@@ -183,14 +199,14 @@ public class DownsampleAction implements LifecycleAction {
         CopyExecutionStateStep copyExecutionStateStep = new CopyExecutionStateStep(
             copyMetadataKey,
             copyIndexLifecycleKey,
-            (indexName, lifecycleState) -> lifecycleState.downsampleIndexName(),
+            (indexName, lifecycleState) -> generateDownsampleIndexName(indexName, fixedInterval),
             nextStepKey
         );
 
         CopySettingsStep copyLifecycleSettingsStep = new CopySettingsStep(
             copyIndexLifecycleKey,
             dataStreamCheckBranchingKey,
-            (index, lifecycleState) -> lifecycleState.downsampleIndexName(),
+            (indexName, lifecycleState) -> generateDownsampleIndexName(indexName, fixedInterval),
             LifecycleSettings.LIFECYCLE_NAME_SETTING.getKey()
         );
 
@@ -212,7 +228,7 @@ public class DownsampleAction implements LifecycleAction {
         ReplaceDataStreamBackingIndexStep replaceDataStreamBackingIndex = new ReplaceDataStreamBackingIndexStep(
             replaceDataStreamIndexKey,
             deleteIndexKey,
-            (sourceIndexName, lifecycleState) -> lifecycleState.downsampleIndexName()
+            (sourceIndexName, lifecycleState) -> generateDownsampleIndexName(sourceIndexName, fixedInterval)
         );
         DeleteStep deleteSourceIndexStep = new DeleteStep(deleteIndexKey, nextStepKey, client);
 
@@ -220,7 +236,7 @@ public class DownsampleAction implements LifecycleAction {
             swapAliasesKey,
             nextStepKey,
             client,
-            (indexName, lifecycleState) -> lifecycleState.downsampleIndexName(),
+            (indexName, lifecycleState) -> generateDownsampleIndexName(indexName, fixedInterval),
             false
         );
 

+ 26 - 62
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/DownsampleStep.java

@@ -15,13 +15,14 @@ import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.ClusterStateObserver;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.LifecycleExecutionState;
-import org.elasticsearch.common.Strings;
 import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
 import org.elasticsearch.xpack.core.downsample.DownsampleAction;
 
 import java.util.Objects;
 
+import static org.elasticsearch.xpack.core.ilm.DownsampleAction.DOWNSAMPLED_INDEX_PREFIX;
+
 /**
  * ILM step that invokes the downsample action for an index using a {@link DateHistogramInterval}. The downsample
  * index name is retrieved from the lifecycle state {@link LifecycleExecutionState#downsampleIndexName()}
@@ -30,25 +31,21 @@ import java.util.Objects;
  */
 public class DownsampleStep extends AsyncActionStep {
     public static final String NAME = "rollup";
-
-    private static final Logger logger = LogManager.getLogger(DownsampleStep.class);
+    private static final Logger LOGGER = LogManager.getLogger(DownsampleStep.class);
 
     private final DateHistogramInterval fixedInterval;
-    private final StepKey nextStepOnSuccess;
-    private final StepKey nextStepOnFailure;
-    private volatile boolean downsampleFailed;
+    private final TimeValue waitTimeout;
 
     public DownsampleStep(
-        StepKey key,
-        StepKey nextStepOnSuccess,
-        StepKey nextStepOnFailure,
-        Client client,
-        DateHistogramInterval fixedInterval
+        final StepKey key,
+        final StepKey nextStepKey,
+        final Client client,
+        final DateHistogramInterval fixedInterval,
+        final TimeValue waitTimeout
     ) {
-        super(key, null, client);
-        this.nextStepOnSuccess = nextStepOnSuccess;
-        this.nextStepOnFailure = nextStepOnFailure;
+        super(key, nextStepKey, client);
         this.fixedInterval = fixedInterval;
+        this.waitTimeout = waitTimeout;
     }
 
     @Override
@@ -70,16 +67,7 @@ public class DownsampleStep extends AsyncActionStep {
 
         final String policyName = indexMetadata.getLifecyclePolicyName();
         final String indexName = indexMetadata.getIndex().getName();
-        final String downsampleIndexName = lifecycleState.downsampleIndexName();
-        if (Strings.hasText(downsampleIndexName) == false) {
-            downsampleFailed = true;
-            listener.onFailure(
-                new IllegalStateException(
-                    "downsample index name was not generated for policy [" + policyName + "] and index [" + indexName + "]"
-                )
-            );
-            return;
-        }
+        final String downsampleIndexName = generateDownsampleIndexName(indexName, fixedInterval);
 
         IndexMetadata downsampleIndexMetadata = currentState.metadata().index(downsampleIndexName);
         if (downsampleIndexMetadata != null) {
@@ -89,7 +77,7 @@ public class DownsampleStep extends AsyncActionStep {
             if (IndexMetadata.DownsampleTaskStatus.SUCCESS.equals(downsampleIndexStatus)) {
                 // Downsample index has already been created with the generated name and its status is "success".
                 // So we skip index downsample creation.
-                logger.warn(
+                LOGGER.info(
                     "skipping [{}] step for index [{}] as part of policy [{}] as the downsample index [{}] already exists",
                     DownsampleStep.NAME,
                     indexName,
@@ -97,56 +85,31 @@ public class DownsampleStep extends AsyncActionStep {
                     downsampleIndexName
                 );
                 listener.onResponse(null);
-            } else {
-                // Downsample index has already been created with the generated name but its status is not "success".
-                // So we fail this step so that we go back to cleaning up the index and try again with a new downsample
-                // index name.
-                downsampleFailed = true;
-                listener.onFailure(
-                    new IllegalStateException(
-                        "failing ["
-                            + DownsampleStep.NAME
-                            + "] step for index ["
-                            + indexName
-                            + "] as part of policy ["
-                            + policyName
-                            + "] because the downsample index ["
-                            + downsampleIndexName
-                            + "] already exists with downsample status ["
-                            + downsampleIndexStatus
-                            + "]"
-                    )
-                );
             }
         } else {
-            performDownsampleIndex(indexName, downsampleIndexName, ActionListener.wrap(listener::onResponse, e -> {
-                downsampleFailed = true;
-                listener.onFailure(e);
-            }));
+            performDownsampleIndex(indexName, downsampleIndexName, ActionListener.wrap(listener::onResponse, listener::onFailure));
         }
     }
 
     void performDownsampleIndex(String indexName, String downsampleIndexName, ActionListener<Void> listener) {
         DownsampleConfig config = new DownsampleConfig(fixedInterval);
-        DownsampleAction.Request request = new DownsampleAction.Request(indexName, downsampleIndexName, config).masterNodeTimeout(
-            TimeValue.MAX_VALUE
-        );
+        DownsampleAction.Request request = new DownsampleAction.Request(indexName, downsampleIndexName, waitTimeout, config)
+            .masterNodeTimeout(TimeValue.MAX_VALUE);
         // Currently, DownsampleAction always acknowledges action was complete when no exceptions are thrown.
         getClient().execute(DownsampleAction.INSTANCE, request, listener.delegateFailureAndWrap((l, response) -> l.onResponse(null)));
     }
 
-    @Override
-    public final StepKey getNextStepKey() {
-        return downsampleFailed ? nextStepOnFailure : nextStepOnSuccess;
-    }
-
     public DateHistogramInterval getFixedInterval() {
         return fixedInterval;
     }
 
+    public TimeValue getWaitTimeout() {
+        return waitTimeout;
+    }
+
     @Override
     public int hashCode() {
-        return Objects.hash(super.hashCode(), fixedInterval, nextStepOnSuccess, nextStepOnFailure);
+        return Objects.hash(super.hashCode(), fixedInterval, waitTimeout);
     }
 
     @Override
@@ -161,9 +124,10 @@ public class DownsampleStep extends AsyncActionStep {
             return false;
         }
         DownsampleStep other = (DownsampleStep) obj;
-        return super.equals(obj)
-            && Objects.equals(fixedInterval, other.fixedInterval)
-            && Objects.equals(nextStepOnSuccess, other.nextStepOnSuccess)
-            && Objects.equals(nextStepOnFailure, other.nextStepOnFailure);
+        return super.equals(obj) && Objects.equals(fixedInterval, other.fixedInterval) && Objects.equals(waitTimeout, other.waitTimeout);
+    }
+
+    static String generateDownsampleIndexName(String sourceIndexName, DateHistogramInterval fixedInterval) {
+        return DOWNSAMPLED_INDEX_PREFIX + sourceIndexName + "-" + fixedInterval;
     }
 }

+ 28 - 2
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/GenerateUniqueIndexNameStep.java

@@ -13,12 +13,16 @@ import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.LifecycleExecutionState;
 import org.elasticsearch.cluster.metadata.LifecycleExecutionState.Builder;
+import org.elasticsearch.common.UUIDs;
 import org.elasticsearch.index.Index;
 
+import java.util.Locale;
 import java.util.Objects;
 import java.util.function.BiFunction;
+import java.util.function.Supplier;
 
-import static org.elasticsearch.common.IndexNameGenerator.generateValidIndexName;
+import static org.elasticsearch.common.IndexNameGenerator.ILLEGAL_INDEXNAME_CHARS_REGEX;
+import static org.elasticsearch.common.IndexNameGenerator.MAX_GENERATED_UUID_LENGTH;
 import static org.elasticsearch.common.IndexNameGenerator.validateGeneratedIndexName;
 
 /**
@@ -73,7 +77,7 @@ public class GenerateUniqueIndexNameStep extends ClusterStateActionStep {
 
         Builder newLifecycleState = LifecycleExecutionState.builder(lifecycleState);
         String policyName = indexMetadata.getLifecyclePolicyName();
-        String generatedIndexName = generateValidIndexName(prefix, index.getName());
+        String generatedIndexName = generateIndexName(prefix, index.getName());
         ActionRequestValidationException validationException = validateGeneratedIndexName(generatedIndexName, clusterState);
         if (validationException != null) {
             logger.warn(
@@ -110,4 +114,26 @@ public class GenerateUniqueIndexNameStep extends ClusterStateActionStep {
         return Objects.hash(super.hashCode(), prefix);
     }
 
+    public String generateIndexName(final String prefix, final String indexName) {
+        return generateValidIndexName(prefix, indexName);
+    }
+
+    /**
+     * This generates a valid unique index name by using the provided prefix, appended with a generated UUID, and the index name.
+     */
+    static String generateValidIndexName(String prefix, String indexName) {
+        String randomUUID = generateValidIndexSuffix(UUIDs::randomBase64UUID);
+        randomUUID = randomUUID.substring(0, Math.min(randomUUID.length(), MAX_GENERATED_UUID_LENGTH));
+        return prefix + randomUUID + "-" + indexName;
+    }
+
+    static String generateValidIndexSuffix(Supplier<String> randomGenerator) {
+        String randomSuffix = randomGenerator.get().toLowerCase(Locale.ROOT);
+        randomSuffix = randomSuffix.replaceAll(ILLEGAL_INDEXNAME_CHARS_REGEX, "");
+        if (randomSuffix.length() == 0) {
+            throw new IllegalArgumentException("unable to generate random index name suffix");
+        }
+
+        return randomSuffix;
+    }
 }

+ 17 - 2
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/RollupShardIndexerStatus.java

@@ -7,13 +7,28 @@
 
 package org.elasticsearch.xpack.core.rollup.action;
 
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.io.stream.Writeable;
+
+import java.io.IOException;
+
 /**
  * Status of the rollup indexer task
  */
-public enum RollupShardIndexerStatus {
+public enum RollupShardIndexerStatus implements Writeable {
     INITIALIZED,
     STARTED,
     FAILED,
     COMPLETED,
-    CANCELLED
+    CANCELLED;
+
+    public static RollupShardIndexerStatus readFromStream(final StreamInput in) throws IOException {
+        return in.readEnum(RollupShardIndexerStatus.class);
+    }
+
+    @Override
+    public void writeTo(final StreamOutput out) throws IOException {
+        out.writeEnum(this);
+    }
 }

+ 126 - 0
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/RollupShardPersistentTaskState.java

@@ -0,0 +1,126 @@
+/*
+ * 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.rollup.action;
+
+import org.apache.lucene.util.BytesRef;
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.persistent.PersistentTaskState;
+import org.elasticsearch.xcontent.ObjectParser;
+import org.elasticsearch.xcontent.ParseField;
+import org.elasticsearch.xcontent.XContentBuilder;
+import org.elasticsearch.xcontent.XContentParser;
+
+import java.io.IOException;
+
+/**
+ * @param rollupShardIndexerStatus An instance of {@link RollupShardIndexerStatus} with the rollupShardIndexerStatus of the rollup task
+ * @param tsid The latest successfully processed tsid component of a tuple (tsid, timestamp)
+ */
+public record RollupShardPersistentTaskState(RollupShardIndexerStatus rollupShardIndexerStatus, BytesRef tsid)
+    implements
+        PersistentTaskState {
+
+    public static final String NAME = RollupShardTask.TASK_NAME;
+    private static final ParseField ROLLUP_SHARD_INDEXER_STATUS = new ParseField("status");
+    private static final ParseField TSID = new ParseField("tsid");
+
+    public static final ObjectParser<RollupShardPersistentTaskState.Builder, Void> PARSER = new ObjectParser<>(NAME);
+
+    static {
+        PARSER.declareField(
+            RollupShardPersistentTaskState.Builder::status,
+            (p, c) -> RollupShardIndexerStatus.valueOf(p.textOrNull()),
+            ROLLUP_SHARD_INDEXER_STATUS,
+            ObjectParser.ValueType.STRING
+        );
+        PARSER.declareField(
+            RollupShardPersistentTaskState.Builder::tsid,
+            (p, c) -> new BytesRef(p.textOrNull()),
+            TSID,
+            ObjectParser.ValueType.STRING
+        );
+    }
+
+    public RollupShardPersistentTaskState(final StreamInput in) throws IOException {
+        this(RollupShardIndexerStatus.readFromStream(in), in.readBytesRef());
+    }
+
+    @Override
+    public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
+        builder.startObject();
+        builder.field(ROLLUP_SHARD_INDEXER_STATUS.getPreferredName(), rollupShardIndexerStatus);
+        if (tsid != null) {
+            builder.field(TSID.getPreferredName(), tsid.utf8ToString());
+        }
+        return builder.endObject();
+    }
+
+    @Override
+    public String getWriteableName() {
+        return RollupShardTask.TASK_NAME;
+    }
+
+    @Override
+    public void writeTo(StreamOutput out) throws IOException {
+        rollupShardIndexerStatus.writeTo(out);
+        out.writeBytesRef(tsid);
+    }
+
+    @Override
+    public RollupShardIndexerStatus rollupShardIndexerStatus() {
+        return rollupShardIndexerStatus;
+    }
+
+    public boolean done() {
+        return RollupShardIndexerStatus.COMPLETED.equals(rollupShardIndexerStatus)
+            || RollupShardIndexerStatus.CANCELLED.equals(rollupShardIndexerStatus)
+            || RollupShardIndexerStatus.FAILED.equals(rollupShardIndexerStatus);
+    }
+
+    public boolean started() {
+        return RollupShardIndexerStatus.STARTED.equals(rollupShardIndexerStatus);
+    }
+
+    public boolean cancelled() {
+        return RollupShardIndexerStatus.CANCELLED.equals(rollupShardIndexerStatus);
+    }
+
+    public boolean failed() {
+        return RollupShardIndexerStatus.FAILED.equals(rollupShardIndexerStatus);
+    }
+
+    public static RollupShardPersistentTaskState readFromStream(final StreamInput in) throws IOException {
+        return new RollupShardPersistentTaskState(RollupShardIndexerStatus.readFromStream(in), in.readBytesRef());
+    }
+
+    public static RollupShardPersistentTaskState fromXContent(final XContentParser parser) throws IOException {
+        final RollupShardPersistentTaskState.Builder builder = new RollupShardPersistentTaskState.Builder();
+        PARSER.parse(parser, builder, null);
+        return builder.build();
+    }
+
+    public static class Builder {
+        private RollupShardIndexerStatus status;
+        private BytesRef tsid;
+
+        public Builder status(final RollupShardIndexerStatus status) {
+            this.status = status;
+            return this;
+        }
+
+        public Builder tsid(final BytesRef tsid) {
+            this.tsid = tsid;
+            return this;
+        }
+
+        public RollupShardPersistentTaskState build() {
+            return new RollupShardPersistentTaskState(status, tsid);
+        }
+    }
+}

+ 37 - 11
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/RollupShardTask.java

@@ -9,15 +9,18 @@ package org.elasticsearch.xpack.core.rollup.action;
 
 import org.elasticsearch.action.downsample.DownsampleConfig;
 import org.elasticsearch.index.shard.ShardId;
-import org.elasticsearch.tasks.CancellableTask;
+import org.elasticsearch.persistent.AllocatedPersistentTask;
+import org.elasticsearch.persistent.PersistentTasksService;
 import org.elasticsearch.tasks.TaskId;
+import org.elasticsearch.tasks.TaskManager;
 import org.elasticsearch.xpack.core.rollup.RollupField;
 
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 
-public class RollupShardTask extends CancellableTask {
+public class RollupShardTask extends AllocatedPersistentTask {
+    public static final String TASK_NAME = "rollup-shard";
     private final String rollupIndex;
     private volatile long totalShardDocCount;
     private volatile long docsProcessed;
@@ -37,20 +40,23 @@ public class RollupShardTask extends CancellableTask {
         RollupShardIndexerStatus.INITIALIZED
     );
     private final RollupBulkStats rollupBulkStats;
-    private final AtomicReference<RollupBeforeBulkInfo> lastBeforeBulkInfo = new AtomicReference<>(null);
-    private final AtomicReference<RollupAfterBulkInfo> lastAfterBulkInfo = new AtomicReference<>(null);
+    // Need to set initial values, because these atomic references can be read before bulk indexing started or when downsampling empty index
+    private final AtomicReference<RollupBeforeBulkInfo> lastBeforeBulkInfo = new AtomicReference<>(new RollupBeforeBulkInfo(0, 0, 0, 0));
+    private final AtomicReference<RollupAfterBulkInfo> lastAfterBulkInfo = new AtomicReference<>(
+        new RollupAfterBulkInfo(0, 0, 0, 0, false, 0)
+    );
 
     public RollupShardTask(
         long id,
-        String type,
-        String action,
-        TaskId parentTask,
-        String rollupIndex,
+        final String type,
+        final String action,
+        final TaskId parentTask,
+        final String rollupIndex,
         long indexStartTimeMillis,
         long indexEndTimeMillis,
-        DownsampleConfig config,
-        Map<String, String> headers,
-        ShardId shardId
+        final DownsampleConfig config,
+        final Map<String, String> headers,
+        final ShardId shardId
     ) {
         super(id, type, action, RollupField.NAME + "_" + rollupIndex + "[" + shardId.id() + "]", parentTask, headers);
         this.rollupIndex = rollupIndex;
@@ -62,6 +68,26 @@ public class RollupShardTask extends CancellableTask {
         this.rollupBulkStats = new RollupBulkStats();
     }
 
+    @Override
+    protected void init(
+        final PersistentTasksService persistentTasksService,
+        final TaskManager taskManager,
+        final String persistentTaskId,
+        final long allocationId
+    ) {
+        super.init(persistentTasksService, taskManager, persistentTaskId, allocationId);
+    }
+
+    // TODO: just for testing
+    public void testInit(
+        final PersistentTasksService persistentTasksService,
+        final TaskManager taskManager,
+        final String persistentTaskId,
+        final long allocationId
+    ) {
+        init(persistentTasksService, taskManager, persistentTaskId, allocationId);
+    }
+
     public String getRollupIndex() {
         return rollupIndex;
     }

+ 12 - 8
x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/DownsampleActionTests.java

@@ -7,6 +7,7 @@
 package org.elasticsearch.xpack.core.ilm;
 
 import org.elasticsearch.common.io.stream.Writeable.Reader;
+import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
 import org.elasticsearch.test.EqualsHashCodeTestUtils;
 import org.elasticsearch.xcontent.XContentParser;
@@ -14,6 +15,7 @@ import org.elasticsearch.xpack.core.ilm.Step.StepKey;
 import org.elasticsearch.xpack.core.rollup.ConfigTestHelpers;
 
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 
 import static org.elasticsearch.xpack.core.ilm.DownsampleAction.CONDITIONAL_DATASTREAM_CHECK_KEY;
 import static org.elasticsearch.xpack.core.ilm.DownsampleAction.CONDITIONAL_TIME_SERIES_CHECK_KEY;
@@ -22,8 +24,10 @@ import static org.hamcrest.Matchers.equalTo;
 
 public class DownsampleActionTests extends AbstractActionTestCase<DownsampleAction> {
 
+    public static final TimeValue WAIT_TIMEOUT = new TimeValue(1, TimeUnit.MINUTES);
+
     static DownsampleAction randomInstance() {
-        return new DownsampleAction(ConfigTestHelpers.randomInterval());
+        return new DownsampleAction(ConfigTestHelpers.randomInterval(), WAIT_TIMEOUT);
     }
 
     @Override
@@ -53,7 +57,7 @@ public class DownsampleActionTests extends AbstractActionTestCase<DownsampleActi
 
     @Override
     public void testToSteps() {
-        DownsampleAction action = new DownsampleAction(ConfigTestHelpers.randomInterval());
+        DownsampleAction action = new DownsampleAction(ConfigTestHelpers.randomInterval(), WAIT_TIMEOUT);
         String phase = randomAlphaOfLengthBetween(1, 10);
         StepKey nextStepKey = new StepKey(
             randomAlphaOfLengthBetween(1, 10),
@@ -80,13 +84,13 @@ public class DownsampleActionTests extends AbstractActionTestCase<DownsampleActi
 
         assertTrue(steps.get(3) instanceof ReadOnlyStep);
         assertThat(steps.get(3).getKey().name(), equalTo(ReadOnlyStep.NAME));
-        assertThat(steps.get(3).getNextStepKey().name(), equalTo(CleanupTargetIndexStep.NAME));
+        assertThat(steps.get(3).getNextStepKey().name(), equalTo(DownsampleStep.NAME));
 
-        assertTrue(steps.get(4) instanceof CleanupTargetIndexStep);
+        assertTrue(steps.get(4) instanceof NoopStep);
         assertThat(steps.get(4).getKey().name(), equalTo(CleanupTargetIndexStep.NAME));
-        assertThat(steps.get(4).getNextStepKey().name(), equalTo(GENERATE_DOWNSAMPLE_STEP_NAME));
+        assertThat(steps.get(4).getNextStepKey().name(), equalTo(DownsampleStep.NAME));
 
-        assertTrue(steps.get(5) instanceof GenerateUniqueIndexNameStep);
+        assertTrue(steps.get(5) instanceof NoopStep);
         assertThat(steps.get(5).getKey().name(), equalTo(GENERATE_DOWNSAMPLE_STEP_NAME));
         assertThat(steps.get(5).getNextStepKey().name(), equalTo(DownsampleStep.NAME));
 
@@ -130,11 +134,11 @@ public class DownsampleActionTests extends AbstractActionTestCase<DownsampleActi
     }
 
     DownsampleAction copy(DownsampleAction downsampleAction) {
-        return new DownsampleAction(downsampleAction.fixedInterval());
+        return new DownsampleAction(downsampleAction.fixedInterval(), downsampleAction.waitTimeout());
     }
 
     DownsampleAction notCopy(DownsampleAction downsampleAction) {
         DateHistogramInterval fixedInterval = randomValueOtherThan(downsampleAction.fixedInterval(), ConfigTestHelpers::randomInterval);
-        return new DownsampleAction(fixedInterval);
+        return new DownsampleAction(fixedInterval, WAIT_TIMEOUT);
     }
 }

+ 34 - 79
x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/DownsampleStepTests.java

@@ -15,6 +15,7 @@ import org.elasticsearch.cluster.metadata.DataStream;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.LifecycleExecutionState;
 import org.elasticsearch.cluster.metadata.Metadata;
+import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.index.IndexVersion;
 import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
 import org.elasticsearch.test.client.NoOpClient;
@@ -41,7 +42,13 @@ public class DownsampleStepTests extends AbstractStepTestCase<DownsampleStep> {
         StepKey stepKey = randomStepKey();
         StepKey nextStepKey = randomStepKey();
         DateHistogramInterval fixedInterval = ConfigTestHelpers.randomInterval();
-        return new DownsampleStep(stepKey, nextStepKey, null, client, fixedInterval);
+        return new DownsampleStep(
+            stepKey,
+            nextStepKey,
+            client,
+            fixedInterval,
+            TimeValue.parseTimeValue(randomTimeValue(1, 1000, "d", "h", "ms", "s", "m"), "timeout")
+        );
     }
 
     @Override
@@ -49,20 +56,31 @@ public class DownsampleStepTests extends AbstractStepTestCase<DownsampleStep> {
         StepKey key = instance.getKey();
         StepKey nextKey = instance.getNextStepKey();
         DateHistogramInterval fixedInterval = instance.getFixedInterval();
+        TimeValue timeout = instance.getWaitTimeout();
 
-        switch (between(0, 2)) {
+        switch (between(0, 3)) {
             case 0 -> key = new StepKey(key.phase(), key.action(), key.name() + randomAlphaOfLength(5));
             case 1 -> nextKey = new StepKey(nextKey.phase(), nextKey.action(), nextKey.name() + randomAlphaOfLength(5));
             case 2 -> fixedInterval = randomValueOtherThan(instance.getFixedInterval(), ConfigTestHelpers::randomInterval);
+            case 3 -> timeout = randomValueOtherThan(
+                instance.getWaitTimeout(),
+                () -> TimeValue.parseTimeValue(randomTimeValue(1, 1000, "d", "h", "ms", "s", "m"), "timeout")
+            );
             default -> throw new AssertionError("Illegal randomisation branch");
         }
 
-        return new DownsampleStep(key, nextKey, null, instance.getClient(), fixedInterval);
+        return new DownsampleStep(key, nextKey, instance.getClient(), fixedInterval, timeout);
     }
 
     @Override
     public DownsampleStep copyInstance(DownsampleStep instance) {
-        return new DownsampleStep(instance.getKey(), instance.getNextStepKey(), null, instance.getClient(), instance.getFixedInterval());
+        return new DownsampleStep(
+            instance.getKey(),
+            instance.getNextStepKey(),
+            instance.getClient(),
+            instance.getFixedInterval(),
+            instance.getWaitTimeout()
+        );
     }
 
     private IndexMetadata getIndexMetadata(String index, String lifecycleName, DownsampleStep step) {
@@ -71,7 +89,6 @@ public class DownsampleStepTests extends AbstractStepTestCase<DownsampleStep> {
         lifecycleState.setAction(step.getKey().action());
         lifecycleState.setStep(step.getKey().name());
         lifecycleState.setIndexCreationDate(randomNonNegativeLong());
-        lifecycleState.setDownsampleIndexName("downsample-index");
 
         return IndexMetadata.builder(index)
             .settings(settings(IndexVersion.current()).put(LifecycleSettings.LIFECYCLE_NAME, lifecycleName))
@@ -84,7 +101,10 @@ public class DownsampleStepTests extends AbstractStepTestCase<DownsampleStep> {
     private static void assertDownsampleActionRequest(DownsampleAction.Request request, String sourceIndex) {
         assertNotNull(request);
         assertThat(request.getSourceIndex(), equalTo(sourceIndex));
-        assertThat(request.getTargetIndex(), equalTo("downsample-index"));
+        assertThat(
+            request.getTargetIndex(),
+            equalTo(DOWNSAMPLED_INDEX_PREFIX + sourceIndex + "-" + request.getDownsampleConfig().getFixedInterval())
+        );
     }
 
     public void testPerformAction() throws Exception {
@@ -150,59 +170,6 @@ public class DownsampleStepTests extends AbstractStepTestCase<DownsampleStep> {
         PlainActionFuture.<Void, Exception>get(f -> step.performAction(indexMetadata, clusterState, null, f));
     }
 
-    /**
-     * Test downsample step when a successfully completed downsample index already exists.
-     */
-    public void testPerformActionCompletedDownsampleIndexExists() {
-        String sourceIndexName = randomAlphaOfLength(10);
-        String lifecycleName = randomAlphaOfLength(5);
-        DownsampleStep step = createRandomInstance();
-
-        LifecycleExecutionState.Builder lifecycleState = LifecycleExecutionState.builder();
-        lifecycleState.setPhase(step.getKey().phase());
-        lifecycleState.setAction(step.getKey().action());
-        lifecycleState.setStep(step.getKey().name());
-        lifecycleState.setIndexCreationDate(randomNonNegativeLong());
-
-        String downsampleIndex = generateValidIndexName(DOWNSAMPLED_INDEX_PREFIX, sourceIndexName);
-        lifecycleState.setDownsampleIndexName(downsampleIndex);
-
-        IndexMetadata sourceIndexMetadata = IndexMetadata.builder(sourceIndexName)
-            .settings(settings(IndexVersion.current()).put(LifecycleSettings.LIFECYCLE_NAME, lifecycleName))
-            .putCustom(ILM_CUSTOM_METADATA_KEY, lifecycleState.build().asMap())
-            .numberOfShards(randomIntBetween(1, 5))
-            .numberOfReplicas(randomIntBetween(0, 5))
-            .build();
-
-        // Create a successfully completed downsample index (index.downsample.status: success)
-        IndexMetadata indexMetadata = IndexMetadata.builder(downsampleIndex)
-            .settings(
-                settings(IndexVersion.current()).put(
-                    IndexMetadata.INDEX_DOWNSAMPLE_STATUS.getKey(),
-                    IndexMetadata.DownsampleTaskStatus.SUCCESS
-                )
-            )
-            .numberOfShards(1)
-            .numberOfReplicas(0)
-            .build();
-        Map<String, IndexMetadata> indices = Map.of(downsampleIndex, indexMetadata);
-        ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metadata(Metadata.builder().indices(indices)).build();
-
-        Mockito.doThrow(new IllegalStateException("Downsample action should not be invoked"))
-            .when(client)
-            .execute(Mockito.any(), Mockito.any(), Mockito.any());
-
-        step.performAction(sourceIndexMetadata, clusterState, null, new ActionListener<>() {
-            @Override
-            public void onResponse(Void unused) {}
-
-            @Override
-            public void onFailure(Exception e) {
-                fail("onFailure should not be called in this test, called with exception: " + e.getMessage());
-            }
-        });
-    }
-
     /**
      * Test downsample step when an in-progress downsample index already exists.
      */
@@ -277,36 +244,24 @@ public class DownsampleStepTests extends AbstractStepTestCase<DownsampleStep> {
             .build();
         {
             try (NoOpClient client = new NoOpClient(getTestName())) {
-                StepKey nextKeyOnComplete = randomStepKey();
-                StepKey nextKeyOnIncomplete = randomStepKey();
+                StepKey nextKey = randomStepKey();
                 DateHistogramInterval fixedInterval = ConfigTestHelpers.randomInterval();
-                DownsampleStep completeStep = new DownsampleStep(
-                    randomStepKey(),
-                    nextKeyOnComplete,
-                    nextKeyOnIncomplete,
-                    client,
-                    fixedInterval
-                ) {
+                TimeValue timeout = DownsampleAction.DEFAULT_WAIT_TIMEOUT;
+                DownsampleStep completeStep = new DownsampleStep(randomStepKey(), nextKey, client, fixedInterval, timeout) {
                     void performDownsampleIndex(String indexName, String downsampleIndexName, ActionListener<Void> listener) {
                         listener.onResponse(null);
                     }
                 };
                 completeStep.performAction(sourceIndexMetadata, clusterState, null, ActionListener.noop());
-                assertThat(completeStep.getNextStepKey(), is(nextKeyOnComplete));
+                assertThat(completeStep.getNextStepKey(), is(nextKey));
             }
         }
         {
             try (NoOpClient client = new NoOpClient(getTestName())) {
-                StepKey nextKeyOnComplete = randomStepKey();
-                StepKey nextKeyOnIncomplete = randomStepKey();
+                StepKey nextKey = randomStepKey();
                 DateHistogramInterval fixedInterval = ConfigTestHelpers.randomInterval();
-                DownsampleStep doubleInvocationStep = new DownsampleStep(
-                    randomStepKey(),
-                    nextKeyOnComplete,
-                    nextKeyOnIncomplete,
-                    client,
-                    fixedInterval
-                ) {
+                TimeValue timeout = DownsampleAction.DEFAULT_WAIT_TIMEOUT;
+                DownsampleStep doubleInvocationStep = new DownsampleStep(randomStepKey(), nextKey, client, fixedInterval, timeout) {
                     void performDownsampleIndex(String indexName, String downsampleIndexName, ActionListener<Void> listener) {
                         listener.onFailure(
                             new IllegalStateException(
@@ -326,7 +281,7 @@ public class DownsampleStepTests extends AbstractStepTestCase<DownsampleStep> {
                     }
                 };
                 doubleInvocationStep.performAction(sourceIndexMetadata, clusterState, null, ActionListener.noop());
-                assertThat(doubleInvocationStep.getNextStepKey(), is(nextKeyOnIncomplete));
+                assertThat(doubleInvocationStep.getNextStepKey(), is(nextKey));
             }
         }
     }

+ 12 - 10
x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/TimeseriesLifecycleTypeTests.java

@@ -19,6 +19,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 import java.util.function.Consumer;
 import java.util.function.Function;
 import java.util.stream.Collectors;
@@ -81,7 +82,8 @@ public class TimeseriesLifecycleTypeTests extends ESTestCase {
     // keeping the migrate action disabled as otherwise it could conflict with the allocate action if both are randomly selected for the
     // same phase
     private static final MigrateAction TEST_MIGRATE_ACTION = MigrateAction.DISABLED;
-    private static final DownsampleAction TEST_DOWNSAMPLE_ACTION = new DownsampleAction(DateHistogramInterval.DAY);
+    public static final TimeValue TIMEOUT = new TimeValue(1, TimeUnit.MINUTES);
+    private static final DownsampleAction TEST_DOWNSAMPLE_ACTION = new DownsampleAction(DateHistogramInterval.DAY, TIMEOUT);
 
     public void testValidatePhases() {
         boolean invalid = randomBoolean();
@@ -365,12 +367,12 @@ public class TimeseriesLifecycleTypeTests extends ESTestCase {
             Phase warmPhase = new Phase(
                 "warm",
                 TimeValue.ZERO,
-                Map.of(DownsampleAction.NAME, new DownsampleAction(DateHistogramInterval.hours(1)))
+                Map.of(DownsampleAction.NAME, new DownsampleAction(DateHistogramInterval.hours(1), TIMEOUT))
             );
             Phase coldPhase = new Phase(
                 "cold",
                 TimeValue.ZERO,
-                Map.of(DownsampleAction.NAME, new DownsampleAction(DateHistogramInterval.hours(1)))
+                Map.of(DownsampleAction.NAME, new DownsampleAction(DateHistogramInterval.hours(1), TIMEOUT))
             );
 
             IllegalArgumentException e = expectThrows(
@@ -387,12 +389,12 @@ public class TimeseriesLifecycleTypeTests extends ESTestCase {
             Phase warmPhase = new Phase(
                 "warm",
                 TimeValue.ZERO,
-                Map.of(DownsampleAction.NAME, new DownsampleAction(DateHistogramInterval.hours(1)))
+                Map.of(DownsampleAction.NAME, new DownsampleAction(DateHistogramInterval.hours(1), TIMEOUT))
             );
             Phase coldPhase = new Phase(
                 "cold",
                 TimeValue.ZERO,
-                Map.of(DownsampleAction.NAME, new DownsampleAction(DateHistogramInterval.minutes(30)))
+                Map.of(DownsampleAction.NAME, new DownsampleAction(DateHistogramInterval.minutes(30), TIMEOUT))
             );
 
             IllegalArgumentException e = expectThrows(
@@ -409,12 +411,12 @@ public class TimeseriesLifecycleTypeTests extends ESTestCase {
             Phase warmPhase = new Phase(
                 "warm",
                 TimeValue.ZERO,
-                Map.of(DownsampleAction.NAME, new DownsampleAction(DateHistogramInterval.hours(1)))
+                Map.of(DownsampleAction.NAME, new DownsampleAction(DateHistogramInterval.hours(1), TIMEOUT))
             );
             Phase coldPhase = new Phase(
                 "cold",
                 TimeValue.ZERO,
-                Map.of(DownsampleAction.NAME, new DownsampleAction(DateHistogramInterval.minutes(130)))
+                Map.of(DownsampleAction.NAME, new DownsampleAction(DateHistogramInterval.minutes(130), TIMEOUT))
             );
 
             IllegalArgumentException e = expectThrows(
@@ -435,18 +437,18 @@ public class TimeseriesLifecycleTypeTests extends ESTestCase {
                     RolloverAction.NAME,
                     TEST_ROLLOVER_ACTION,
                     DownsampleAction.NAME,
-                    new DownsampleAction(DateHistogramInterval.minutes(10))
+                    new DownsampleAction(DateHistogramInterval.minutes(10), TIMEOUT)
                 )
             );
             Phase warmPhase = new Phase(
                 "warm",
                 TimeValue.ZERO,
-                Map.of(DownsampleAction.NAME, new DownsampleAction(DateHistogramInterval.minutes(30)))
+                Map.of(DownsampleAction.NAME, new DownsampleAction(DateHistogramInterval.minutes(30), TIMEOUT))
             );
             Phase coldPhase = new Phase(
                 "cold",
                 TimeValue.ZERO,
-                Map.of(DownsampleAction.NAME, new DownsampleAction(DateHistogramInterval.hours(2)))
+                Map.of(DownsampleAction.NAME, new DownsampleAction(DateHistogramInterval.hours(2), TIMEOUT))
             );
 
             // This is a valid interval combination

+ 67 - 0
x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/RollupShardPersistentTaskStateTests.java

@@ -0,0 +1,67 @@
+/*
+ * 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.rollup;
+
+import org.apache.lucene.util.BytesRef;
+import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
+import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.persistent.PersistentTaskState;
+import org.elasticsearch.test.AbstractXContentSerializingTestCase;
+import org.elasticsearch.xcontent.XContentParser;
+import org.elasticsearch.xpack.core.rollup.action.RollupShardIndexerStatus;
+import org.elasticsearch.xpack.core.rollup.action.RollupShardPersistentTaskState;
+
+import java.io.IOException;
+import java.util.List;
+
+public class RollupShardPersistentTaskStateTests extends AbstractXContentSerializingTestCase<RollupShardPersistentTaskState> {
+
+    @Override
+    protected NamedWriteableRegistry getNamedWriteableRegistry() {
+        return new NamedWriteableRegistry(
+            List.of(
+                new NamedWriteableRegistry.Entry(
+                    PersistentTaskState.class,
+                    RollupShardPersistentTaskState.NAME,
+                    RollupShardPersistentTaskState::readFromStream
+                )
+            )
+        );
+    }
+
+    @Override
+    protected Writeable.Reader<RollupShardPersistentTaskState> instanceReader() {
+        return RollupShardPersistentTaskState::new;
+    }
+
+    @Override
+    protected RollupShardPersistentTaskState createTestInstance() {
+        try {
+            return createRollupShardPersistentTaskState();
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private static RollupShardPersistentTaskState createRollupShardPersistentTaskState() throws IOException {
+        return new RollupShardPersistentTaskState(
+            randomFrom(RollupShardIndexerStatus.values()),
+            new BytesRef(randomAlphaOfLengthBetween(10, 100))
+        );
+    }
+
+    @Override
+    protected RollupShardPersistentTaskState mutateInstance(RollupShardPersistentTaskState instance) throws IOException {
+        return null; // TODO implement https://github.com/elastic/elasticsearch/issues/25929
+    }
+
+    @Override
+    protected RollupShardPersistentTaskState doParseInstance(XContentParser parser) throws IOException {
+        return RollupShardPersistentTaskState.fromXContent(parser);
+    }
+}

+ 32 - 16
x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/DownsampleActionIT.java

@@ -22,6 +22,7 @@ import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.index.IndexMode;
 import org.elasticsearch.index.IndexSettings;
 import org.elasticsearch.rest.action.admin.indices.RestPutIndexTemplateAction;
+import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
 import org.elasticsearch.test.rest.ESRestTestCase;
 import org.elasticsearch.xcontent.XContentBuilder;
 import org.elasticsearch.xcontent.XContentFactory;
@@ -152,12 +153,13 @@ public class DownsampleActionIT extends ESRestTestCase {
         index(client(), index, true, null, "@timestamp", "2020-01-01T05:10:00Z", "volume", 11.0, "metricset", randomAlphaOfLength(5));
 
         String phaseName = randomFrom("warm", "cold");
-        createNewSingletonPolicy(client(), policy, phaseName, new DownsampleAction(ConfigTestHelpers.randomInterval()));
+        DateHistogramInterval fixedInterval = ConfigTestHelpers.randomInterval();
+        createNewSingletonPolicy(client(), policy, phaseName, new DownsampleAction(fixedInterval, DownsampleAction.DEFAULT_WAIT_TIMEOUT));
         updatePolicy(client(), index, policy);
 
-        String rollupIndex = waitAndGetRollupIndexName(client(), index);
+        String rollupIndex = waitAndGetRollupIndexName(client(), index, fixedInterval);
         assertNotNull("Cannot retrieve rollup index name", rollupIndex);
-        assertBusy(() -> assertTrue("Rollup index does not exist", indexExists(rollupIndex)), 30, TimeUnit.SECONDS);
+        assertBusy(() -> assertTrue("Rollup index does exist", indexExists(rollupIndex)), 30, TimeUnit.SECONDS);
         assertBusy(() -> assertFalse("Source index should have been deleted", indexExists(index)), 30, TimeUnit.SECONDS);
         assertBusy(
             () -> assertThat(getStepKeyForIndex(client(), rollupIndex), equalTo(PhaseCompleteStep.finalStep(phaseName).getKey())),
@@ -181,7 +183,12 @@ public class DownsampleActionIT extends ESRestTestCase {
 
         ResponseException e = expectThrows(
             ResponseException.class,
-            () -> createNewSingletonPolicy(client(), policy, "hot", new DownsampleAction(ConfigTestHelpers.randomInterval()))
+            () -> createNewSingletonPolicy(
+                client(),
+                policy,
+                "hot",
+                new DownsampleAction(ConfigTestHelpers.randomInterval(), DownsampleAction.DEFAULT_WAIT_TIMEOUT)
+            )
         );
         assertTrue(
             e.getMessage()
@@ -193,11 +200,12 @@ public class DownsampleActionIT extends ESRestTestCase {
         String originalIndex = index + "-000001";
 
         // add a policy
+        DateHistogramInterval fixedInterval = ConfigTestHelpers.randomInterval();
         Map<String, LifecycleAction> hotActions = Map.of(
             RolloverAction.NAME,
             new RolloverAction(null, null, null, 1L, null, null, null, null, null, null),
             DownsampleAction.NAME,
-            new DownsampleAction(ConfigTestHelpers.randomInterval())
+            new DownsampleAction(fixedInterval, DownsampleAction.DEFAULT_WAIT_TIMEOUT)
         );
         Map<String, Phase> phases = Map.of("hot", new Phase("hot", TimeValue.ZERO, hotActions));
         LifecyclePolicy lifecyclePolicy = new LifecyclePolicy(policy, phases);
@@ -235,7 +243,7 @@ public class DownsampleActionIT extends ESRestTestCase {
             randomAlphaOfLength(5)
         );
 
-        String rollupIndex = waitAndGetRollupIndexName(client(), originalIndex);
+        String rollupIndex = waitAndGetRollupIndexName(client(), originalIndex, fixedInterval);
         assertNotNull("Cannot retrieve rollup index name", rollupIndex);
         assertBusy(() -> assertTrue("Rollup index does not exist", indexExists(rollupIndex)), 30, TimeUnit.SECONDS);
         assertBusy(() -> assertFalse("Source index should have been deleted", indexExists(originalIndex)), 30, TimeUnit.SECONDS);
@@ -254,7 +262,8 @@ public class DownsampleActionIT extends ESRestTestCase {
 
     public void testTsdbDataStreams() throws Exception {
         // Create the ILM policy
-        createNewSingletonPolicy(client(), policy, "warm", new DownsampleAction(ConfigTestHelpers.randomInterval()));
+        DateHistogramInterval fixedInterval = ConfigTestHelpers.randomInterval();
+        createNewSingletonPolicy(client(), policy, "warm", new DownsampleAction(fixedInterval, DownsampleAction.DEFAULT_WAIT_TIMEOUT));
 
         // Create a template
         Request createIndexTemplateRequest = new Request("POST", "/_index_template/" + dataStream);
@@ -278,7 +287,7 @@ public class DownsampleActionIT extends ESRestTestCase {
         // Manual rollover the original index such that it's not the write index in the data stream anymore
         rolloverMaxOneDocCondition(client(), dataStream);
 
-        String rollupIndex = waitAndGetRollupIndexName(client(), backingIndexName);
+        String rollupIndex = waitAndGetRollupIndexName(client(), backingIndexName, fixedInterval);
         assertNotNull("Cannot retrieve rollup index name", rollupIndex);
         assertBusy(() -> assertTrue("Rollup index does not exist", indexExists(rollupIndex)), 30, TimeUnit.SECONDS);
         assertBusy(() -> assertFalse("Source index should have been deleted", indexExists(backingIndexName)), 30, TimeUnit.SECONDS);
@@ -295,11 +304,12 @@ public class DownsampleActionIT extends ESRestTestCase {
         index(client(), index, true, null, "@timestamp", "2020-01-01T05:10:00Z", "volume", 11.0, "metricset", randomAlphaOfLength(5));
 
         String phaseName = randomFrom("warm", "cold");
-        createNewSingletonPolicy(client(), policy, phaseName, new DownsampleAction(ConfigTestHelpers.randomInterval()));
+        DateHistogramInterval fixedInterval = ConfigTestHelpers.randomInterval();
+        createNewSingletonPolicy(client(), policy, phaseName, new DownsampleAction(fixedInterval, DownsampleAction.DEFAULT_WAIT_TIMEOUT));
         updatePolicy(client(), index, policy);
 
         assertBusy(() -> assertThat(getStepKeyForIndex(client(), index), equalTo(PhaseCompleteStep.finalStep(phaseName).getKey())));
-        String rollupIndex = getRollupIndexName(client(), index);
+        String rollupIndex = getRollupIndexName(client(), index, fixedInterval);
         assertNull("Rollup index should not have been created", rollupIndex);
         assertTrue("Source index should not have been deleted", indexExists(index));
     }
@@ -310,11 +320,12 @@ public class DownsampleActionIT extends ESRestTestCase {
      * @param originalIndexName the name of the source index used to generate the rollup index name
      * @return the name of the rollup index for a given index, null if none exist
      */
-    public String waitAndGetRollupIndexName(RestClient client, String originalIndexName) throws InterruptedException {
+    public String waitAndGetRollupIndexName(RestClient client, String originalIndexName, DateHistogramInterval fixedInterval)
+        throws InterruptedException {
         final String[] rollupIndexName = new String[1];
         waitUntil(() -> {
             try {
-                rollupIndexName[0] = getRollupIndexName(client, originalIndexName);
+                rollupIndexName[0] = getRollupIndexName(client, originalIndexName, fixedInterval);
                 return rollupIndexName[0] != null;
             } catch (IOException e) {
                 return false;
@@ -324,10 +335,15 @@ public class DownsampleActionIT extends ESRestTestCase {
         return rollupIndexName[0];
     }
 
-    public static String getRollupIndexName(RestClient client, String originalIndexName) throws IOException {
-        Response response = client.performRequest(
-            new Request("GET", "/" + DownsampleAction.DOWNSAMPLED_INDEX_PREFIX + "*-" + originalIndexName + "/?expand_wildcards=all")
-        );
+    public static String getRollupIndexName(RestClient client, String originalIndexName, DateHistogramInterval fixedInterval)
+        throws IOException {
+        String endpoint = "/"
+            + DownsampleAction.DOWNSAMPLED_INDEX_PREFIX
+            + originalIndexName
+            + "-"
+            + fixedInterval
+            + "*/?expand_wildcards=all";
+        Response response = client.performRequest(new Request("GET", endpoint));
         Map<String, Object> asMap = responseAsMap(response);
         if (asMap.size() == 1) {
             return (String) asMap.keySet().toArray()[0];

+ 33 - 0
x-pack/plugin/ilm/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/ilm/70_downsampling.yml

@@ -62,3 +62,36 @@ setup:
               }
             }
           }
+
+---
+"Test downsample timeout parameter":
+  - skip:
+      version: " - 8.9.99"
+      reason: "Downsample timeout added in version 8.10.0"
+
+  - do:
+      ilm.put_lifecycle:
+        policy: "timeout_policy"
+        body: |
+          {
+            "policy": {
+              "phases": {
+                "warm": {
+                  "min_age": "1d",
+                  "actions": {
+                    "downsample": {
+                      "fixed_interval": "1h",
+                      "wait_timeout": "1h"
+                    }
+                  }
+                }
+              }
+            }
+          }
+  - do:
+      ilm.get_lifecycle:
+        policy: "timeout_policy"
+
+  - match: { timeout_policy.policy.phases.warm.min_age: "1d" }
+  - match: { timeout_policy.policy.phases.warm.actions.downsample.fixed_interval: "1h" }
+  - match: { timeout_policy.policy.phases.warm.actions.downsample.wait_timeout: "1h" }

+ 3 - 0
x-pack/plugin/rollup/build.gradle

@@ -1,4 +1,5 @@
 apply plugin: 'elasticsearch.internal-es-plugin'
+apply plugin: 'elasticsearch.internal-cluster-test'
 esplugin {
   name 'x-pack-rollup'
   description 'Elasticsearch Expanded Pack Plugin - Rollup'
@@ -11,11 +12,13 @@ base {
 }
 
 dependencies {
+  testImplementation project(path: ':test:framework')
   compileOnly project(path: xpackModule('core'))
   testImplementation project(':modules:data-streams')
   testImplementation project(path: xpackModule('ilm'))
   compileOnly project(path: xpackModule('mapper-aggregate-metric'))
   testImplementation(testArtifact(project(xpackModule('core'))))
+  testImplementation project(xpackModule('ccr'))
 }
 
 addQaCheckDependencies(project)

+ 442 - 0
x-pack/plugin/rollup/src/internalClusterTest/java/org/elasticsearch/xpack/downsample/DownsampleClusterDisruptionIT.java

@@ -0,0 +1,442 @@
+/*
+ * 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.downsample;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.elasticsearch.action.DocWriteRequest;
+import org.elasticsearch.action.admin.indices.get.GetIndexRequest;
+import org.elasticsearch.action.admin.indices.get.GetIndexResponse;
+import org.elasticsearch.action.bulk.BulkItemResponse;
+import org.elasticsearch.action.bulk.BulkRequestBuilder;
+import org.elasticsearch.action.bulk.BulkResponse;
+import org.elasticsearch.action.downsample.DownsampleConfig;
+import org.elasticsearch.action.index.IndexRequest;
+import org.elasticsearch.action.search.SearchResponse;
+import org.elasticsearch.action.support.WriteRequest;
+import org.elasticsearch.cluster.metadata.IndexMetadata;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.time.DateFormatter;
+import org.elasticsearch.core.TimeValue;
+import org.elasticsearch.index.IndexMode;
+import org.elasticsearch.index.IndexSettings;
+import org.elasticsearch.index.engine.VersionConflictEngineException;
+import org.elasticsearch.index.mapper.DateFieldMapper;
+import org.elasticsearch.index.query.MatchAllQueryBuilder;
+import org.elasticsearch.plugins.Plugin;
+import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
+import org.elasticsearch.test.ESIntegTestCase;
+import org.elasticsearch.test.InternalTestCluster;
+import org.elasticsearch.xcontent.XContentBuilder;
+import org.elasticsearch.xcontent.XContentFactory;
+import org.elasticsearch.xpack.aggregatemetric.AggregateMetricMapperPlugin;
+import org.elasticsearch.xpack.core.LocalStateCompositeXPackPlugin;
+import org.elasticsearch.xpack.core.downsample.DownsampleAction;
+import org.elasticsearch.xpack.rollup.Rollup;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Locale;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
+import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder;
+import static org.elasticsearch.xpack.core.rollup.ConfigTestHelpers.randomInterval;
+
+@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, numClientNodes = 4)
+public class DownsampleClusterDisruptionIT extends ESIntegTestCase {
+    private static final Logger logger = LogManager.getLogger(DownsampleClusterDisruptionIT.class);
+    private static final DateFormatter DATE_FORMATTER = DateFormatter.forPattern("yyyy-MM-dd'T'HH:mm:ss.SSSXXX");
+    private static final TimeValue TIMEOUT = new TimeValue(1, TimeUnit.MINUTES);
+    public static final String FIELD_TIMESTAMP = "@timestamp";
+    public static final String FIELD_DIMENSION_1 = "dimension_kw";
+    public static final String FIELD_DIMENSION_2 = "dimension_long";
+    public static final String FIELD_METRIC_COUNTER = "counter";
+    public static final int DOC_COUNT = 10_000;
+
+    @Override
+    protected Collection<Class<? extends Plugin>> nodePlugins() {
+        return List.of(LocalStateCompositeXPackPlugin.class, Rollup.class, AggregateMetricMapperPlugin.class);
+    }
+
+    interface DisruptionListener {
+        void disruptionStart();
+
+        void disruptionEnd();
+    }
+
+    private class Disruptor implements Runnable {
+        final InternalTestCluster cluster;
+        private final String sourceIndex;
+        private final DisruptionListener listener;
+        private final String clientNode;
+        private final Consumer<String> disruption;
+
+        private Disruptor(
+            final InternalTestCluster cluster,
+            final String sourceIndex,
+            final DisruptionListener listener,
+            final String clientNode,
+            final Consumer<String> disruption
+        ) {
+            this.cluster = cluster;
+            this.sourceIndex = sourceIndex;
+            this.listener = listener;
+            this.clientNode = clientNode;
+            this.disruption = disruption;
+        }
+
+        @Override
+        public void run() {
+            listener.disruptionStart();
+            try {
+                final String candidateNode = cluster.client(clientNode)
+                    .admin()
+                    .cluster()
+                    .prepareSearchShards(sourceIndex)
+                    .get()
+                    .getNodes()[0].getName();
+                logger.info("Candidate node [" + candidateNode + "]");
+                disruption.accept(candidateNode);
+                ensureGreen(sourceIndex);
+                ensureStableCluster(cluster.numDataAndMasterNodes(), clientNode);
+
+            } catch (Exception e) {
+                logger.error("Ignoring Error while injecting disruption [" + e.getMessage() + "]");
+            } finally {
+                listener.disruptionEnd();
+            }
+        }
+    }
+
+    public void setup(final String sourceIndex, int numOfShards, int numOfReplicas, long startTime) throws IOException {
+        final Settings.Builder settings = indexSettings(numOfShards, numOfReplicas).put(IndexSettings.MODE.getKey(), IndexMode.TIME_SERIES)
+            .putList(IndexMetadata.INDEX_ROUTING_PATH.getKey(), List.of(FIELD_DIMENSION_1))
+            .put(
+                IndexSettings.TIME_SERIES_START_TIME.getKey(),
+                DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.formatMillis(Instant.ofEpochMilli(startTime).toEpochMilli())
+            )
+            .put(IndexSettings.TIME_SERIES_END_TIME.getKey(), "2106-01-08T23:40:53.384Z");
+
+        if (randomBoolean()) {
+            settings.put(IndexMetadata.SETTING_INDEX_HIDDEN, randomBoolean());
+        }
+
+        final XContentBuilder mapping = jsonBuilder().startObject().startObject("_doc").startObject("properties");
+        mapping.startObject(FIELD_TIMESTAMP).field("type", "date").endObject();
+
+        mapping.startObject(FIELD_DIMENSION_1).field("type", "keyword").field("time_series_dimension", true).endObject();
+        mapping.startObject(FIELD_DIMENSION_2).field("type", "long").field("time_series_dimension", true).endObject();
+
+        mapping.startObject(FIELD_METRIC_COUNTER)
+            .field("type", "double") /* numeric label indexed as a metric */
+            .field("time_series_metric", "counter")
+            .endObject();
+
+        mapping.endObject().endObject().endObject();
+        assertAcked(indicesAdmin().prepareCreate(sourceIndex).setSettings(settings.build()).setMapping(mapping).get());
+    }
+
+    public void testDownsampleIndexWithDataNodeRestart() throws Exception {
+        try (InternalTestCluster cluster = internalCluster()) {
+            final List<String> masterNodes = cluster.startMasterOnlyNodes(1);
+            cluster.startDataOnlyNodes(3);
+            ensureStableCluster(cluster.size());
+            ensureGreen();
+
+            final String sourceIndex = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
+            final String targetIndex = randomAlphaOfLength(11).toLowerCase(Locale.ROOT);
+            long startTime = LocalDateTime.parse("2020-09-09T18:00:00").atZone(ZoneId.of("UTC")).toInstant().toEpochMilli();
+            setup(sourceIndex, 1, 0, startTime);
+            final DownsampleConfig config = new DownsampleConfig(randomInterval());
+            final DownsampleActionSingleNodeTests.SourceSupplier sourceSupplier = () -> {
+                final String ts = randomDateForInterval(config.getInterval(), startTime);
+                double counterValue = DATE_FORMATTER.parseMillis(ts);
+                final List<String> dimensionValues = new ArrayList<>(5);
+                for (int j = 0; j < randomIntBetween(1, 5); j++) {
+                    dimensionValues.add(randomAlphaOfLength(6));
+                }
+                return XContentFactory.jsonBuilder()
+                    .startObject()
+                    .field(FIELD_TIMESTAMP, ts)
+                    .field(FIELD_DIMENSION_1, randomFrom(dimensionValues))
+                    .field(FIELD_DIMENSION_2, randomIntBetween(1, 10))
+                    .field(FIELD_METRIC_COUNTER, counterValue)
+                    .endObject();
+            };
+            int indexedDocs = bulkIndex(sourceIndex, sourceSupplier, DOC_COUNT);
+            prepareSourceIndex(sourceIndex);
+            final CountDownLatch disruptionStart = new CountDownLatch(1);
+            final CountDownLatch disruptionEnd = new CountDownLatch(1);
+
+            new Thread(new Disruptor(cluster, sourceIndex, new DisruptionListener() {
+                @Override
+                public void disruptionStart() {
+                    disruptionStart.countDown();
+                }
+
+                @Override
+                public void disruptionEnd() {
+                    disruptionEnd.countDown();
+                }
+            }, masterNodes.get(0), (node) -> {
+                try {
+                    cluster.restartNode(node, new InternalTestCluster.RestartCallback() {
+                        @Override
+                        public boolean validateClusterForming() {
+                            return true;
+                        }
+                    });
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+            })).start();
+            startDownsampleTaskDuringDisruption(sourceIndex, targetIndex, config, disruptionStart, disruptionEnd);
+            waitUntil(() -> cluster.client().admin().cluster().preparePendingClusterTasks().get().pendingTasks().isEmpty());
+            ensureStableCluster(cluster.numDataAndMasterNodes());
+            assertTargetIndex(cluster, sourceIndex, targetIndex, indexedDocs);
+        }
+    }
+
+    public void testDownsampleIndexWithRollingRestart() throws Exception {
+        try (InternalTestCluster cluster = internalCluster()) {
+            final List<String> masterNodes = cluster.startMasterOnlyNodes(1);
+            cluster.startDataOnlyNodes(3);
+            ensureStableCluster(cluster.size());
+            ensureGreen();
+
+            final String sourceIndex = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
+            final String targetIndex = randomAlphaOfLength(11).toLowerCase(Locale.ROOT);
+            long startTime = LocalDateTime.parse("2020-09-09T18:00:00").atZone(ZoneId.of("UTC")).toInstant().toEpochMilli();
+            setup(sourceIndex, 1, 0, startTime);
+            final DownsampleConfig config = new DownsampleConfig(randomInterval());
+            final DownsampleActionSingleNodeTests.SourceSupplier sourceSupplier = () -> {
+                final String ts = randomDateForInterval(config.getInterval(), startTime);
+                double counterValue = DATE_FORMATTER.parseMillis(ts);
+                final List<String> dimensionValues = new ArrayList<>(5);
+                for (int j = 0; j < randomIntBetween(1, 5); j++) {
+                    dimensionValues.add(randomAlphaOfLength(6));
+                }
+                return XContentFactory.jsonBuilder()
+                    .startObject()
+                    .field(FIELD_TIMESTAMP, ts)
+                    .field(FIELD_DIMENSION_1, randomFrom(dimensionValues))
+                    .field(FIELD_DIMENSION_2, randomIntBetween(1, 10))
+                    .field(FIELD_METRIC_COUNTER, counterValue)
+                    .endObject();
+            };
+            int indexedDocs = bulkIndex(sourceIndex, sourceSupplier, DOC_COUNT);
+            prepareSourceIndex(sourceIndex);
+            final CountDownLatch disruptionStart = new CountDownLatch(1);
+            final CountDownLatch disruptionEnd = new CountDownLatch(1);
+
+            new Thread(new Disruptor(cluster, sourceIndex, new DisruptionListener() {
+                @Override
+                public void disruptionStart() {
+                    disruptionStart.countDown();
+                }
+
+                @Override
+                public void disruptionEnd() {
+                    disruptionEnd.countDown();
+                }
+            }, masterNodes.get(0), (ignored) -> {
+                try {
+                    cluster.rollingRestart(new InternalTestCluster.RestartCallback() {
+                        @Override
+                        public boolean validateClusterForming() {
+                            return true;
+                        }
+                    });
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+            })).start();
+
+            startDownsampleTaskDuringDisruption(sourceIndex, targetIndex, config, disruptionStart, disruptionEnd);
+            waitUntil(() -> cluster.client().admin().cluster().preparePendingClusterTasks().get().pendingTasks().isEmpty());
+            ensureStableCluster(cluster.numDataAndMasterNodes());
+            assertTargetIndex(cluster, sourceIndex, targetIndex, indexedDocs);
+        }
+    }
+
+    /**
+     * Starts a downsample operation.
+     *
+     * @param sourceIndex the idex to read data from
+     * @param targetIndex the idnex to write downsampled data to
+     * @param config the downsample configuration including the downsample granularity
+     * @param disruptionStart a latch to synchronize on the disruption starting
+     * @param disruptionEnd a latch to synchronize on the disruption ending
+     * @throws InterruptedException if the thread is interrupted while waiting
+     */
+    private void startDownsampleTaskDuringDisruption(
+        final String sourceIndex,
+        final String targetIndex,
+        final DownsampleConfig config,
+        final CountDownLatch disruptionStart,
+        final CountDownLatch disruptionEnd
+    ) throws Exception {
+        disruptionStart.await();
+        assertBusy(() -> {
+            try {
+                downsample(sourceIndex, targetIndex, config);
+            } catch (Exception e) {
+                throw new AssertionError(e);
+            }
+        }, 60, TimeUnit.SECONDS);
+        disruptionEnd.await();
+    }
+
+    public void testDownsampleIndexWithFullClusterRestart() throws Exception {
+        try (InternalTestCluster cluster = internalCluster()) {
+            final List<String> masterNodes = cluster.startMasterOnlyNodes(1);
+            cluster.startDataOnlyNodes(3);
+            ensureStableCluster(cluster.size());
+            ensureGreen();
+
+            final String sourceIndex = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
+            final String downsampleIndex = randomAlphaOfLength(11).toLowerCase(Locale.ROOT);
+            long startTime = LocalDateTime.parse("2020-09-09T18:00:00").atZone(ZoneId.of("UTC")).toInstant().toEpochMilli();
+            setup(sourceIndex, 1, 0, startTime);
+            final DownsampleConfig config = new DownsampleConfig(randomInterval());
+            final DownsampleActionSingleNodeTests.SourceSupplier sourceSupplier = () -> {
+                final String ts = randomDateForInterval(config.getInterval(), startTime);
+                double counterValue = DATE_FORMATTER.parseMillis(ts);
+                final List<String> dimensionValues = new ArrayList<>(5);
+                for (int j = 0; j < randomIntBetween(1, 5); j++) {
+                    dimensionValues.add(randomAlphaOfLength(6));
+                }
+                return XContentFactory.jsonBuilder()
+                    .startObject()
+                    .field(FIELD_TIMESTAMP, ts)
+                    .field(FIELD_DIMENSION_1, randomFrom(dimensionValues))
+                    .field(FIELD_DIMENSION_2, randomIntBetween(1, 10))
+                    .field(FIELD_METRIC_COUNTER, counterValue)
+                    .endObject();
+            };
+            int indexedDocs = bulkIndex(sourceIndex, sourceSupplier, DOC_COUNT);
+            prepareSourceIndex(sourceIndex);
+            final CountDownLatch disruptionStart = new CountDownLatch(1);
+            final CountDownLatch disruptionEnd = new CountDownLatch(1);
+
+            new Thread(new Disruptor(cluster, sourceIndex, new DisruptionListener() {
+                @Override
+                public void disruptionStart() {
+                    disruptionStart.countDown();
+                }
+
+                @Override
+                public void disruptionEnd() {
+                    disruptionEnd.countDown();
+                }
+            }, masterNodes.get(0), (ignored) -> {
+                try {
+                    cluster.fullRestart(new InternalTestCluster.RestartCallback() {
+                        @Override
+                        public boolean validateClusterForming() {
+                            return true;
+                        }
+                    });
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+            })).start();
+
+            startDownsampleTaskDuringDisruption(sourceIndex, downsampleIndex, config, disruptionStart, disruptionEnd);
+            waitUntil(() -> cluster.client().admin().cluster().preparePendingClusterTasks().get().pendingTasks().isEmpty());
+            ensureStableCluster(cluster.numDataAndMasterNodes());
+            assertTargetIndex(cluster, sourceIndex, downsampleIndex, indexedDocs);
+        }
+    }
+
+    private void assertTargetIndex(final InternalTestCluster cluster, final String sourceIndex, final String targetIndex, int indexedDocs) {
+        final GetIndexResponse getIndexResponse = cluster.client()
+            .admin()
+            .indices()
+            .getIndex(new GetIndexRequest().indices(targetIndex))
+            .actionGet();
+        assertEquals(1, getIndexResponse.indices().length);
+        final SearchResponse sourceIndexSearch = cluster.client()
+            .prepareSearch(sourceIndex)
+            .setQuery(new MatchAllQueryBuilder())
+            .setSize(Math.min(DOC_COUNT, indexedDocs))
+            .setTrackTotalHitsUpTo(Integer.MAX_VALUE)
+            .get();
+        assertEquals(indexedDocs, sourceIndexSearch.getHits().getHits().length);
+        final SearchResponse targetIndexSearch = cluster.client()
+            .prepareSearch(targetIndex)
+            .setQuery(new MatchAllQueryBuilder())
+            .setSize(Math.min(DOC_COUNT, indexedDocs))
+            .setTrackTotalHitsUpTo(Integer.MAX_VALUE)
+            .get();
+        assertTrue(targetIndexSearch.getHits().getHits().length > 0);
+    }
+
+    private int bulkIndex(final String indexName, final DownsampleActionSingleNodeTests.SourceSupplier sourceSupplier, int docCount)
+        throws IOException {
+        BulkRequestBuilder bulkRequestBuilder = internalCluster().client().prepareBulk();
+        bulkRequestBuilder.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE);
+        for (int i = 0; i < docCount; i++) {
+            IndexRequest indexRequest = new IndexRequest(indexName).opType(DocWriteRequest.OpType.CREATE);
+            XContentBuilder source = sourceSupplier.get();
+            indexRequest.source(source);
+            bulkRequestBuilder.add(indexRequest);
+        }
+        BulkResponse bulkResponse = bulkRequestBuilder.get();
+        int duplicates = 0;
+        for (BulkItemResponse response : bulkResponse.getItems()) {
+            if (response.isFailed()) {
+                if (response.getFailure().getCause() instanceof VersionConflictEngineException) {
+                    // A duplicate event was created by random generator. We should not fail for this
+                    // reason.
+                    logger.debug("We tried to insert a duplicate: [{}]", response.getFailureMessage());
+                    duplicates++;
+                } else {
+                    fail("Failed to index data: " + bulkResponse.buildFailureMessage());
+                }
+            }
+        }
+        int docsIndexed = docCount - duplicates;
+        logger.info("Indexed [{}] documents. Dropped [{}] duplicates.", docsIndexed, duplicates);
+        return docsIndexed;
+    }
+
+    private void prepareSourceIndex(String sourceIndex) {
+        // Set the source index to read-only state
+        assertAcked(
+            indicesAdmin().prepareUpdateSettings(sourceIndex)
+                .setSettings(Settings.builder().put(IndexMetadata.INDEX_BLOCKS_WRITE_SETTING.getKey(), true).build())
+                .get()
+        );
+    }
+
+    private void downsample(final String sourceIndex, final String downsampleIndex, final DownsampleConfig config) {
+        assertAcked(
+            internalCluster().client()
+                .execute(DownsampleAction.INSTANCE, new DownsampleAction.Request(sourceIndex, downsampleIndex, TIMEOUT, config))
+                .actionGet()
+        );
+    }
+
+    private String randomDateForInterval(final DateHistogramInterval interval, final long startTime) {
+        long endTime = startTime + 10 * interval.estimateMillis();
+        return randomDateForRange(startTime, endTime);
+    }
+
+    private String randomDateForRange(long start, long end) {
+        return DATE_FORMATTER.formatMillis(randomLongBetween(start, end));
+    }
+}

+ 7 - 43
x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/downsample/DownsampleTransportFailureTests.java → x-pack/plugin/rollup/src/internalClusterTest/java/org/elasticsearch/xpack/downsample/DownsampleTransportFailureIT.java

@@ -33,11 +33,11 @@ import org.elasticsearch.test.InternalTestCluster;
 import org.elasticsearch.test.transport.MockTransportService;
 import org.elasticsearch.transport.TransportService;
 import org.elasticsearch.xcontent.XContentBuilder;
+import org.elasticsearch.xcontent.XContentFactory;
 import org.elasticsearch.xcontent.XContentType;
 import org.elasticsearch.xpack.aggregatemetric.AggregateMetricMapperPlugin;
 import org.elasticsearch.xpack.core.LocalStateCompositeXPackPlugin;
 import org.elasticsearch.xpack.core.downsample.DownsampleAction;
-import org.elasticsearch.xpack.core.downsample.DownsampleIndexerAction;
 import org.elasticsearch.xpack.rollup.Rollup;
 import org.junit.Before;
 
@@ -50,14 +50,11 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
-import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder;
-
 @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 2, numClientNodes = 1, supportsDedicatedMasters = false)
-public class DownsampleTransportFailureTests extends ESIntegTestCase {
-
-    public static final String ROLLUP_INDEXER_SHARD_ACTION = DownsampleIndexerAction.NAME + "[s]";
+public class DownsampleTransportFailureIT extends ESIntegTestCase {
 
     private static class TestClusterHelper {
         private final InternalTestCluster cluster;
@@ -137,6 +134,7 @@ public class DownsampleTransportFailureTests extends ESIntegTestCase {
     private static final int DOWNSAMPLE_ACTION_TIMEOUT_MILLIS = 10_000;
     private static final String SOURCE_INDEX_NAME = "source";
     private static final String TARGET_INDEX_NAME = "target";
+    private static final TimeValue WAIT_TIMEOUT = new TimeValue(1, TimeUnit.MINUTES);
     private long startTime;
     private long endTime;
     private TestClusterHelper testCluster;
@@ -209,7 +207,7 @@ public class DownsampleTransportFailureTests extends ESIntegTestCase {
     }
 
     public XContentBuilder indexMapping() throws IOException {
-        final XContentBuilder mapping = jsonBuilder().startObject().startObject("_doc").startObject("properties");
+        final XContentBuilder mapping = XContentFactory.jsonBuilder().startObject().startObject("_doc").startObject("properties");
         mapping.startObject("@timestamp").field("type", "date").endObject();
         mapping.startObject("dim1").field("type", "keyword").field("time_series_dimension", true).endObject();
         mapping.startObject("dim2").field("type", "long").field("time_series_dimension", true).endObject();
@@ -281,6 +279,7 @@ public class DownsampleTransportFailureTests extends ESIntegTestCase {
         final DownsampleAction.Request downsampleRequest = new DownsampleAction.Request(
             SOURCE_INDEX_NAME,
             TARGET_INDEX_NAME,
+            WAIT_TIMEOUT,
             new DownsampleConfig(DateHistogramInterval.MINUTE)
         );
 
@@ -307,6 +306,7 @@ public class DownsampleTransportFailureTests extends ESIntegTestCase {
         final DownsampleAction.Request downsampleRequest = new DownsampleAction.Request(
             SOURCE_INDEX_NAME,
             TARGET_INDEX_NAME,
+            WAIT_TIMEOUT,
             new DownsampleConfig(DateHistogramInterval.HOUR)
         );
 
@@ -339,40 +339,4 @@ public class DownsampleTransportFailureTests extends ESIntegTestCase {
         ensureStableCluster(testCluster.size());
         assertDownsampleFailure(testCluster.coordinatorName());
     }
-
-    public void testRollupIndexerActionExceptionDisruption() {
-        // GIVEN
-        final MockTransportService master = testCluster.masterMockTransportService();
-        final DownsampleAction.Request downsampleRequest = new DownsampleAction.Request(
-            SOURCE_INDEX_NAME,
-            TARGET_INDEX_NAME,
-            new DownsampleConfig(DateHistogramInterval.HOUR)
-        );
-
-        // WHEN (disruption)
-        testCluster.allMockTransportServices()
-            .forEach(
-                mockTransportService -> master.addSendBehavior(mockTransportService, (connection, requestId, action, request, options) -> {
-                    if (ROLLUP_INDEXER_SHARD_ACTION.equals(action)) {
-                        logger.info("Simulated disruption: node [" + connection.getNode().getName() + "] action [" + action + "]");
-                        throw new ElasticsearchException(
-                            "Simulated disruption: node [" + connection.getNode().getName() + "] action [" + action + "]"
-                        );
-                    }
-                    connection.sendRequest(requestId, action, request, options);
-                })
-            );
-
-        // THEN
-        expectThrows(
-            ElasticsearchException.class,
-            () -> testCluster.coordinatorClient()
-                .execute(DownsampleAction.INSTANCE, downsampleRequest)
-                .actionGet(TimeValue.timeValueMillis(DOWNSAMPLE_ACTION_TIMEOUT_MILLIS))
-        );
-
-        master.clearAllRules();
-        ensureStableCluster(testCluster.size());
-        assertDownsampleFailure(testCluster.coordinatorName());
-    }
 }

+ 331 - 0
x-pack/plugin/rollup/src/internalClusterTest/java/org/elasticsearch/xpack/downsample/ILMDownsampleDisruptionIT.java

@@ -0,0 +1,331 @@
+/*
+ * 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.downsample;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.elasticsearch.action.DocWriteRequest;
+import org.elasticsearch.action.admin.indices.get.GetIndexRequest;
+import org.elasticsearch.action.admin.indices.get.GetIndexResponse;
+import org.elasticsearch.action.admin.indices.settings.get.GetSettingsRequest;
+import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest;
+import org.elasticsearch.action.bulk.BulkItemResponse;
+import org.elasticsearch.action.bulk.BulkRequestBuilder;
+import org.elasticsearch.action.bulk.BulkResponse;
+import org.elasticsearch.action.downsample.DownsampleConfig;
+import org.elasticsearch.action.index.IndexRequest;
+import org.elasticsearch.action.search.SearchResponse;
+import org.elasticsearch.action.support.WriteRequest;
+import org.elasticsearch.cluster.metadata.IndexMetadata;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.time.DateFormatter;
+import org.elasticsearch.core.TimeValue;
+import org.elasticsearch.index.IndexMode;
+import org.elasticsearch.index.IndexSettings;
+import org.elasticsearch.index.engine.VersionConflictEngineException;
+import org.elasticsearch.index.mapper.DateFieldMapper;
+import org.elasticsearch.index.query.MatchAllQueryBuilder;
+import org.elasticsearch.plugins.Plugin;
+import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
+import org.elasticsearch.test.ESIntegTestCase;
+import org.elasticsearch.test.InternalTestCluster;
+import org.elasticsearch.xcontent.XContentBuilder;
+import org.elasticsearch.xcontent.XContentFactory;
+import org.elasticsearch.xpack.aggregatemetric.AggregateMetricMapperPlugin;
+import org.elasticsearch.xpack.ccr.Ccr;
+import org.elasticsearch.xpack.core.LocalStateCompositeXPackPlugin;
+import org.elasticsearch.xpack.core.ilm.LifecyclePolicy;
+import org.elasticsearch.xpack.core.ilm.LifecycleSettings;
+import org.elasticsearch.xpack.core.ilm.Phase;
+import org.elasticsearch.xpack.core.ilm.action.PutLifecycleAction;
+import org.elasticsearch.xpack.ilm.IndexLifecycle;
+import org.elasticsearch.xpack.rollup.Rollup;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
+import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder;
+import static org.elasticsearch.xpack.core.rollup.ConfigTestHelpers.randomInterval;
+import static org.hamcrest.Matchers.equalTo;
+
+@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, numClientNodes = 4)
+public class ILMDownsampleDisruptionIT extends ESIntegTestCase {
+    private static final Logger logger = LogManager.getLogger(ILMDownsampleDisruptionIT.class);
+    private static final DateFormatter DATE_FORMATTER = DateFormatter.forPattern("yyyy-MM-dd'T'HH:mm:ss.SSSXXX");
+    private static final String FIELD_TIMESTAMP = "@timestamp";
+    private static final String FIELD_DIMENSION_1 = "dimension_kw";
+    private static final String FIELD_DIMENSION_2 = "dimension_long";
+    private static final String FIELD_METRIC_COUNTER = "counter";
+    private static final String POLICY_NAME = "mypolicy";
+    public static final int DOC_COUNT = 10_000;
+
+    @Override
+    protected Collection<Class<? extends Plugin>> nodePlugins() {
+        return List.of(
+            LocalStateCompositeXPackPlugin.class,
+            Rollup.class,
+            AggregateMetricMapperPlugin.class,
+            LocalStateCompositeXPackPlugin.class,
+            IndexLifecycle.class,
+            Ccr.class
+        );
+    }
+
+    @Override
+    protected Settings nodeSettings(int nodeOrdinal, Settings otherSettings) {
+        Settings.Builder nodeSettings = Settings.builder().put(super.nodeSettings(nodeOrdinal, otherSettings));
+        nodeSettings.put(LifecycleSettings.LIFECYCLE_POLL_INTERVAL, "1s");
+
+        // This is necessary to prevent ILM installing a lifecycle policy, these tests assume a blank slate
+        nodeSettings.put(LifecycleSettings.LIFECYCLE_HISTORY_INDEX_ENABLED, false);
+        return nodeSettings.build();
+    }
+
+    public void setup(final String sourceIndex, int numOfShards, int numOfReplicas, long startTime) throws IOException {
+        final Settings.Builder settings = indexSettings(numOfShards, numOfReplicas).put(IndexSettings.MODE.getKey(), IndexMode.TIME_SERIES)
+            .putList(IndexMetadata.INDEX_ROUTING_PATH.getKey(), List.of(FIELD_DIMENSION_1))
+            .put(
+                IndexSettings.TIME_SERIES_START_TIME.getKey(),
+                DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.formatMillis(Instant.ofEpochMilli(startTime).toEpochMilli())
+            )
+            .put(IndexSettings.TIME_SERIES_END_TIME.getKey(), "2106-01-08T23:40:53.384Z");
+
+        if (randomBoolean()) {
+            settings.put(IndexMetadata.SETTING_INDEX_HIDDEN, randomBoolean());
+        }
+
+        final XContentBuilder mapping = jsonBuilder().startObject().startObject("_doc").startObject("properties");
+        mapping.startObject(FIELD_TIMESTAMP).field("type", "date").endObject();
+
+        mapping.startObject(FIELD_DIMENSION_1).field("type", "keyword").field("time_series_dimension", true).endObject();
+        mapping.startObject(FIELD_DIMENSION_2).field("type", "long").field("time_series_dimension", true).endObject();
+
+        mapping.startObject(FIELD_METRIC_COUNTER)
+            .field("type", "double") /* numeric label indexed as a metric */
+            .field("time_series_metric", "counter")
+            .endObject();
+
+        mapping.endObject().endObject().endObject();
+        assertAcked(indicesAdmin().prepareCreate(sourceIndex).setSettings(settings.build()).setMapping(mapping).get());
+
+        Map<String, Phase> phases = new HashMap<>();
+        phases.put(
+            "warm",
+            new Phase(
+                "warm",
+                TimeValue.ZERO,
+                Map.of("downsample", new org.elasticsearch.xpack.core.ilm.DownsampleAction(DateHistogramInterval.HOUR, null))
+            )
+        );
+        LifecyclePolicy policy = new LifecyclePolicy(POLICY_NAME, phases);
+        PutLifecycleAction.Request putLifecycleRequest = new PutLifecycleAction.Request(policy);
+        assertAcked(client().execute(PutLifecycleAction.INSTANCE, putLifecycleRequest).actionGet());
+    }
+
+    public void testILMDownsampleRollingRestart() throws Exception {
+        try (InternalTestCluster cluster = internalCluster()) {
+            final List<String> masterNodes = cluster.startMasterOnlyNodes(1);
+            cluster.startDataOnlyNodes(3);
+            ensureStableCluster(cluster.size());
+            ensureGreen();
+
+            final String sourceIndex = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
+            long startTime = LocalDateTime.parse("2020-09-09T18:00:00").atZone(ZoneId.of("UTC")).toInstant().toEpochMilli();
+            setup(sourceIndex, 1, 0, startTime);
+            final DownsampleConfig config = new DownsampleConfig(randomInterval());
+            final SourceSupplier sourceSupplier = () -> {
+                final String ts = randomDateForInterval(config.getInterval(), startTime);
+                double counterValue = DATE_FORMATTER.parseMillis(ts);
+                final List<String> dimensionValues = new ArrayList<>(5);
+                for (int j = 0; j < randomIntBetween(1, 5); j++) {
+                    dimensionValues.add(randomAlphaOfLength(6));
+                }
+                return XContentFactory.jsonBuilder()
+                    .startObject()
+                    .field(FIELD_TIMESTAMP, ts)
+                    .field(FIELD_DIMENSION_1, randomFrom(dimensionValues))
+                    .field(FIELD_DIMENSION_2, randomIntBetween(1, 10))
+                    .field(FIELD_METRIC_COUNTER, counterValue)
+                    .endObject();
+            };
+            int indexedDocs = bulkIndex(sourceIndex, sourceSupplier, DOC_COUNT);
+            final CountDownLatch disruptionStart = new CountDownLatch(1);
+            final CountDownLatch disruptionEnd = new CountDownLatch(1);
+
+            new Thread(new Disruptor(cluster, sourceIndex, new DisruptionListener() {
+                @Override
+                public void disruptionStart() {
+                    disruptionStart.countDown();
+                }
+
+                @Override
+                public void disruptionEnd() {
+                    disruptionEnd.countDown();
+                }
+            }, masterNodes.get(0), (ignored) -> {
+                try {
+                    cluster.rollingRestart(new InternalTestCluster.RestartCallback() {
+                        @Override
+                        public boolean validateClusterForming() {
+                            return true;
+                        }
+                    });
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+            })).start();
+
+            final String targetIndex = "downsample-" + sourceIndex + "-1h";
+            startRollupTaskViaIlm(sourceIndex, targetIndex, disruptionStart, disruptionEnd);
+            waitUntil(() -> cluster.client().admin().cluster().preparePendingClusterTasks().get().pendingTasks().isEmpty());
+            ensureStableCluster(cluster.numDataAndMasterNodes());
+            assertTargetIndex(cluster, targetIndex, indexedDocs);
+        }
+    }
+
+    private void startRollupTaskViaIlm(String sourceIndex, String targetIndex, CountDownLatch disruptionStart, CountDownLatch disruptionEnd)
+        throws Exception {
+        disruptionStart.await();
+        var request = new UpdateSettingsRequest(sourceIndex).settings(
+            Settings.builder().put(LifecycleSettings.LIFECYCLE_NAME, POLICY_NAME)
+        );
+        assertBusy(() -> {
+            try {
+                client().admin().indices().updateSettings(request).actionGet();
+            } catch (Exception e) {
+                throw new AssertionError(e);
+            }
+        });
+        assertBusy(() -> {
+            assertTrue("target index [" + targetIndex + "] does not exist", indexExists(targetIndex));
+            var getSettingsResponse = client().admin().indices().getSettings(new GetSettingsRequest().indices(targetIndex)).actionGet();
+            assertThat(getSettingsResponse.getSetting(targetIndex, IndexMetadata.INDEX_DOWNSAMPLE_STATUS.getKey()), equalTo("success"));
+        }, 60, TimeUnit.SECONDS);
+        disruptionEnd.await();
+    }
+
+    private void assertTargetIndex(final InternalTestCluster cluster, final String targetIndex, int indexedDocs) {
+        final GetIndexResponse getIndexResponse = cluster.client()
+            .admin()
+            .indices()
+            .getIndex(new GetIndexRequest().indices(targetIndex))
+            .actionGet();
+        assertEquals(1, getIndexResponse.indices().length);
+        final SearchResponse targetIndexSearch = cluster.client()
+            .prepareSearch(targetIndex)
+            .setQuery(new MatchAllQueryBuilder())
+            .setSize(Math.min(DOC_COUNT, indexedDocs))
+            .setTrackTotalHitsUpTo(Integer.MAX_VALUE)
+            .get();
+        assertTrue(targetIndexSearch.getHits().getHits().length > 0);
+    }
+
+    private int bulkIndex(final String indexName, final SourceSupplier sourceSupplier, int docCount) throws IOException {
+        BulkRequestBuilder bulkRequestBuilder = internalCluster().client().prepareBulk();
+        bulkRequestBuilder.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE);
+        for (int i = 0; i < docCount; i++) {
+            IndexRequest indexRequest = new IndexRequest(indexName).opType(DocWriteRequest.OpType.CREATE);
+            XContentBuilder source = sourceSupplier.get();
+            indexRequest.source(source);
+            bulkRequestBuilder.add(indexRequest);
+        }
+        BulkResponse bulkResponse = bulkRequestBuilder.get();
+        int duplicates = 0;
+        for (BulkItemResponse response : bulkResponse.getItems()) {
+            if (response.isFailed()) {
+                if (response.getFailure().getCause() instanceof VersionConflictEngineException) {
+                    // A duplicate event was created by random generator. We should not fail for this
+                    // reason.
+                    logger.debug("We tried to insert a duplicate: [{}]", response.getFailureMessage());
+                    duplicates++;
+                } else {
+                    fail("Failed to index data: " + bulkResponse.buildFailureMessage());
+                }
+            }
+        }
+        int docsIndexed = docCount - duplicates;
+        logger.info("Indexed [{}] documents. Dropped [{}] duplicates.", docsIndexed, duplicates);
+        return docsIndexed;
+    }
+
+    private String randomDateForInterval(final DateHistogramInterval interval, final long startTime) {
+        long endTime = startTime + 10 * interval.estimateMillis();
+        return randomDateForRange(startTime, endTime);
+    }
+
+    private String randomDateForRange(long start, long end) {
+        return DATE_FORMATTER.formatMillis(randomLongBetween(start, end));
+    }
+
+    @FunctionalInterface
+    public interface SourceSupplier {
+        XContentBuilder get() throws IOException;
+    }
+
+    interface DisruptionListener {
+        void disruptionStart();
+
+        void disruptionEnd();
+    }
+
+    private class Disruptor implements Runnable {
+        final InternalTestCluster cluster;
+        private final String sourceIndex;
+        private final DisruptionListener listener;
+        private final String clientNode;
+        private final Consumer<String> disruption;
+
+        private Disruptor(
+            final InternalTestCluster cluster,
+            final String sourceIndex,
+            final DisruptionListener listener,
+            final String clientNode,
+            final Consumer<String> disruption
+        ) {
+            this.cluster = cluster;
+            this.sourceIndex = sourceIndex;
+            this.listener = listener;
+            this.clientNode = clientNode;
+            this.disruption = disruption;
+        }
+
+        @Override
+        public void run() {
+            listener.disruptionStart();
+            try {
+                final String candidateNode = cluster.client(clientNode)
+                    .admin()
+                    .cluster()
+                    .prepareSearchShards(sourceIndex)
+                    .get()
+                    .getNodes()[0].getName();
+                logger.info("Candidate node [" + candidateNode + "]");
+                disruption.accept(candidateNode);
+                ensureGreen(sourceIndex);
+                ensureStableCluster(cluster.numDataAndMasterNodes(), clientNode);
+
+            } catch (Exception e) {
+                logger.error("Ignoring Error while injecting disruption [" + e.getMessage() + "]");
+            } finally {
+                listener.disruptionEnd();
+            }
+        }
+    }
+}

+ 211 - 0
x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/downsample/DownsampleShardTaskParams.java

@@ -0,0 +1,211 @@
+/*
+ * 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.downsample;
+
+import org.elasticsearch.TransportVersion;
+import org.elasticsearch.action.downsample.DownsampleConfig;
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.index.shard.ShardId;
+import org.elasticsearch.persistent.PersistentTaskParams;
+import org.elasticsearch.xcontent.ObjectParser;
+import org.elasticsearch.xcontent.ParseField;
+import org.elasticsearch.xcontent.XContentBuilder;
+import org.elasticsearch.xcontent.XContentParser;
+import org.elasticsearch.xpack.core.rollup.action.RollupShardTask;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+
+public record DownsampleShardTaskParams(
+    DownsampleConfig downsampleConfig,
+    String rollupIndex,
+    long indexStartTimeMillis,
+    long indexEndTimeMillis,
+    ShardId shardId,
+    String[] metrics,
+    String[] labels
+) implements PersistentTaskParams {
+
+    public static final String NAME = RollupShardTask.TASK_NAME;
+    private static final ParseField DOWNSAMPLE_CONFIG = new ParseField("downsample_config");
+    private static final ParseField ROLLUP_INDEX = new ParseField("rollup_index");
+    private static final ParseField INDEX_START_TIME_MILLIS = new ParseField("index_start_time_millis");
+    private static final ParseField INDEX_END_TIME_MILLIS = new ParseField("index_end_time_millis");
+    private static final ParseField SHARD_ID = new ParseField("shard_id");
+    private static final ParseField METRICS = new ParseField("metrics");
+    private static final ParseField LABELS = new ParseField("labels");
+    public static final ObjectParser<DownsampleShardTaskParams.Builder, Void> PARSER = new ObjectParser<>(NAME);
+
+    static {
+        PARSER.declareObject(
+            DownsampleShardTaskParams.Builder::downsampleConfig,
+            (p, c) -> DownsampleConfig.fromXContent(p),
+            DOWNSAMPLE_CONFIG
+        );
+        PARSER.declareString(DownsampleShardTaskParams.Builder::rollupIndex, ROLLUP_INDEX);
+        PARSER.declareLong(DownsampleShardTaskParams.Builder::indexStartTimeMillis, INDEX_START_TIME_MILLIS);
+        PARSER.declareLong(DownsampleShardTaskParams.Builder::indexEndTimeMillis, INDEX_END_TIME_MILLIS);
+        PARSER.declareString(DownsampleShardTaskParams.Builder::shardId, SHARD_ID);
+        PARSER.declareStringArray(DownsampleShardTaskParams.Builder::metrics, METRICS);
+        PARSER.declareStringArray(DownsampleShardTaskParams.Builder::labels, LABELS);
+    }
+
+    DownsampleShardTaskParams(final StreamInput in) throws IOException {
+        this(
+            new DownsampleConfig(in),
+            in.readString(),
+            in.readVLong(),
+            in.readVLong(),
+            new ShardId(in),
+            in.readStringArray(),
+            in.readStringArray()
+        );
+    }
+
+    @Override
+    public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
+        builder.startObject();
+        builder.field(DOWNSAMPLE_CONFIG.getPreferredName(), downsampleConfig);
+        builder.field(ROLLUP_INDEX.getPreferredName(), rollupIndex);
+        builder.field(INDEX_START_TIME_MILLIS.getPreferredName(), indexStartTimeMillis);
+        builder.field(INDEX_END_TIME_MILLIS.getPreferredName(), indexEndTimeMillis);
+        builder.field(SHARD_ID.getPreferredName(), shardId);
+        builder.array(METRICS.getPreferredName(), metrics);
+        builder.array(LABELS.getPreferredName(), labels);
+        return builder.endObject();
+    }
+
+    @Override
+    public String getWriteableName() {
+        return NAME;
+    }
+
+    @Override
+    public TransportVersion getMinimalSupportedVersion() {
+        return TransportVersion.V_8_500_054;
+    }
+
+    @Override
+    public void writeTo(StreamOutput out) throws IOException {
+        downsampleConfig.writeTo(out);
+        out.writeString(rollupIndex);
+        out.writeVLong(indexStartTimeMillis);
+        out.writeVLong(indexEndTimeMillis);
+        shardId.writeTo(out);
+        out.writeStringArray(metrics);
+        out.writeStringArray(labels);
+    }
+
+    public static DownsampleShardTaskParams readFromStream(final StreamInput in) throws IOException {
+        return new DownsampleShardTaskParams(
+            new DownsampleConfig(in),
+            in.readString(),
+            in.readVLong(),
+            in.readVLong(),
+            new ShardId(in),
+            in.readStringArray(),
+            in.readStringArray()
+        );
+    }
+
+    public static DownsampleShardTaskParams fromXContent(XContentParser parser) throws IOException {
+        final DownsampleShardTaskParams.Builder builder = new DownsampleShardTaskParams.Builder();
+        PARSER.parse(parser, builder, null);
+        return builder.build();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        DownsampleShardTaskParams that = (DownsampleShardTaskParams) o;
+        return indexStartTimeMillis == that.indexStartTimeMillis
+            && indexEndTimeMillis == that.indexEndTimeMillis
+            && Objects.equals(downsampleConfig, that.downsampleConfig)
+            && Objects.equals(rollupIndex, that.rollupIndex)
+            && Objects.equals(shardId.id(), that.shardId.id())
+            && Objects.equals(shardId.getIndexName(), that.shardId.getIndexName())
+            && Arrays.equals(metrics, that.metrics)
+            && Arrays.equals(labels, that.labels);
+    }
+
+    @Override
+    public int hashCode() {
+        int result = Objects.hash(
+            downsampleConfig,
+            rollupIndex,
+            indexStartTimeMillis,
+            indexEndTimeMillis,
+            shardId.id(),
+            shardId.getIndexName()
+        );
+        result = 31 * result + Arrays.hashCode(metrics);
+        result = 31 * result + Arrays.hashCode(labels);
+        return result;
+    }
+
+    public static class Builder {
+        DownsampleConfig downsampleConfig;
+        String rollupIndex;
+        long indexStartTimeMillis;
+        long indexEndTimeMillis;
+        ShardId shardId;
+        String[] metrics;
+        String[] labels;
+
+        public Builder downsampleConfig(final DownsampleConfig downsampleConfig) {
+            this.downsampleConfig = downsampleConfig;
+            return this;
+        }
+
+        public Builder rollupIndex(final String rollupIndex) {
+            this.rollupIndex = rollupIndex;
+            return this;
+        }
+
+        public Builder indexStartTimeMillis(final Long indexStartTimeMillis) {
+            this.indexStartTimeMillis = indexStartTimeMillis;
+            return this;
+        }
+
+        public Builder indexEndTimeMillis(final Long indexEndTimeMillis) {
+            this.indexEndTimeMillis = indexEndTimeMillis;
+            return this;
+        }
+
+        public Builder shardId(final String shardId) {
+            this.shardId = ShardId.fromString(shardId);
+            return this;
+        }
+
+        public Builder metrics(final List<String> metrics) {
+            this.metrics = metrics.toArray(String[]::new);
+            return this;
+        }
+
+        public Builder labels(final List<String> labels) {
+            this.labels = labels.toArray(String[]::new);
+            return this;
+        }
+
+        public DownsampleShardTaskParams build() {
+            return new DownsampleShardTaskParams(
+                downsampleConfig,
+                rollupIndex,
+                indexStartTimeMillis,
+                indexEndTimeMillis,
+                shardId,
+                metrics,
+                labels
+            );
+        }
+    }
+}

+ 8 - 1
x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/downsample/RestDownsampleAction.java

@@ -9,6 +9,7 @@ package org.elasticsearch.xpack.downsample;
 
 import org.elasticsearch.action.downsample.DownsampleConfig;
 import org.elasticsearch.client.internal.node.NodeClient;
+import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.rest.BaseRestHandler;
 import org.elasticsearch.rest.RestRequest;
 import org.elasticsearch.rest.Scope;
@@ -33,8 +34,14 @@ public class RestDownsampleAction extends BaseRestHandler {
     protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException {
         String sourceIndex = restRequest.param("index");
         String targetIndex = restRequest.param("target_index");
+        String timeout = restRequest.param("timeout");
         DownsampleConfig config = DownsampleConfig.fromXContent(restRequest.contentParser());
-        DownsampleAction.Request request = new DownsampleAction.Request(sourceIndex, targetIndex, config);
+        DownsampleAction.Request request = new DownsampleAction.Request(
+            sourceIndex,
+            targetIndex,
+            TimeValue.parseTimeValue(timeout, null, "wait_timeout"),
+            config
+        );
         return channel -> client.execute(DownsampleAction.INSTANCE, request, new RestToXContentListener<>(channel));
     }
 

+ 93 - 38
x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/downsample/RollupShardIndexer.java

@@ -8,11 +8,14 @@ package org.elasticsearch.xpack.downsample;
 
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
+import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.MatchNoDocsQuery;
+import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.util.BytesRef;
-import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.action.bulk.BulkItemResponse;
 import org.elasticsearch.action.bulk.BulkProcessor2;
 import org.elasticsearch.action.bulk.BulkRequest;
@@ -50,6 +53,7 @@ import org.elasticsearch.xpack.core.downsample.DownsampleIndexerAction;
 import org.elasticsearch.xpack.core.rollup.action.RollupAfterBulkInfo;
 import org.elasticsearch.xpack.core.rollup.action.RollupBeforeBulkInfo;
 import org.elasticsearch.xpack.core.rollup.action.RollupShardIndexerStatus;
+import org.elasticsearch.xpack.core.rollup.action.RollupShardPersistentTaskState;
 import org.elasticsearch.xpack.core.rollup.action.RollupShardTask;
 
 import java.io.Closeable;
@@ -74,11 +78,12 @@ import static org.elasticsearch.core.Strings.format;
  */
 class RollupShardIndexer {
 
+    public static final String NAME = "rollup-shard-indexer";
     private static final Logger logger = LogManager.getLogger(RollupShardIndexer.class);
     public static final int ROLLUP_BULK_ACTIONS = 10000;
     public static final ByteSizeValue ROLLUP_BULK_SIZE = new ByteSizeValue(1, ByteSizeUnit.MB);
     public static final ByteSizeValue ROLLUP_MAX_BYTES_IN_FLIGHT = new ByteSizeValue(50, ByteSizeUnit.MB);
-
+    private final IndexService indexService;
     private final IndexShard indexShard;
     private final Client client;
     private final String rollupIndex;
@@ -89,25 +94,29 @@ class RollupShardIndexer {
     private final Rounding.Prepared rounding;
     private final List<FieldValueFetcher> fieldValueFetchers;
     private final RollupShardTask task;
+    private final RollupShardPersistentTaskState state;
     private volatile boolean abort = false;
     ByteSizeValue rollupBulkSize = ROLLUP_BULK_SIZE;
     ByteSizeValue rollupMaxBytesInFlight = ROLLUP_MAX_BYTES_IN_FLIGHT;
 
     RollupShardIndexer(
-        RollupShardTask task,
-        Client client,
-        IndexService indexService,
-        ShardId shardId,
-        String rollupIndex,
-        DownsampleConfig config,
-        String[] metricFields,
-        String[] labelFields
+        final RollupShardTask task,
+        final Client client,
+        final IndexService indexService,
+        final ShardId shardId,
+        final String rollupIndex,
+        final DownsampleConfig config,
+        final String[] metrics,
+        final String[] labels,
+        final RollupShardPersistentTaskState state
     ) {
         this.task = task;
         this.client = client;
+        this.indexService = indexService;
         this.indexShard = indexService.getShard(shardId.id());
         this.rollupIndex = rollupIndex;
         this.searcher = indexShard.acquireSearcher("downsampling");
+        this.state = state;
         Closeable toClose = searcher;
         try {
             this.searchExecutionContext = indexService.newSearchExecutionContext(
@@ -122,9 +131,9 @@ class RollupShardIndexer {
             this.timestampFormat = timestampField.docValueFormat(null, null);
             this.rounding = config.createRounding();
 
-            List<FieldValueFetcher> fetchers = new ArrayList<>(metricFields.length + labelFields.length);
-            fetchers.addAll(FieldValueFetcher.create(searchExecutionContext, metricFields));
-            fetchers.addAll(FieldValueFetcher.create(searchExecutionContext, labelFields));
+            List<FieldValueFetcher> fetchers = new ArrayList<>(metrics.length + labels.length);
+            fetchers.addAll(FieldValueFetcher.create(searchExecutionContext, metrics));
+            fetchers.addAll(FieldValueFetcher.create(searchExecutionContext, labels));
             this.fieldValueFetchers = Collections.unmodifiableList(fetchers);
             toClose = null;
         } finally {
@@ -133,15 +142,21 @@ class RollupShardIndexer {
     }
 
     public DownsampleIndexerAction.ShardDownsampleResponse execute() throws IOException {
+        final Query initialStateQuery = createQuery();
+        if (initialStateQuery instanceof MatchNoDocsQuery) {
+            return new DownsampleIndexerAction.ShardDownsampleResponse(indexShard.shardId(), task.getNumIndexed());
+        }
         long startTime = client.threadPool().relativeTimeInMillis();
         task.setTotalShardDocCount(searcher.getDirectoryReader().numDocs());
         task.setRollupShardIndexerStatus(RollupShardIndexerStatus.STARTED);
+        task.updatePersistentTaskState(new RollupShardPersistentTaskState(RollupShardIndexerStatus.STARTED, null), ActionListener.noop());
+        logger.info("Downsampling task [" + task.getPersistentTaskId() + " on shard " + indexShard.shardId() + " started");
         BulkProcessor2 bulkProcessor = createBulkProcessor();
         try (searcher; bulkProcessor) {
             final TimeSeriesIndexSearcher timeSeriesSearcher = new TimeSeriesIndexSearcher(searcher, List.of(this::checkCancelled));
             TimeSeriesBucketCollector bucketCollector = new TimeSeriesBucketCollector(bulkProcessor);
             bucketCollector.preCollection();
-            timeSeriesSearcher.search(new MatchAllDocsQuery(), bucketCollector);
+            timeSeriesSearcher.search(initialStateQuery, bucketCollector);
             bucketCollector.postCollection();
         }
 
@@ -157,37 +172,55 @@ class RollupShardIndexer {
 
         if (task.getNumIndexed() != task.getNumSent()) {
             task.setRollupShardIndexerStatus(RollupShardIndexerStatus.FAILED);
-            throw new ElasticsearchException(
-                "Shard ["
-                    + indexShard.shardId()
-                    + "] failed to index all rollup documents. Sent ["
-                    + task.getNumSent()
-                    + "], indexed ["
-                    + task.getNumIndexed()
-                    + "]."
+            task.updatePersistentTaskState(
+                new RollupShardPersistentTaskState(RollupShardIndexerStatus.FAILED, null),
+                ActionListener.noop()
             );
+            final String error = "Downsampling task ["
+                + task.getPersistentTaskId()
+                + "] on shard "
+                + indexShard.shardId()
+                + " failed indexing, "
+                + " indexed ["
+                + task.getNumIndexed()
+                + "] sent ["
+                + task.getNumSent()
+                + "]";
+            logger.info(error);
+            throw new RollupShardIndexerException(error, false);
         }
 
         if (task.getNumFailed() > 0) {
-            task.setRollupShardIndexerStatus(RollupShardIndexerStatus.FAILED);
-            throw new ElasticsearchException(
-                "Shard ["
-                    + indexShard.shardId()
-                    + "] failed to index all rollup documents. Sent ["
-                    + task.getNumSent()
-                    + "], failed ["
-                    + task.getNumFailed()
-                    + "]."
+            final String error = "Downsampling task ["
+                + task.getPersistentTaskId()
+                + "] on shard "
+                + indexShard.shardId()
+                + " failed indexing ["
+                + task.getNumFailed()
+                + "]";
+            logger.info(error);
+            task.updatePersistentTaskState(
+                new RollupShardPersistentTaskState(RollupShardIndexerStatus.FAILED, null),
+                ActionListener.noop()
             );
+            throw new RollupShardIndexerException(error, false);
         }
 
         task.setRollupShardIndexerStatus(RollupShardIndexerStatus.COMPLETED);
-
+        task.updatePersistentTaskState(new RollupShardPersistentTaskState(RollupShardIndexerStatus.COMPLETED, null), ActionListener.noop());
+        logger.info("Downsampling task [" + task.getPersistentTaskId() + " on shard " + indexShard.shardId() + " completed");
         return new DownsampleIndexerAction.ShardDownsampleResponse(indexShard.shardId(), task.getNumIndexed());
     }
 
+    private Query createQuery() {
+        if (this.state.started() && this.state.tsid() != null) {
+            return SortedSetDocValuesField.newSlowRangeQuery(TimeSeriesIdFieldMapper.NAME, this.state.tsid(), null, true, false);
+        }
+        return new MatchAllDocsQuery();
+    }
+
     private void checkCancelled() {
-        if (task.isCancelled() || abort) {
+        if (task.isCancelled()) {
             logger.warn(
                 "Shard [{}] rollup abort, sent [{}], indexed [{}], failed[{}]",
                 indexShard.shardId(),
@@ -196,7 +229,32 @@ class RollupShardIndexer {
                 task.getNumFailed()
             );
             task.setRollupShardIndexerStatus(RollupShardIndexerStatus.CANCELLED);
-            throw new TaskCancelledException(format("Shard %s rollup cancelled", indexShard.shardId()));
+            task.updatePersistentTaskState(
+                new RollupShardPersistentTaskState(RollupShardIndexerStatus.CANCELLED, null),
+                ActionListener.noop()
+            );
+            logger.info("Downsampling task [" + task.getPersistentTaskId() + "] on shard " + indexShard.shardId() + " cancelled");
+            throw new RollupShardIndexerException(
+                new TaskCancelledException(format("Shard %s rollup cancelled", indexShard.shardId())),
+                format("Shard %s rollup cancelled", indexShard.shardId()),
+                false
+            );
+
+        }
+        if (abort) {
+            logger.warn(
+                "Shard [{}] rollup abort, sent [{}], indexed [{}], failed[{}]",
+                indexShard.shardId(),
+                task.getNumSent(),
+                task.getNumIndexed(),
+                task.getNumFailed()
+            );
+            task.setRollupShardIndexerStatus(RollupShardIndexerStatus.FAILED);
+            task.updatePersistentTaskState(
+                new RollupShardPersistentTaskState(RollupShardIndexerStatus.FAILED, null),
+                ActionListener.noop()
+            );
+            throw new RollupShardIndexerException("Bulk indexing failure", true);
         }
     }
 
@@ -247,7 +305,6 @@ class RollupShardIndexer {
                         );
                     logger.error("Shard [{}] failed to populate rollup index. Failures: [{}]", indexShard.shardId(), failures);
 
-                    // cancel rollup task
                     abort = true;
                 }
             }
@@ -259,7 +316,6 @@ class RollupShardIndexer {
                     task.addNumFailed(items);
                     logger.error(() -> format("Shard [%s] failed to populate rollup index.", indexShard.shardId()), failure);
 
-                    // cancel rollup task
                     abort = true;
                 }
             }
@@ -394,7 +450,7 @@ class RollupShardIndexer {
         }
 
         @Override
-        public void preCollection() throws IOException {
+        public void preCollection() {
             // check cancel when start running
             checkCancelled();
         }
@@ -487,7 +543,6 @@ class RollupShardIndexer {
                 builder.endObject();
                 return builder;
             }
-
             builder.field(timestampField.name(), timestampFormat.format(timestamp));
             builder.field(DocCountFieldMapper.NAME, docCount);
             // Extract dimension values from _tsid field, so we avoid loading them from doc_values

+ 28 - 0
x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/downsample/RollupShardIndexerException.java

@@ -0,0 +1,28 @@
+/*
+ * 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.downsample;
+
+import org.elasticsearch.ElasticsearchException;
+
+public class RollupShardIndexerException extends ElasticsearchException {
+    private final boolean retriable;
+
+    public RollupShardIndexerException(final Throwable cause, final String message, boolean retriable) {
+        super(message, cause);
+        this.retriable = retriable;
+    }
+
+    public RollupShardIndexerException(final String message, boolean retriable) {
+        super(message);
+        this.retriable = retriable;
+    }
+
+    public boolean isRetriable() {
+        return retriable;
+    }
+}

+ 187 - 0
x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/downsample/RollupShardPersistentTaskExecutor.java

@@ -0,0 +1,187 @@
+/*
+ * 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.downsample;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.elasticsearch.action.ActionListener;
+import org.elasticsearch.action.search.SearchRequest;
+import org.elasticsearch.client.internal.Client;
+import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.cluster.node.DiscoveryNode;
+import org.elasticsearch.cluster.routing.ShardRouting;
+import org.elasticsearch.common.util.concurrent.AbstractRunnable;
+import org.elasticsearch.index.IndexService;
+import org.elasticsearch.index.mapper.TimeSeriesIdFieldMapper;
+import org.elasticsearch.index.shard.ShardId;
+import org.elasticsearch.indices.IndicesService;
+import org.elasticsearch.persistent.AllocatedPersistentTask;
+import org.elasticsearch.persistent.PersistentTaskState;
+import org.elasticsearch.persistent.PersistentTasksCustomMetadata;
+import org.elasticsearch.persistent.PersistentTasksExecutor;
+import org.elasticsearch.search.SearchHit;
+import org.elasticsearch.search.sort.SortOrder;
+import org.elasticsearch.tasks.TaskId;
+import org.elasticsearch.xpack.core.rollup.action.RollupShardIndexerStatus;
+import org.elasticsearch.xpack.core.rollup.action.RollupShardPersistentTaskState;
+import org.elasticsearch.xpack.core.rollup.action.RollupShardTask;
+import org.elasticsearch.xpack.rollup.Rollup;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Objects;
+
+public class RollupShardPersistentTaskExecutor extends PersistentTasksExecutor<DownsampleShardTaskParams> {
+    private static final Logger logger = LogManager.getLogger(RollupShardPersistentTaskExecutor.class);
+    private final Client client;
+    private final IndicesService indicesService;
+
+    public RollupShardPersistentTaskExecutor(
+        final Client client,
+        final IndicesService indicesService,
+        final String taskName,
+        final String executorName
+    ) {
+        super(taskName, executorName);
+        this.client = Objects.requireNonNull(client);
+        this.indicesService = Objects.requireNonNull(indicesService);
+    }
+
+    @Override
+    protected void nodeOperation(
+        final AllocatedPersistentTask task,
+        final DownsampleShardTaskParams params,
+        final PersistentTaskState state
+    ) {
+        // NOTE: query the downsampling target index so that we can start the downsampling task from the latest indexed tsid.
+        final SearchRequest searchRequest = new SearchRequest(params.rollupIndex());
+        searchRequest.source().sort(TimeSeriesIdFieldMapper.NAME, SortOrder.DESC).size(1);
+        searchRequest.preference("_shards:" + params.shardId().id());
+        client.search(
+            searchRequest,
+            ActionListener.wrap(
+                searchResponse -> fork(task, params, searchResponse.getHits().getHits()),
+                e -> fork(task, params, new SearchHit[] {})
+            )
+        );
+    }
+
+    private void fork(final AllocatedPersistentTask task, final DownsampleShardTaskParams params, final SearchHit[] lastRollupTsidHits) {
+        client.threadPool().executor(Rollup.DOWSAMPLE_TASK_THREAD_POOL_NAME).execute(new AbstractRunnable() {
+            @Override
+            public void onFailure(Exception e) {
+                task.markAsFailed(e);
+            }
+
+            @Override
+            protected void doRun() throws Exception {
+                startRollupShardIndexer(task, params, lastRollupTsidHits);
+            }
+        });
+    }
+
+    private void startRollupShardIndexer(
+        final AllocatedPersistentTask task,
+        final DownsampleShardTaskParams params,
+        final SearchHit[] lastRollupTsidHits
+    ) {
+        final RollupShardPersistentTaskState initialState = lastRollupTsidHits.length == 0
+            ? new RollupShardPersistentTaskState(RollupShardIndexerStatus.INITIALIZED, null)
+            : new RollupShardPersistentTaskState(
+                RollupShardIndexerStatus.STARTED,
+                Arrays.stream(lastRollupTsidHits).findFirst().get().field("_tsid").getValue()
+            );
+        final RollupShardIndexer rollupShardIndexer = new RollupShardIndexer(
+            (RollupShardTask) task,
+            client,
+            getIndexService(indicesService, params),
+            params.shardId(),
+            params.rollupIndex(),
+            params.downsampleConfig(),
+            params.metrics(),
+            params.labels(),
+            initialState
+        );
+        try {
+            rollupShardIndexer.execute();
+            task.markAsCompleted();
+        } catch (final RollupShardIndexerException e) {
+            if (e.isRetriable()) {
+                logger.error("Downsampling task [" + task.getPersistentTaskId() + " retriable failure [" + e.getMessage() + "]");
+                task.markAsLocallyAborted(e.getMessage());
+            } else {
+                logger.error("Downsampling task [" + task.getPersistentTaskId() + " non retriable failure [" + e.getMessage() + "]");
+                task.markAsFailed(e);
+            }
+        } catch (final Exception e) {
+            logger.error("Downsampling task [" + task.getPersistentTaskId() + " non-retriable failure [" + e.getMessage() + "]");
+            task.markAsFailed(e);
+        }
+    }
+
+    private static IndexService getIndexService(final IndicesService indicesService, final DownsampleShardTaskParams params) {
+        return indicesService.indexService(params.shardId().getIndex());
+    }
+
+    @Override
+    protected AllocatedPersistentTask createTask(
+        long id,
+        final String type,
+        final String action,
+        final TaskId parentTaskId,
+        final PersistentTasksCustomMetadata.PersistentTask<DownsampleShardTaskParams> taskInProgress,
+        final Map<String, String> headers
+    ) {
+        final DownsampleShardTaskParams params = taskInProgress.getParams();
+        return new RollupShardTask(
+            id,
+            type,
+            action,
+            parentTaskId,
+            params.rollupIndex(),
+            params.indexStartTimeMillis(),
+            params.indexEndTimeMillis(),
+            params.downsampleConfig(),
+            headers,
+            params.shardId()
+        );
+    }
+
+    @Override
+    public PersistentTasksCustomMetadata.Assignment getAssignment(
+        final DownsampleShardTaskParams params,
+        final Collection<DiscoveryNode> candidateNodes,
+        final ClusterState clusterState
+    ) {
+        // NOTE: downsampling works by running a task per each shard of the source index.
+        // Here we make sure we assign the task to the actual node holding the shard identified by
+        // the downsampling task shard id.
+        final ShardId shardId = params.shardId();
+        final ShardRouting shardRouting = clusterState.routingTable().shardRoutingTable(shardId).primaryShard();
+        if (shardRouting.started() == false) {
+            return NO_NODE_FOUND;
+        }
+
+        return candidateNodes.stream()
+            .filter(candidateNode -> candidateNode.getId().equals(shardRouting.currentNodeId()))
+            .findAny()
+            .map(
+                node -> new PersistentTasksCustomMetadata.Assignment(
+                    node.getId(),
+                    "downsampling using node holding shard [" + shardId + "]"
+                )
+            )
+            .orElse(NO_NODE_FOUND);
+    }
+
+    @Override
+    public String getExecutor() {
+        return Rollup.DOWSAMPLE_TASK_THREAD_POOL_NAME;
+    }
+}

+ 405 - 254
x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/downsample/TransportDownsampleAction.java

@@ -9,19 +9,19 @@ package org.elasticsearch.xpack.downsample;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.ResourceAlreadyExistsException;
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.action.ActionRequestValidationException;
 import org.elasticsearch.action.admin.cluster.stats.MappingVisitor;
 import org.elasticsearch.action.admin.indices.create.CreateIndexClusterStateUpdateRequest;
-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.mapping.get.GetMappingsRequest;
 import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
 import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
 import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest;
 import org.elasticsearch.action.downsample.DownsampleConfig;
 import org.elasticsearch.action.support.ActionFilters;
+import org.elasticsearch.action.support.ActiveShardCount;
 import org.elasticsearch.action.support.master.AcknowledgedResponse;
 import org.elasticsearch.action.support.master.AcknowledgedTransportMasterNodeAction;
 import org.elasticsearch.client.internal.Client;
@@ -32,6 +32,7 @@ import org.elasticsearch.cluster.SimpleBatchedExecutor;
 import org.elasticsearch.cluster.block.ClusterBlockException;
 import org.elasticsearch.cluster.block.ClusterBlockLevel;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
+import org.elasticsearch.cluster.metadata.IndexMetadata.DownsampleTaskStatus;
 import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
 import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.cluster.metadata.MetadataCreateIndexService;
@@ -40,6 +41,7 @@ import org.elasticsearch.cluster.routing.allocation.allocator.AllocationActionLi
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.cluster.service.MasterServiceTaskQueue;
 import org.elasticsearch.common.Priority;
+import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.compress.CompressedXContent;
 import org.elasticsearch.common.inject.Inject;
@@ -48,6 +50,7 @@ import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.util.concurrent.ThreadContext;
 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;
@@ -57,7 +60,11 @@ import org.elasticsearch.index.mapper.DateFieldMapper;
 import org.elasticsearch.index.mapper.MappedFieldType;
 import org.elasticsearch.index.mapper.MapperService;
 import org.elasticsearch.index.mapper.TimeSeriesParams;
+import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.indices.IndicesService;
+import org.elasticsearch.persistent.PersistentTaskParams;
+import org.elasticsearch.persistent.PersistentTasksCustomMetadata;
+import org.elasticsearch.persistent.PersistentTasksService;
 import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
 import org.elasticsearch.tasks.Task;
 import org.elasticsearch.tasks.TaskId;
@@ -69,25 +76,30 @@ import org.elasticsearch.xcontent.XContentType;
 import org.elasticsearch.xpack.aggregatemetric.mapper.AggregateDoubleMetricFieldMapper;
 import org.elasticsearch.xpack.core.ClientHelper;
 import org.elasticsearch.xpack.core.downsample.DownsampleAction;
-import org.elasticsearch.xpack.core.downsample.DownsampleIndexerAction;
 import org.elasticsearch.xpack.core.ilm.LifecycleSettings;
+import org.elasticsearch.xpack.core.rollup.action.RollupShardPersistentTaskState;
+import org.elasticsearch.xpack.core.rollup.action.RollupShardTask;
 import org.elasticsearch.xpack.core.security.authz.AuthorizationServiceField;
 import org.elasticsearch.xpack.core.security.authz.accesscontrol.IndicesAccessControl;
 
 import java.io.IOException;
+import java.time.Instant;
 import java.time.OffsetDateTime;
 import java.time.format.DateTimeFormatter;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Predicate;
 
 import static org.elasticsearch.index.mapper.TimeSeriesParams.TIME_SERIES_METRIC_PARAM;
+import static org.elasticsearch.xpack.core.ilm.DownsampleAction.DOWNSAMPLED_INDEX_PREFIX;
 
 /**
- * The master rollup action that coordinates
- *  -  creating the rollup index
- *  -  calling {@link TransportDownsampleIndexerAction} to index rollup documents
+ * The master downsample action that coordinates
+ *  -  creating the downsample index
+ *  -  instantiating {@link RollupShardIndexer}s to index downsample documents
  *  -  cleaning up state
  */
 public class TransportDownsampleAction extends AcknowledgedTransportMasterNodeAction<DownsampleAction.Request> {
@@ -97,10 +109,11 @@ public class TransportDownsampleAction extends AcknowledgedTransportMasterNodeAc
     private final Client client;
     private final IndicesService indicesService;
     private final ClusterService clusterService;
-    private final MasterServiceTaskQueue<RollupClusterStateUpdateTask> taskQueue;
+    private final MasterServiceTaskQueue<DownsampleClusterStateUpdateTask> taskQueue;
     private final MetadataCreateIndexService metadataCreateIndexService;
     private final IndexScopedSettings indexScopedSettings;
     private final ThreadContext threadContext;
+    private final PersistentTasksService persistentTasksService;
 
     private static final Set<String> FORBIDDEN_SETTINGS = Set.of(
         IndexSettings.DEFAULT_PIPELINE.getKey(),
@@ -114,15 +127,16 @@ public class TransportDownsampleAction extends AcknowledgedTransportMasterNodeAc
     /**
      * This is the cluster state task executor for cluster state update actions.
      */
-    private static final SimpleBatchedExecutor<RollupClusterStateUpdateTask, Void> STATE_UPDATE_TASK_EXECUTOR =
+    private static final SimpleBatchedExecutor<DownsampleClusterStateUpdateTask, Void> STATE_UPDATE_TASK_EXECUTOR =
         new SimpleBatchedExecutor<>() {
             @Override
-            public Tuple<ClusterState, Void> executeTask(RollupClusterStateUpdateTask task, ClusterState clusterState) throws Exception {
+            public Tuple<ClusterState, Void> executeTask(DownsampleClusterStateUpdateTask task, ClusterState clusterState)
+                throws Exception {
                 return Tuple.tuple(task.execute(clusterState), null);
             }
 
             @Override
-            public void taskSucceeded(RollupClusterStateUpdateTask task, Void unused) {
+            public void taskSucceeded(DownsampleClusterStateUpdateTask task, Void unused) {
                 task.listener.onResponse(AcknowledgedResponse.TRUE);
             }
         };
@@ -137,7 +151,8 @@ public class TransportDownsampleAction extends AcknowledgedTransportMasterNodeAc
         MetadataCreateIndexService metadataCreateIndexService,
         ActionFilters actionFilters,
         IndexNameExpressionResolver indexNameExpressionResolver,
-        IndexScopedSettings indexScopedSettings
+        IndexScopedSettings indexScopedSettings,
+        PersistentTasksService persistentTasksService
     ) {
         super(
             DownsampleAction.NAME,
@@ -156,6 +171,7 @@ public class TransportDownsampleAction extends AcknowledgedTransportMasterNodeAc
         this.indexScopedSettings = indexScopedSettings;
         this.threadContext = threadPool.getThreadContext();
         this.taskQueue = clusterService.createTaskQueue("rollup", Priority.URGENT, STATE_UPDATE_TASK_EXECUTOR);
+        this.persistentTasksService = persistentTasksService;
     }
 
     @Override
@@ -216,20 +232,36 @@ public class TransportDownsampleAction extends AcknowledgedTransportMasterNodeAc
             return;
         }
 
-        final String rollupIndexName = request.getTargetIndex();
-        // Assert rollup index does not exist
-        MetadataCreateIndexService.validateIndexName(rollupIndexName, state);
+        // Shortcircuit if target index has been downsampled:
+        final String downsampleIndexName = request.getTargetIndex();
+        IndexMetadata downsampleIndex = state.getMetadata().index(downsampleIndexName);
+        if (downsampleIndex != null) {
+            var downsampleStatus = IndexMetadata.INDEX_DOWNSAMPLE_STATUS.get(downsampleIndex.getSettings());
+            if (downsampleStatus == DownsampleTaskStatus.UNKNOWN) {
+                // This isn't a downsample index, so fail:
+                listener.onFailure(new ResourceAlreadyExistsException(downsampleIndex.getIndex()));
+                return;
+            } else if (downsampleStatus == DownsampleTaskStatus.SUCCESS) {
+                listener.onResponse(AcknowledgedResponse.TRUE);
+                return;
+            }
+        }
+        try {
+            MetadataCreateIndexService.validateIndexName(downsampleIndexName, state);
+        } catch (ResourceAlreadyExistsException e) {
+            // ignore index already exists
+        }
 
-        // Rollup will perform the following tasks:
+        // Downsample will perform the following tasks:
         // 1. Extract source index mappings
-        // 2. Extract rollup config from index mappings
-        // 3. Create the rollup index
-        // 4. Run rollup indexer
-        // 5. Make rollup index read-only and set replicas
-        // 6. Refresh rollup index
-        // 7. Mark rollup index as "completed successfully"
-        // 8. Force-merge the rollup index to a single segment
-        // At any point if there is an issue, delete the rollup index
+        // 2. Extract downsample config from index mappings
+        // 3. Create the downsample index
+        // 4. Run downsample indexer
+        // 5. Make downsample index read-only and set replicas
+        // 6. Refresh downsample index
+        // 7. Mark downsample index as "completed successfully"
+        // 8. Force-merge the downsample index to a single segment
+        // At any point if there is an issue, delete the downsample index
 
         // 1. Extract source index mappings
         final TaskId parentTask = new TaskId(clusterService.localNode().getId(), task.getId());
@@ -244,7 +276,7 @@ public class TransportDownsampleAction extends AcknowledgedTransportMasterNodeAc
                 .map(mappingMetadata -> mappingMetadata.getValue().sourceAsMap())
                 .orElseThrow(() -> new IllegalArgumentException("No mapping found for rollup source index [" + sourceIndexName + "]"));
 
-            // 2. Extract rollup config from index mappings
+            // 2. Extract downsample config from index mappings
             final MapperService mapperService = indicesService.createIndexMapperServiceForValidation(sourceIndexMetadata);
             final CompressedXContent sourceIndexCompressedXContent = new CompressedXContent(sourceIndexMappings);
             mapperService.merge(MapperService.SINGLE_MAPPING_NAME, sourceIndexCompressedXContent, MapperService.MergeReason.INDEX_TEMPLATE);
@@ -280,166 +312,205 @@ public class TransportDownsampleAction extends AcknowledgedTransportMasterNodeAc
 
             final String mapping;
             try {
-                mapping = createRollupIndexMapping(helper, request.getDownsampleConfig(), mapperService, sourceIndexMappings);
+                mapping = createDownsampleIndexMapping(helper, request.getDownsampleConfig(), mapperService, sourceIndexMappings);
             } catch (IOException e) {
                 listener.onFailure(e);
                 return;
             }
-            // 3. Create rollup index
-            createRollupIndex(rollupIndexName, sourceIndexMetadata, mapping, request, ActionListener.wrap(createIndexResp -> {
+            // 3. Create downsample index
+            createDownsampleIndex(downsampleIndexName, sourceIndexMetadata, mapping, request, ActionListener.wrap(createIndexResp -> {
                 if (createIndexResp.isAcknowledged()) {
-                    // 3. Rollup index created. Run rollup indexer
-                    DownsampleIndexerAction.Request rollupIndexerRequest = new DownsampleIndexerAction.Request(
+                    performShardDownsampling(
                         request,
-                        OffsetDateTime.parse(
-                            sourceIndexMetadata.getSettings().get(IndexSettings.TIME_SERIES_START_TIME.getKey()),
-                            DateTimeFormatter.ISO_DATE_TIME
-                        ).toInstant().toEpochMilli(),
-                        OffsetDateTime.parse(
-                            sourceIndexMetadata.getSettings().get(IndexSettings.TIME_SERIES_END_TIME.getKey()),
-                            DateTimeFormatter.ISO_DATE_TIME
-                        ).toInstant().toEpochMilli(),
-                        dimensionFields.toArray(new String[0]),
-                        metricFields.toArray(new String[0]),
-                        labelFields.toArray(new String[0])
+                        listener,
+                        sourceIndexMetadata,
+                        downsampleIndexName,
+                        parentTask,
+                        metricFields,
+                        labelFields
                     );
-                    rollupIndexerRequest.setParentTask(parentTask);
-                    client.execute(DownsampleIndexerAction.INSTANCE, rollupIndexerRequest, ActionListener.wrap(indexerResp -> {
-                        if (indexerResp.isCreated()) {
-                            // 4. Make rollup index read-only and set the correct number of replicas
-                            final Settings.Builder settings = Settings.builder().put(IndexMetadata.SETTING_BLOCKS_WRITE, true);
-                            // Number of replicas had been previously set to 0 to speed up index population
-                            if (sourceIndexMetadata.getNumberOfReplicas() > 0) {
-                                settings.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, sourceIndexMetadata.getNumberOfReplicas());
-                            }
-                            // Setting index.hidden has been initially set to true. We revert this to the value of the source index
-                            if (sourceIndexMetadata.isHidden() == false) {
-                                if (sourceIndexMetadata.getSettings().keySet().contains(IndexMetadata.SETTING_INDEX_HIDDEN)) {
-                                    settings.put(IndexMetadata.SETTING_INDEX_HIDDEN, false);
-                                } else {
-                                    settings.putNull(IndexMetadata.SETTING_INDEX_HIDDEN);
-                                }
-                            }
-                            UpdateSettingsRequest updateSettingsReq = new UpdateSettingsRequest(settings.build(), rollupIndexName);
-                            updateSettingsReq.setParentTask(parentTask);
-                            client.admin().indices().updateSettings(updateSettingsReq, ActionListener.wrap(updateSettingsResponse -> {
-                                if (updateSettingsResponse.isAcknowledged()) {
-                                    // 5. Refresh rollup index
-                                    refreshIndex(rollupIndexName, parentTask, ActionListener.wrap(refreshIndexResponse -> {
-                                        if (refreshIndexResponse.getFailedShards() == 0) {
-                                            // 6. Mark rollup index as "completed successfully"
-                                            updateRollupMetadata(rollupIndexName, request, ActionListener.wrap(resp -> {
-                                                if (resp.isAcknowledged()) {
-                                                    // 7. Force-merge the rollup index to a single segment
-                                                    forceMergeIndex(
-                                                        rollupIndexName,
-                                                        parentTask,
-                                                        ActionListener.wrap(
-                                                            mergeIndexResp -> listener.onResponse(AcknowledgedResponse.TRUE),
-                                                            e -> {
-                                                                /*
-                                                                 * At this point rollup has been created successfully even if
-                                                                 * force-merge fails. So, we should not fail the rollup operation.
-                                                                 */
-                                                                logger.error(
-                                                                    "Failed to force-merge rollup index [" + rollupIndexName + "]",
-                                                                    e
-                                                                );
-                                                                listener.onResponse(AcknowledgedResponse.TRUE);
-                                                            }
-                                                        )
-                                                    );
-                                                } else {
-                                                    deleteRollupIndex(
-                                                        sourceIndexName,
-                                                        rollupIndexName,
-                                                        parentTask,
-                                                        listener,
-                                                        new ElasticsearchException(
-                                                            "Failed to publish new cluster state with rollup metadata"
-                                                        )
-                                                    );
-                                                }
-                                            },
-                                                e -> deleteRollupIndex(
-                                                    sourceIndexName,
-                                                    rollupIndexName,
-                                                    parentTask,
-                                                    listener,
-                                                    new ElasticsearchException(
-                                                        "Failed to publish new cluster state with rollup metadata",
-                                                        e
-                                                    )
-                                                )
-                                            ));
-                                        } else {
-                                            deleteRollupIndex(
-                                                sourceIndexName,
-                                                rollupIndexName,
-                                                parentTask,
-                                                listener,
-                                                new ElasticsearchException("Failed to refresh rollup index [" + rollupIndexName + "]")
-                                            );
-                                        }
-                                    },
-                                        e -> deleteRollupIndex(
-                                            sourceIndexName,
-                                            rollupIndexName,
-                                            parentTask,
-                                            listener,
-                                            new ElasticsearchException("Failed to refresh rollup index [" + rollupIndexName + "]", e)
-                                        )
-                                    ));
-                                } else {
-                                    deleteRollupIndex(
-                                        sourceIndexName,
-                                        rollupIndexName,
-                                        parentTask,
-                                        listener,
-                                        new ElasticsearchException("Unable to update settings of rollup index [" + rollupIndexName + "]")
-                                    );
-                                }
-                            },
-                                e -> deleteRollupIndex(
-                                    sourceIndexName,
-                                    rollupIndexName,
-                                    parentTask,
-                                    listener,
-                                    new ElasticsearchException("Unable to update settings of rollup index [" + rollupIndexName + "]", e)
-                                )
-                            ));
-                        } else {
-                            deleteRollupIndex(
-                                sourceIndexName,
-                                rollupIndexName,
-                                parentTask,
-                                listener,
-                                new ElasticsearchException("Unable to index into rollup index [" + rollupIndexName + "]")
-                            );
-                        }
-                    }, e -> deleteRollupIndex(sourceIndexName, rollupIndexName, parentTask, listener, e)));
                 } else {
-                    listener.onFailure(new ElasticsearchException("Failed to create rollup index [" + rollupIndexName + "]"));
+                    listener.onFailure(new ElasticsearchException("Failed to create rollup index [" + downsampleIndexName + "]"));
                 }
-            }, listener::onFailure));
+            }, e -> {
+                if (e instanceof ResourceAlreadyExistsException) {
+                    performShardDownsampling(
+                        request,
+                        listener,
+                        sourceIndexMetadata,
+                        downsampleIndexName,
+                        parentTask,
+                        metricFields,
+                        labelFields
+                    );
+                } else {
+                    listener.onFailure(e);
+                }
+            }));
         }, listener::onFailure));
     }
 
+    // 3. downsample index created or already exist (in case of retry). Run downsample indexer persistent task on each shard.
+    private void performShardDownsampling(
+        DownsampleAction.Request request,
+        ActionListener<AcknowledgedResponse> listener,
+        IndexMetadata sourceIndexMetadata,
+        String downsampleIndexName,
+        TaskId parentTask,
+        List<String> metricFields,
+        List<String> labelFields
+    ) {
+        final int numberOfShards = sourceIndexMetadata.getNumberOfShards();
+        final Index sourceIndex = sourceIndexMetadata.getIndex();
+        // NOTE: before we set the number of replicas to 0, as a result here we are
+        // only dealing with primary shards.
+        final AtomicInteger countDown = new AtomicInteger(numberOfShards);
+        for (int shardNum = 0; shardNum < numberOfShards; shardNum++) {
+            final ShardId shardId = new ShardId(sourceIndex, shardNum);
+            final String persistentTaskId = createPersistentTaskId(
+                downsampleIndexName,
+                shardId,
+                request.getDownsampleConfig().getInterval()
+            );
+            final DownsampleShardTaskParams params = createPersistentTaskParams(
+                request.getDownsampleConfig(),
+                sourceIndexMetadata,
+                downsampleIndexName,
+                metricFields,
+                labelFields,
+                shardId
+            );
+            Predicate<PersistentTasksCustomMetadata.PersistentTask<?>> predicate = runningTask -> {
+                if (runningTask == null) {
+                    // NOTE: don't need to wait if the persistent task completed and was removed
+                    return true;
+                }
+                RollupShardPersistentTaskState runningPersistentTaskState = (RollupShardPersistentTaskState) runningTask.getState();
+                return runningPersistentTaskState != null && runningPersistentTaskState.done();
+            };
+            var taskListener = new PersistentTasksService.WaitForPersistentTaskListener<>() {
+
+                @Override
+                public void onResponse(PersistentTasksCustomMetadata.PersistentTask<PersistentTaskParams> persistentTask) {
+                    logger.info("Downsampling task [" + persistentTaskId + " completed for shard " + params.shardId());
+                    if (countDown.decrementAndGet() == 0) {
+                        logger.info("All downsampling tasks completed [" + numberOfShards + "]");
+                        updateTargetIndexSettingStep(request, listener, sourceIndexMetadata, downsampleIndexName, parentTask);
+                    }
+                }
+
+                @Override
+                public void onFailure(Exception e) {
+                    logger.error("error while waiting for downsampling persistent task", e);
+                    listener.onFailure(e);
+                }
+            };
+            persistentTasksService.sendStartRequest(
+                persistentTaskId,
+                RollupShardTask.TASK_NAME,
+                params,
+                ActionListener.wrap(
+                    startedTask -> persistentTasksService.waitForPersistentTaskCondition(
+                        startedTask.getId(),
+                        predicate,
+                        request.getWaitTimeout(),
+                        taskListener
+                    ),
+                    e -> {
+                        if (e instanceof ResourceAlreadyExistsException) {
+                            logger.info("Task [" + persistentTaskId + "] already exists. Waiting.");
+                            persistentTasksService.waitForPersistentTaskCondition(
+                                persistentTaskId,
+                                predicate,
+                                request.getWaitTimeout(),
+                                taskListener
+                            );
+                        } else {
+                            listener.onFailure(new ElasticsearchException("Task [" + persistentTaskId + "] failed starting", e));
+                        }
+                    }
+                )
+            );
+        }
+    }
+
+    // 4. Make downsample index read-only and set the correct number of replicas
+    private void updateTargetIndexSettingStep(
+        final DownsampleAction.Request request,
+        final ActionListener<AcknowledgedResponse> listener,
+        final IndexMetadata sourceIndexMetadata,
+        final String downsampleIndexName,
+        final TaskId parentTask
+    ) {
+        // 4. Make downsample index read-only and set the correct number of replicas
+        final Settings.Builder settings = Settings.builder().put(IndexMetadata.SETTING_BLOCKS_WRITE, true);
+        // Number of replicas had been previously set to 0 to speed up index population
+        if (sourceIndexMetadata.getNumberOfReplicas() > 0) {
+            settings.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, sourceIndexMetadata.getNumberOfReplicas());
+        }
+        // Setting index.hidden has been initially set to true. We revert this to the value of the
+        // source index
+        if (sourceIndexMetadata.isHidden() == false) {
+            if (sourceIndexMetadata.getSettings().keySet().contains(IndexMetadata.SETTING_INDEX_HIDDEN)) {
+                settings.put(IndexMetadata.SETTING_INDEX_HIDDEN, false);
+            } else {
+                settings.putNull(IndexMetadata.SETTING_INDEX_HIDDEN);
+            }
+        }
+        UpdateSettingsRequest updateSettingsReq = new UpdateSettingsRequest(settings.build(), downsampleIndexName);
+        updateSettingsReq.setParentTask(parentTask);
+        client.admin()
+            .indices()
+            .updateSettings(
+                updateSettingsReq,
+                new UpdateDownsampleIndexSettingsActionListener(listener, parentTask, downsampleIndexName, request.getWaitTimeout())
+            );
+    }
+
+    private static DownsampleShardTaskParams createPersistentTaskParams(
+        final DownsampleConfig downsampleConfig,
+        final IndexMetadata sourceIndexMetadata,
+        final String targetIndexName,
+        final List<String> metricFields,
+        final List<String> labelFields,
+        final ShardId shardId
+    ) {
+        return new DownsampleShardTaskParams(
+            downsampleConfig,
+            targetIndexName,
+            parseTimestamp(sourceIndexMetadata, IndexSettings.TIME_SERIES_START_TIME),
+            parseTimestamp(sourceIndexMetadata, IndexSettings.TIME_SERIES_END_TIME),
+            shardId,
+            metricFields.toArray(new String[0]),
+            labelFields.toArray(new String[0])
+        );
+    }
+
+    private static long parseTimestamp(final IndexMetadata sourceIndexMetadata, final Setting<Instant> timestampSetting) {
+        return OffsetDateTime.parse(sourceIndexMetadata.getSettings().get(timestampSetting.getKey()), DateTimeFormatter.ISO_DATE_TIME)
+            .toInstant()
+            .toEpochMilli();
+    }
+
+    private static String createPersistentTaskId(final String targetIndex, final ShardId shardId, final DateHistogramInterval interval) {
+        return DOWNSAMPLED_INDEX_PREFIX + targetIndex + "-" + shardId.id() + "-" + interval;
+    }
+
     @Override
     protected ClusterBlockException checkBlock(DownsampleAction.Request request, ClusterState state) {
         return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE);
     }
 
     /**
-     * This method creates the mapping for the rollup index, based on the
+     * This method creates the mapping for the downsample index, based on the
      * mapping (dimensions and metrics) from the source index, as well as the
-     * rollup configuration.
+     * downsample configuration.
      *
-     * @param config the rollup configuration
+     * @param config the downsample configuration
      * @param sourceIndexMappings a map with the source index mapping
-     * @return the mapping of the rollup index
+     * @return the mapping of the downsample index
      */
-    public static String createRollupIndexMapping(
+    public static String createDownsampleIndexMapping(
         final TimeseriesFieldTypeHelper helper,
         final DownsampleConfig config,
         final MapperService mapperService,
@@ -457,10 +528,10 @@ public class TransportDownsampleAction extends AcknowledgedTransportMasterNodeAc
         builder.endObject(); // match initial startObject
         builder.endObject(); // match startObject("properties")
 
-        final CompressedXContent rollupDiffXContent = CompressedXContent.fromJSON(
+        final CompressedXContent mappingDiffXContent = CompressedXContent.fromJSON(
             XContentHelper.convertToJson(BytesReference.bytes(builder), false, XContentType.JSON)
         );
-        return mapperService.merge(MapperService.SINGLE_MAPPING_NAME, rollupDiffXContent, MapperService.MergeReason.INDEX_TEMPLATE)
+        return mapperService.merge(MapperService.SINGLE_MAPPING_NAME, mappingDiffXContent, MapperService.MergeReason.INDEX_TEMPLATE)
             .mappingSource()
             .uncompressed()
             .utf8ToString();
@@ -572,23 +643,23 @@ public class TransportDownsampleAction extends AcknowledgedTransportMasterNodeAc
     }
 
     /**
-     * Copy index settings from the source index to the rollup index. Settings that
-     * have already been set in the rollup index will not be overridden.
+     * Copy index settings from the source index to the downsample index. Settings that
+     * have already been set in the downsample index will not be overridden.
      */
     static IndexMetadata.Builder copyIndexMetadata(
         final IndexMetadata sourceIndexMetadata,
-        final IndexMetadata rollupIndexMetadata,
+        final IndexMetadata downsampleIndexMetadata,
         final IndexScopedSettings indexScopedSettings
     ) {
         // Copy index settings from the source index, but do not override the settings
-        // that already have been set in the rollup index
-        final Settings.Builder targetSettings = Settings.builder().put(rollupIndexMetadata.getSettings());
+        // that already have been set in the downsample index
+        final Settings.Builder targetSettings = Settings.builder().put(downsampleIndexMetadata.getSettings());
         for (final String key : sourceIndexMetadata.getSettings().keySet()) {
             final Setting<?> setting = indexScopedSettings.get(key);
             if (setting == null) {
                 assert indexScopedSettings.isPrivateSetting(key) : "expected [" + key + "] to be private but it was not";
             } else if (setting.getProperties().contains(Setting.Property.NotCopyableOnResize)) {
-                // we leverage the NotCopyableOnResize setting property for rollup, because
+                // we leverage the NotCopyableOnResize setting property for downsample, because
                 // the same rules with resize apply
                 continue;
             }
@@ -600,7 +671,7 @@ public class TransportDownsampleAction extends AcknowledgedTransportMasterNodeAc
             if (OVERRIDE_SETTINGS.contains(key)) {
                 targetSettings.put(key, sourceIndexMetadata.getSettings().get(key));
             }
-            // Do not override settings that have already been set in the rollup index.
+            // Do not override settings that have already been set in the downsample index.
             if (targetSettings.keys().contains(key)) {
                 continue;
             }
@@ -609,8 +680,8 @@ public class TransportDownsampleAction extends AcknowledgedTransportMasterNodeAc
         }
 
         /*
-         * Add the source index name and UUID to the rollup index metadata.
-         * If the source index is a rollup index, we will add the name and UUID
+         * Add the source index name and UUID to the downsample index metadata.
+         * If the source index is a downsample index, we will add the name and UUID
          * of the first index that we initially rolled up.
          */
         if (IndexMetadata.INDEX_DOWNSAMPLE_SOURCE_UUID.exists(sourceIndexMetadata.getSettings()) == false
@@ -620,7 +691,7 @@ public class TransportDownsampleAction extends AcknowledgedTransportMasterNodeAc
                 .put(IndexMetadata.INDEX_DOWNSAMPLE_SOURCE_UUID.getKey(), sourceIndex.getUUID());
         }
 
-        return IndexMetadata.builder(rollupIndexMetadata).settings(targetSettings);
+        return IndexMetadata.builder(downsampleIndexMetadata).settings(targetSettings);
     }
 
     /**
@@ -639,28 +710,28 @@ public class TransportDownsampleAction extends AcknowledgedTransportMasterNodeAc
             .endArray();
     }
 
-    private void createRollupIndex(
-        String rollupIndexName,
+    private void createDownsampleIndex(
+        String downsampleIndexName,
         IndexMetadata sourceIndexMetadata,
         String mapping,
         DownsampleAction.Request request,
         ActionListener<AcknowledgedResponse> listener
     ) {
         /*
-         * When creating the rollup index, we copy the index.number_of_shards from source index,
+         * When creating the downsample index, we copy the index.number_of_shards from source index,
          * and we set the index.number_of_replicas to 0, to avoid replicating the index being built.
          * Also, we set the index.refresh_interval to -1.
          * We will set the correct number of replicas and refresh the index later.
          *
-         * We should note that there is a risk of losing a node during the rollup process. In this
-         * case rollup will fail.
+         * We should note that there is a risk of losing a node during the downsample process. In this
+         * case downsample will fail.
          */
         Settings.Builder builder = Settings.builder()
             .put(IndexMetadata.SETTING_INDEX_HIDDEN, true)
             .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, sourceIndexMetadata.getNumberOfShards())
             .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0)
             .put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), "-1")
-            .put(IndexMetadata.INDEX_DOWNSAMPLE_STATUS.getKey(), IndexMetadata.DownsampleTaskStatus.STARTED);
+            .put(IndexMetadata.INDEX_DOWNSAMPLE_STATUS.getKey(), DownsampleTaskStatus.STARTED);
         if (sourceIndexMetadata.getSettings().hasValue(MapperService.INDEX_MAPPING_TOTAL_FIELDS_LIMIT_SETTING.getKey())) {
             builder.put(
                 MapperService.INDEX_MAPPING_TOTAL_FIELDS_LIMIT_SETTING.getKey(),
@@ -669,110 +740,190 @@ public class TransportDownsampleAction extends AcknowledgedTransportMasterNodeAc
         }
 
         CreateIndexClusterStateUpdateRequest createIndexClusterStateUpdateRequest = new CreateIndexClusterStateUpdateRequest(
-            "rollup",
-            rollupIndexName,
-            rollupIndexName
-        ).settings(builder.build()).mappings(mapping);
+            "downsample",
+            downsampleIndexName,
+            downsampleIndexName
+        ).settings(builder.build()).mappings(mapping).waitForActiveShards(ActiveShardCount.ONE);
         var delegate = new AllocationActionListener<>(listener, threadPool.getThreadContext());
-        taskQueue.submitTask("create-rollup-index [" + rollupIndexName + "]", new RollupClusterStateUpdateTask(listener) {
+        taskQueue.submitTask("create-downsample-index [" + downsampleIndexName + "]", new DownsampleClusterStateUpdateTask(listener) {
             @Override
             public ClusterState execute(ClusterState currentState) throws Exception {
                 return metadataCreateIndexService.applyCreateIndexRequest(
                     currentState,
                     createIndexClusterStateUpdateRequest,
                     true,
-                    // Copy index metadata from source index to rollup index
-                    (builder, rollupIndexMetadata) -> builder.put(
-                        copyIndexMetadata(sourceIndexMetadata, rollupIndexMetadata, indexScopedSettings)
-                    ),
+                    // Copy index metadata from source index to downsample index
+                    (builder, indexMetadata) -> builder.put(copyIndexMetadata(sourceIndexMetadata, indexMetadata, indexScopedSettings)),
                     delegate.reroute()
                 );
             }
         }, request.masterNodeTimeout());
     }
 
-    private void updateRollupMetadata(
-        String rollupIndexName,
-        DownsampleAction.Request request,
-        ActionListener<AcknowledgedResponse> listener
-    ) {
-        // 6. Mark rollup index as "completed successfully" ("index.rollup.status": "success")
-        taskQueue.submitTask("update-rollup-metadata [" + rollupIndexName + "]", new RollupClusterStateUpdateTask(listener) {
+    /**
+     * A specialized cluster state update task that always takes a listener handling an
+     * AcknowledgedResponse, as all template actions have simple acknowledged yes/no responses.
+     */
+    abstract static class DownsampleClusterStateUpdateTask implements ClusterStateTaskListener {
+        final ActionListener<AcknowledgedResponse> listener;
 
-            @Override
-            public ClusterState execute(ClusterState currentState) {
-                Metadata metadata = currentState.metadata();
-                Metadata.Builder metadataBuilder = Metadata.builder(metadata);
-                Index rollupIndex = metadata.index(rollupIndexName).getIndex();
-                IndexMetadata rollupIndexMetadata = metadata.index(rollupIndex);
-
-                metadataBuilder.updateSettings(
-                    Settings.builder()
-                        .put(rollupIndexMetadata.getSettings())
-                        .put(IndexMetadata.INDEX_DOWNSAMPLE_STATUS.getKey(), IndexMetadata.DownsampleTaskStatus.SUCCESS)
-                        .build(),
-                    rollupIndexName
-                );
-                return ClusterState.builder(currentState).metadata(metadataBuilder.build()).build();
-            }
-        }, request.masterNodeTimeout());
-    }
+        DownsampleClusterStateUpdateTask(ActionListener<AcknowledgedResponse> listener) {
+            this.listener = listener;
+        }
 
-    private void refreshIndex(String index, TaskId parentTask, ActionListener<RefreshResponse> listener) {
-        RefreshRequest request = new RefreshRequest(index);
-        request.setParentTask(parentTask);
-        client.admin().indices().refresh(request, listener);
+        public abstract ClusterState execute(ClusterState currentState) throws Exception;
+
+        @Override
+        public void onFailure(Exception e) {
+            listener.onFailure(e);
+        }
     }
 
-    private void forceMergeIndex(String index, TaskId parentTask, ActionListener<ForceMergeResponse> listener) {
-        ForceMergeRequest request = new ForceMergeRequest(index);
-        request.maxNumSegments(1);
-        request.setParentTask(parentTask);
-        client.admin().indices().forceMerge(request, listener);
+    /**
+     * Refreshes the downsample target index
+     */
+    class UpdateDownsampleIndexSettingsActionListener implements ActionListener<AcknowledgedResponse> {
+        final ActionListener<AcknowledgedResponse> listener;
+        final TaskId parentTask;
+        final String downsampleIndexName;
+        final TimeValue timeout;
+
+        UpdateDownsampleIndexSettingsActionListener(
+            final ActionListener<AcknowledgedResponse> listener,
+            final TaskId parentTask,
+            final String downsampleIndexName,
+            final TimeValue timeout
+        ) {
+            this.listener = listener;
+            this.parentTask = parentTask;
+            this.downsampleIndexName = downsampleIndexName;
+            this.timeout = timeout;
+        }
+
+        @Override
+        public void onResponse(final AcknowledgedResponse response) {
+            final RefreshRequest request = new RefreshRequest(downsampleIndexName);
+            request.setParentTask(parentTask);
+            client.admin()
+                .indices()
+                .refresh(request, new RefreshDownsampleIndexActionListener(listener, parentTask, downsampleIndexName, timeout));
+        }
+
+        @Override
+        public void onFailure(Exception e) {
+            listener.onFailure(e);
+        }
+
     }
 
-    private void deleteRollupIndex(
-        String sourceIndex,
-        String rollupIndex,
-        TaskId parentTask,
-        ActionListener<AcknowledgedResponse> listener,
-        Exception e
-    ) {
-        DeleteIndexRequest request = new DeleteIndexRequest(rollupIndex);
-        request.setParentTask(parentTask);
-        client.admin().indices().delete(request, new ActionListener<>() {
-            @Override
-            public void onResponse(AcknowledgedResponse acknowledgedResponse) {
-                if (e == null && acknowledgedResponse.isAcknowledged()) {
-                    listener.onResponse(acknowledgedResponse);
-                } else {
-                    listener.onFailure(new ElasticsearchException("Unable to rollup index [" + sourceIndex + "]", e));
-                }
-            }
+    /**
+     * Updates the downsample target index metadata (task status)
+     */
+    class RefreshDownsampleIndexActionListener implements ActionListener<RefreshResponse> {
+
+        private final ActionListener<AcknowledgedResponse> actionListener;
+        private final TaskId parentTask;
+        private final String downsampleIndexName;
+        private final TimeValue timeout;
+
+        RefreshDownsampleIndexActionListener(
+            final ActionListener<AcknowledgedResponse> actionListener,
+            TaskId parentTask,
+            final String downsampleIndexName,
+            final TimeValue timeout
+        ) {
+            this.actionListener = actionListener;
+            this.parentTask = parentTask;
+            this.downsampleIndexName = downsampleIndexName;
+            this.timeout = timeout;
+        }
 
-            @Override
-            public void onFailure(Exception deleteException) {
-                listener.onFailure(new ElasticsearchException("Unable to delete rollup index [" + rollupIndex + "]", e));
+        @Override
+        public void onResponse(final RefreshResponse response) {
+            if (response.getFailedShards() != 0) {
+                logger.info("Post refresh failed [{}],{}", downsampleIndexName, Strings.toString(response));
             }
-        });
+            // Mark downsample index as "completed successfully" ("index.downsample.status": "success")
+            taskQueue.submitTask(
+                "update-downsample-metadata [" + downsampleIndexName + "]",
+                new DownsampleClusterStateUpdateTask(new ForceMergeActionListener(parentTask, downsampleIndexName, actionListener)) {
+
+                    @Override
+                    public ClusterState execute(ClusterState currentState) {
+                        final Metadata metadata = currentState.metadata();
+                        final IndexMetadata downsampleIndex = metadata.index(metadata.index(downsampleIndexName).getIndex());
+                        if (IndexMetadata.INDEX_DOWNSAMPLE_STATUS.get(downsampleIndex.getSettings()) == DownsampleTaskStatus.SUCCESS) {
+                            return currentState;
+                        }
+
+                        final Metadata.Builder metadataBuilder = Metadata.builder(metadata);
+                        metadataBuilder.updateSettings(
+                            Settings.builder()
+                                .put(downsampleIndex.getSettings())
+                                .put(IndexMetadata.INDEX_DOWNSAMPLE_STATUS.getKey(), DownsampleTaskStatus.SUCCESS)
+                                .build(),
+                            downsampleIndexName
+                        );
+                        return ClusterState.builder(currentState).metadata(metadataBuilder.build()).build();
+                    }
+                },
+                timeout
+            );
+        }
+
+        @Override
+        public void onFailure(Exception e) {
+            actionListener.onFailure(e);
+        }
+
     }
 
     /**
-     * A specialized cluster state update task that always takes a listener handling an
-     * AcknowledgedResponse, as all template actions have simple acknowledged yes/no responses.
+     * Triggers a force merge operation on the downsample target index
      */
-    private abstract static class RollupClusterStateUpdateTask implements ClusterStateTaskListener {
-        final ActionListener<AcknowledgedResponse> listener;
-
-        RollupClusterStateUpdateTask(ActionListener<AcknowledgedResponse> listener) {
-            this.listener = listener;
+    class ForceMergeActionListener implements ActionListener<AcknowledgedResponse> {
+
+        final ActionListener<AcknowledgedResponse> actionListener;
+        private final TaskId parentTask;
+        private final String downsampleIndexName;
+
+        ForceMergeActionListener(
+            final TaskId parentTask,
+            final String downsampleIndexName,
+            final ActionListener<AcknowledgedResponse> onFailure
+        ) {
+            this.parentTask = parentTask;
+            this.downsampleIndexName = downsampleIndexName;
+            this.actionListener = onFailure;
         }
 
-        public abstract ClusterState execute(ClusterState currentState) throws Exception;
+        @Override
+        public void onResponse(final AcknowledgedResponse response) {
+            /*
+             * At this point downsample index has been created
+             * successfully even force merge fails.
+             * So, we should not fail the downsmaple operation
+             */
+            ForceMergeRequest request = new ForceMergeRequest(downsampleIndexName);
+            request.maxNumSegments(1);
+            request.setParentTask(parentTask);
+            client.admin()
+                .indices()
+                .forceMerge(request, ActionListener.wrap(mergeIndexResp -> actionListener.onResponse(AcknowledgedResponse.TRUE), t -> {
+                    /*
+                     * At this point downsampel index has been created
+                     * successfully even force merge fails.
+                     * So, we should not fail the downsample operation
+                     */
+                    logger.error("Failed to force-merge " + "downsample index [" + downsampleIndexName + "]", t);
+                    actionListener.onResponse(AcknowledgedResponse.TRUE);
+                }));
+        }
 
         @Override
         public void onFailure(Exception e) {
-            listener.onFailure(e);
+            this.actionListener.onFailure(e);
         }
+
     }
 }

+ 4 - 1
x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/downsample/TransportDownsampleIndexerAction.java

@@ -29,6 +29,8 @@ import org.elasticsearch.tasks.Task;
 import org.elasticsearch.transport.TransportService;
 import org.elasticsearch.xpack.core.ClientHelper;
 import org.elasticsearch.xpack.core.downsample.DownsampleIndexerAction;
+import org.elasticsearch.xpack.core.rollup.action.RollupShardIndexerStatus;
+import org.elasticsearch.xpack.core.rollup.action.RollupShardPersistentTaskState;
 import org.elasticsearch.xpack.core.rollup.action.RollupShardTask;
 import org.elasticsearch.xpack.rollup.Rollup;
 
@@ -142,7 +144,8 @@ public class TransportDownsampleIndexerAction extends TransportBroadcastAction<
             request.getRollupIndex(),
             request.getRollupConfig(),
             request.getMetricFields(),
-            request.getLabelFields()
+            request.getLabelFields(),
+            new RollupShardPersistentTaskState(RollupShardIndexerStatus.INITIALIZED, null)
         );
         return indexer.execute();
     }

+ 98 - 2
x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/Rollup.java

@@ -12,7 +12,9 @@ import org.elasticsearch.action.ActionResponse;
 import org.elasticsearch.client.internal.Client;
 import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
 import org.elasticsearch.cluster.node.DiscoveryNodes;
+import org.elasticsearch.cluster.routing.allocation.AllocationService;
 import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.scheduler.SchedulerEngine;
 import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.IndexScopedSettings;
@@ -20,15 +22,26 @@ import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.settings.SettingsFilter;
 import org.elasticsearch.common.settings.SettingsModule;
 import org.elasticsearch.common.util.concurrent.EsExecutors;
+import org.elasticsearch.env.Environment;
+import org.elasticsearch.env.NodeEnvironment;
+import org.elasticsearch.indices.IndicesService;
+import org.elasticsearch.persistent.PersistentTaskParams;
+import org.elasticsearch.persistent.PersistentTaskState;
 import org.elasticsearch.persistent.PersistentTasksExecutor;
 import org.elasticsearch.plugins.ActionPlugin;
 import org.elasticsearch.plugins.PersistentTaskPlugin;
 import org.elasticsearch.plugins.Plugin;
+import org.elasticsearch.repositories.RepositoriesService;
 import org.elasticsearch.rest.RestController;
 import org.elasticsearch.rest.RestHandler;
+import org.elasticsearch.script.ScriptService;
 import org.elasticsearch.threadpool.ExecutorBuilder;
 import org.elasticsearch.threadpool.FixedExecutorBuilder;
 import org.elasticsearch.threadpool.ThreadPool;
+import org.elasticsearch.tracing.Tracer;
+import org.elasticsearch.watcher.ResourceWatcherService;
+import org.elasticsearch.xcontent.NamedXContentRegistry;
+import org.elasticsearch.xcontent.ParseField;
 import org.elasticsearch.xpack.core.action.XPackInfoFeatureAction;
 import org.elasticsearch.xpack.core.action.XPackUsageFeatureAction;
 import org.elasticsearch.xpack.core.downsample.DownsampleAction;
@@ -40,9 +53,13 @@ import org.elasticsearch.xpack.core.rollup.action.GetRollupIndexCapsAction;
 import org.elasticsearch.xpack.core.rollup.action.GetRollupJobsAction;
 import org.elasticsearch.xpack.core.rollup.action.PutRollupJobAction;
 import org.elasticsearch.xpack.core.rollup.action.RollupSearchAction;
+import org.elasticsearch.xpack.core.rollup.action.RollupShardPersistentTaskState;
+import org.elasticsearch.xpack.core.rollup.action.RollupShardTask;
 import org.elasticsearch.xpack.core.rollup.action.StartRollupJobAction;
 import org.elasticsearch.xpack.core.rollup.action.StopRollupJobAction;
+import org.elasticsearch.xpack.downsample.DownsampleShardTaskParams;
 import org.elasticsearch.xpack.downsample.RestDownsampleAction;
+import org.elasticsearch.xpack.downsample.RollupShardPersistentTaskExecutor;
 import org.elasticsearch.xpack.downsample.TransportDownsampleAction;
 import org.elasticsearch.xpack.downsample.TransportDownsampleIndexerAction;
 import org.elasticsearch.xpack.rollup.action.TransportDeleteRollupJobAction;
@@ -65,7 +82,7 @@ import org.elasticsearch.xpack.rollup.rest.RestStopRollupJobAction;
 
 import java.time.Clock;
 import java.util.Arrays;
-import java.util.Collections;
+import java.util.Collection;
 import java.util.List;
 import java.util.function.Supplier;
 
@@ -86,6 +103,7 @@ public class Rollup extends Plugin implements ActionPlugin, PersistentTaskPlugin
 
     private final SetOnce<SchedulerEngine> schedulerEngine = new SetOnce<>();
     private final Settings settings;
+    private IndicesService indicesService;
 
     public Rollup(Settings settings) {
         this.settings = settings;
@@ -164,7 +182,15 @@ public class Rollup extends Plugin implements ActionPlugin, PersistentTaskPlugin
         IndexNameExpressionResolver expressionResolver
     ) {
         schedulerEngine.set(new SchedulerEngine(settings, getClock()));
-        return Collections.singletonList(new RollupJobTask.RollupJobPersistentTasksExecutor(client, schedulerEngine.get(), threadPool));
+        return List.of(
+            new RollupJobTask.RollupJobPersistentTasksExecutor(client, schedulerEngine.get(), threadPool),
+            new RollupShardPersistentTaskExecutor(
+                client,
+                this.indicesService,
+                RollupShardTask.TASK_NAME,
+                Rollup.DOWSAMPLE_TASK_THREAD_POOL_NAME
+            )
+        );
     }
 
     // overridable by tests
@@ -178,4 +204,74 @@ public class Rollup extends Plugin implements ActionPlugin, PersistentTaskPlugin
             schedulerEngine.get().stop();
         }
     }
+
+    @Override
+    public List<NamedXContentRegistry.Entry> getNamedXContent() {
+        return List.of(
+            new NamedXContentRegistry.Entry(
+                PersistentTaskState.class,
+                new ParseField(RollupShardPersistentTaskState.NAME),
+                RollupShardPersistentTaskState::fromXContent
+            ),
+            new NamedXContentRegistry.Entry(
+                PersistentTaskParams.class,
+                new ParseField(DownsampleShardTaskParams.NAME),
+                DownsampleShardTaskParams::fromXContent
+            )
+        );
+    }
+
+    @Override
+    public List<NamedWriteableRegistry.Entry> getNamedWriteables() {
+        return List.of(
+            new NamedWriteableRegistry.Entry(
+                PersistentTaskState.class,
+                RollupShardPersistentTaskState.NAME,
+                RollupShardPersistentTaskState::readFromStream
+            ),
+            new NamedWriteableRegistry.Entry(
+                PersistentTaskParams.class,
+                DownsampleShardTaskParams.NAME,
+                DownsampleShardTaskParams::readFromStream
+            )
+        );
+    }
+
+    @Override
+    public Collection<Object> createComponents(
+        final Client client,
+        final ClusterService clusterService,
+        final ThreadPool threadPool,
+        final ResourceWatcherService resourceWatcherService,
+        final ScriptService scriptService,
+        final NamedXContentRegistry xContentRegistry,
+        final Environment environment,
+        final NodeEnvironment nodeEnvironment,
+        final NamedWriteableRegistry namedWriteableRegistry,
+        final IndexNameExpressionResolver indexNameExpressionResolver,
+        final Supplier<RepositoriesService> repositoriesServiceSupplier,
+        final Tracer tracer,
+        final AllocationService allocationService,
+        final IndicesService indicesService
+    ) {
+        final Collection<Object> components = super.createComponents(
+            client,
+            clusterService,
+            threadPool,
+            resourceWatcherService,
+            scriptService,
+            xContentRegistry,
+            environment,
+            nodeEnvironment,
+            namedWriteableRegistry,
+            indexNameExpressionResolver,
+            repositoriesServiceSupplier,
+            tracer,
+            allocationService,
+            indicesService
+        );
+
+        this.indicesService = indicesService;
+        return components;
+    }
 }

+ 372 - 65
x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/downsample/DownsampleActionSingleNodeTests.java

@@ -6,6 +6,7 @@
  */
 package org.elasticsearch.xpack.downsample;
 
+import org.apache.lucene.util.BytesRef;
 import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.ResourceAlreadyExistsException;
 import org.elasticsearch.action.ActionListener;
@@ -37,6 +38,7 @@ import org.elasticsearch.common.network.NetworkAddress;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.time.DateFormatter;
 import org.elasticsearch.common.unit.ByteSizeValue;
+import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.datastreams.DataStreamsPlugin;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.index.IndexMode;
@@ -48,8 +50,10 @@ import org.elasticsearch.index.mapper.DateFieldMapper;
 import org.elasticsearch.index.mapper.MapperService;
 import org.elasticsearch.index.mapper.TimeSeriesIdFieldMapper;
 import org.elasticsearch.index.mapper.TimeSeriesParams;
+import org.elasticsearch.index.query.TermQueryBuilder;
 import org.elasticsearch.index.shard.IndexShard;
 import org.elasticsearch.indices.IndicesService;
+import org.elasticsearch.persistent.PersistentTasksService;
 import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.rest.RestStatus;
 import org.elasticsearch.search.SearchHit;
@@ -71,8 +75,8 @@ import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilde
 import org.elasticsearch.search.sort.SortBuilders;
 import org.elasticsearch.search.sort.SortOrder;
 import org.elasticsearch.tasks.TaskCancelHelper;
-import org.elasticsearch.tasks.TaskCancelledException;
 import org.elasticsearch.tasks.TaskId;
+import org.elasticsearch.tasks.TaskManager;
 import org.elasticsearch.test.ESSingleNodeTestCase;
 import org.elasticsearch.xcontent.XContentBuilder;
 import org.elasticsearch.xcontent.XContentFactory;
@@ -84,6 +88,7 @@ import org.elasticsearch.xpack.core.ilm.LifecycleSettings;
 import org.elasticsearch.xpack.core.ilm.RolloverAction;
 import org.elasticsearch.xpack.core.rollup.ConfigTestHelpers;
 import org.elasticsearch.xpack.core.rollup.action.RollupShardIndexerStatus;
+import org.elasticsearch.xpack.core.rollup.action.RollupShardPersistentTaskState;
 import org.elasticsearch.xpack.core.rollup.action.RollupShardTask;
 import org.elasticsearch.xpack.ilm.IndexLifecycle;
 import org.elasticsearch.xpack.rollup.Rollup;
@@ -105,6 +110,7 @@ import java.util.Optional;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.stream.Collectors;
 
 import static java.util.Collections.emptyMap;
@@ -114,6 +120,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitC
 import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.equalTo;
+import static org.mockito.Mockito.mock;
 
 public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
 
@@ -140,6 +147,7 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
 
     private static final int MAX_DIM_VALUES = 5;
     private static final long MAX_NUM_BUCKETS = 10;
+    public static final TimeValue TIMEOUT = new TimeValue(1, TimeUnit.MINUTES);
 
     private String sourceIndex, rollupIndex;
     private long startTime;
@@ -162,9 +170,9 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
         sourceIndex = randomAlphaOfLength(14).toLowerCase(Locale.ROOT);
         rollupIndex = "rollup-" + sourceIndex;
         startTime = randomLongBetween(946769284000L, 1607470084000L); // random date between 2000-2020
-        docCount = randomIntBetween(10, 9000);
-        numOfShards = randomIntBetween(1, 4);
-        numOfReplicas = randomIntBetween(0, 3);
+        docCount = randomIntBetween(1000, 9000);
+        numOfShards = randomIntBetween(1, 1);
+        numOfReplicas = randomIntBetween(0, 0);
 
         // Values for keyword dimensions
         dimensionValues = new ArrayList<>(MAX_DIM_VALUES);
@@ -287,7 +295,7 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
                 .endObject();
         };
         bulkIndex(sourceSupplier);
-        prepareSourceIndex(sourceIndex);
+        prepareSourceIndex(sourceIndex, true);
         rollup(sourceIndex, rollupIndex, config);
         assertRollupIndex(sourceIndex, rollupIndex, config);
     }
@@ -324,7 +332,7 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
         bulkIndex(sourceSupplier);
 
         // Downsample the source index
-        prepareSourceIndex(sourceIndex);
+        prepareSourceIndex(sourceIndex, true);
         rollup(sourceIndex, rollupIndex, config);
         assertRollupIndex(sourceIndex, rollupIndex, config);
 
@@ -375,7 +383,7 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
                 .endObject();
         };
         bulkIndex(sourceSupplier);
-        prepareSourceIndex(sourceIndex);
+        prepareSourceIndex(sourceIndex, true);
         rollup(sourceIndex, rollupIndex, config);
 
         GetIndexResponse indexSettingsResp = indicesAdmin().prepareGetIndex().addIndices(sourceIndex, rollupIndex).get();
@@ -431,14 +439,14 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
             return builder.endObject();
         };
         bulkIndex(sourceSupplier);
-        prepareSourceIndex(sourceIndex);
+        prepareSourceIndex(sourceIndex, true);
         rollup(sourceIndex, rollupIndex, config);
         assertRollupIndex(sourceIndex, rollupIndex, config);
     }
 
     public void testCannotRollupToExistingIndex() throws Exception {
         DownsampleConfig config = new DownsampleConfig(randomInterval());
-        prepareSourceIndex(sourceIndex);
+        prepareSourceIndex(sourceIndex, true);
 
         // Create an empty index with the same name as the rollup index
         assertAcked(indicesAdmin().prepareCreate(rollupIndex).setSettings(indexSettings(1, 0)).get());
@@ -452,7 +460,7 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
     public void testRollupEmptyIndex() throws IOException {
         DownsampleConfig config = new DownsampleConfig(randomInterval());
         // Source index has been created in the setup() method
-        prepareSourceIndex(sourceIndex);
+        prepareSourceIndex(sourceIndex, true);
         rollup(sourceIndex, rollupIndex, config);
         assertRollupIndex(sourceIndex, rollupIndex, config);
     }
@@ -478,7 +486,7 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
             .get();
 
         DownsampleConfig config = new DownsampleConfig(randomInterval());
-        prepareSourceIndex(sourceIndex);
+        prepareSourceIndex(sourceIndex, true);
         rollup(sourceIndex, rollupIndex, config);
         assertRollupIndex(sourceIndex, rollupIndex, config);
     }
@@ -506,7 +514,7 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
             .field(FIELD_NUMERIC_1, randomDouble())
             .endObject();
         bulkIndex(sourceSupplier);
-        prepareSourceIndex(sourceIndex);
+        prepareSourceIndex(sourceIndex, true);
         var rollupListener = new ActionListener<AcknowledgedResponse>() {
             boolean success;
 
@@ -524,7 +532,11 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
                 fail("Rollup failed: " + e.getMessage());
             }
         };
-        client().execute(DownsampleAction.INSTANCE, new DownsampleAction.Request(sourceIndex, rollupIndex, config), rollupListener);
+        client().execute(
+            DownsampleAction.INSTANCE,
+            new DownsampleAction.Request(sourceIndex, rollupIndex, TIMEOUT, config),
+            rollupListener
+        );
         assertBusy(() -> {
             try {
                 assertEquals(indicesAdmin().prepareGetIndex().addIndices(rollupIndex).get().getIndices().length, 1);
@@ -532,11 +544,7 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
                 fail("rollup index has not been created");
             }
         });
-        ResourceAlreadyExistsException exception = expectThrows(
-            ResourceAlreadyExistsException.class,
-            () -> rollup(sourceIndex, rollupIndex, config)
-        );
-        assertThat(exception.getMessage(), containsString(rollupIndex));
+        rollup(sourceIndex, rollupIndex, config);
         // We must wait until the in-progress rollup ends, otherwise data will not be cleaned up
         assertBusy(() -> assertTrue("In progress rollup did not complete", rollupListener.success), 60, TimeUnit.SECONDS);
     }
@@ -556,10 +564,10 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
                 .field(FIELD_NUMERIC_2, DATE_FORMATTER.parseMillis(ts))
                 .endObject();
         };
-        bulkIndex(dataStreamName, sourceSupplier);
+        bulkIndex(dataStreamName, sourceSupplier, docCount);
 
         String sourceIndex = rollover(dataStreamName).getOldIndex();
-        prepareSourceIndex(sourceIndex);
+        prepareSourceIndex(sourceIndex, true);
         String rollupIndex = "rollup-" + sourceIndex;
         rollup(sourceIndex, rollupIndex, config);
         assertRollupIndex(sourceIndex, rollupIndex, config);
@@ -583,7 +591,7 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
             .field(FIELD_NUMERIC_1, randomDouble())
             .endObject();
         bulkIndex(sourceSupplier);
-        prepareSourceIndex(sourceIndex);
+        prepareSourceIndex(sourceIndex, true);
 
         IndicesService indexServices = getInstanceFromNode(IndicesService.class);
         Index srcIndex = resolveIndex(sourceIndex);
@@ -603,6 +611,7 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
             emptyMap(),
             shard.shardId()
         );
+        task.testInit(mock(PersistentTasksService.class), mock(TaskManager.class), randomAlphaOfLength(5), randomIntBetween(1, 5));
         TaskCancelHelper.cancel(task, "test cancel");
 
         // re-use source index as temp index for test
@@ -614,11 +623,12 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
             rollupIndex,
             config,
             new String[] { FIELD_NUMERIC_1, FIELD_NUMERIC_2 },
-            new String[] {}
+            new String[] {},
+            new RollupShardPersistentTaskState(RollupShardIndexerStatus.INITIALIZED, null)
         );
 
-        TaskCancelledException exception = expectThrows(TaskCancelledException.class, () -> indexer.execute());
-        assertThat(exception.getMessage(), equalTo("Shard [" + sourceIndex + "][" + shardNum + "] rollup cancelled"));
+        RollupShardIndexerException exception = expectThrows(RollupShardIndexerException.class, () -> indexer.execute());
+        assertThat(exception.getCause().getMessage(), equalTo("Shard [" + sourceIndex + "][" + shardNum + "] rollup cancelled"));
     }
 
     public void testRollupBulkFailed() throws IOException {
@@ -631,7 +641,39 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
             .field(FIELD_NUMERIC_1, randomDouble())
             .endObject();
         bulkIndex(sourceSupplier);
-        prepareSourceIndex(sourceIndex);
+        prepareSourceIndex(sourceIndex, true);
+
+        IndicesService indexServices = getInstanceFromNode(IndicesService.class);
+        Index srcIndex = resolveIndex(sourceIndex);
+        IndexService indexService = indexServices.indexServiceSafe(srcIndex);
+        int shardNum = randomIntBetween(0, numOfShards - 1);
+        IndexShard shard = indexService.getShard(shardNum);
+        RollupShardTask task = new RollupShardTask(
+            randomLong(),
+            "rollup",
+            "action",
+            TaskId.EMPTY_TASK_ID,
+            rollupIndex,
+            indexService.getIndexSettings().getTimestampBounds().startTime(),
+            indexService.getIndexSettings().getTimestampBounds().endTime(),
+            config,
+            emptyMap(),
+            shard.shardId()
+        );
+        task.testInit(mock(PersistentTasksService.class), mock(TaskManager.class), randomAlphaOfLength(5), randomIntBetween(1, 5));
+
+        // re-use source index as temp index for test
+        RollupShardIndexer indexer = new RollupShardIndexer(
+            task,
+            client(),
+            indexService,
+            shard.shardId(),
+            rollupIndex,
+            config,
+            new String[] { FIELD_NUMERIC_1, FIELD_NUMERIC_2 },
+            new String[] {},
+            new RollupShardPersistentTaskState(RollupShardIndexerStatus.INITIALIZED, null)
+        );
 
         // block rollup index
         assertAcked(
@@ -641,8 +683,19 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
                 .get()
         );
 
-        ElasticsearchException exception = expectThrows(ElasticsearchException.class, () -> rollup(sourceIndex, rollupIndex, config));
-        assertThat(exception.getMessage(), equalTo("Unable to rollup index [" + sourceIndex + "]"));
+        ElasticsearchException exception = expectThrows(ElasticsearchException.class, indexer::execute);
+        assertThat(
+            exception.getMessage(),
+            equalTo(
+                "Downsampling task ["
+                    + task.getPersistentTaskId()
+                    + "] on shard "
+                    + shard.shardId()
+                    + " failed indexing ["
+                    + task.getNumFailed()
+                    + "]"
+            )
+        );
     }
 
     public void testTooManyBytesInFlight() throws IOException {
@@ -655,7 +708,7 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
             .field(FIELD_NUMERIC_1, randomDouble())
             .endObject();
         bulkIndex(sourceSupplier);
-        prepareSourceIndex(sourceIndex);
+        prepareSourceIndex(sourceIndex, true);
 
         IndicesService indexServices = getInstanceFromNode(IndicesService.class);
         Index srcIndex = resolveIndex(sourceIndex);
@@ -674,6 +727,7 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
             emptyMap(),
             shard.shardId()
         );
+        task.testInit(mock(PersistentTasksService.class), mock(TaskManager.class), randomAlphaOfLength(5), randomIntBetween(1, 5));
 
         // re-use source index as temp index for test
         RollupShardIndexer indexer = new RollupShardIndexer(
@@ -684,7 +738,8 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
             rollupIndex,
             config,
             new String[] { FIELD_NUMERIC_1, FIELD_NUMERIC_2 },
-            new String[] {}
+            new String[] {},
+            new RollupShardPersistentTaskState(RollupShardIndexerStatus.INITIALIZED, null)
         );
         /*
          * Here we set the batch size and the total bytes in flight size to tiny numbers so that we are guaranteed to trigger the bulk
@@ -696,6 +751,7 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
     }
 
     public void testRollupStats() throws IOException {
+        final PersistentTasksService persistentTasksService = mock(PersistentTasksService.class);
         final DownsampleConfig config = new DownsampleConfig(randomInterval());
         final SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder()
             .startObject()
@@ -704,7 +760,7 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
             .field(FIELD_NUMERIC_1, randomDouble())
             .endObject();
         bulkIndex(sourceSupplier);
-        prepareSourceIndex(sourceIndex);
+        prepareSourceIndex(sourceIndex, true);
 
         final IndicesService indexServices = getInstanceFromNode(IndicesService.class);
         final Index resolvedSourceIndex = resolveIndex(sourceIndex);
@@ -723,6 +779,7 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
                 emptyMap(),
                 shard.shardId()
             );
+            task.testInit(persistentTasksService, mock(TaskManager.class), randomAlphaOfLength(5), randomIntBetween(1, 5));
 
             final RollupShardIndexer indexer = new RollupShardIndexer(
                 task,
@@ -732,7 +789,8 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
                 rollupIndex,
                 config,
                 new String[] { FIELD_NUMERIC_1, FIELD_NUMERIC_2 },
-                new String[] {}
+                new String[] {},
+                new RollupShardPersistentTaskState(RollupShardIndexerStatus.INITIALIZED, null)
             );
 
             assertEquals(0.0F, task.getDocsProcessedPercentage(), 0.001);
@@ -743,35 +801,199 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
 
             final DownsampleIndexerAction.ShardDownsampleResponse executeResponse = indexer.execute();
 
-            assertEquals(executeResponse.getNumIndexed(), task.getNumIndexed());
-            assertEquals(task.getNumReceived(), task.getTotalShardDocCount());
-            assertEquals(indexService.getShard(shardNum).docStats().getCount(), task.getTotalShardDocCount());
-            assertEquals(100.0F, task.getDocsProcessedPercentage(), 0.001);
-            assertTrue(task.getRollupBulkInfo().bulkTookSumMillis() >= 0);
-            assertEquals(task.getRollupBulkInfo().bulkIngestSumMillis(), task.getRollupBulkInfo().maxBulkIngestMillis());
-            assertEquals(task.getRollupBulkInfo().bulkIngestSumMillis(), task.getRollupBulkInfo().minBulkIngestMillis());
-            assertTrue(task.getRollupBulkInfo().bulkTookSumMillis() >= 0);
-            assertEquals(task.getRollupBulkInfo().bulkTookSumMillis(), task.getRollupBulkInfo().maxBulkTookMillis());
-            assertEquals(task.getRollupBulkInfo().bulkTookSumMillis(), task.getRollupBulkInfo().minBulkTookMillis());
-            assertEquals(1L, task.getRollupBulkInfo().totalBulkCount());
-            assertEquals(indexService.getIndexSettings().getTimestampBounds().startTime(), task.getIndexStartTimeMillis());
-            assertEquals(indexService.getIndexSettings().getTimestampBounds().endTime(), task.getIndexEndTimeMillis());
-            assertEquals(RollupShardIndexerStatus.COMPLETED, task.getRollupShardIndexerStatus());
-            assertEquals(task.getNumSent(), task.getNumIndexed());
-            assertEquals(task.getNumIndexed(), task.getLastBeforeBulkInfo().numberOfActions());
-            assertTrue(task.getLastBeforeBulkInfo().estimatedSizeInBytes() > 0);
-            assertFalse(task.getLastAfterBulkInfo().hasFailures());
-            assertEquals(RestStatus.OK.getStatus(), task.getLastAfterBulkInfo().restStatusCode());
-            assertTrue(task.getLastAfterBulkInfo().lastTookInMillis() >= 0);
-            assertTrue(indexService.getIndexSettings().getTimestampBounds().startTime() <= task.getLastIndexingTimestamp());
-            assertTrue(indexService.getIndexSettings().getTimestampBounds().startTime() <= task.getLastSourceTimestamp());
-            assertTrue(indexService.getIndexSettings().getTimestampBounds().startTime() <= task.getLastTargetTimestamp());
-            assertTrue(indexService.getIndexSettings().getTimestampBounds().endTime() >= task.getLastIndexingTimestamp());
-            assertTrue(indexService.getIndexSettings().getTimestampBounds().endTime() >= task.getLastSourceTimestamp());
-            assertTrue(indexService.getIndexSettings().getTimestampBounds().endTime() >= task.getLastTargetTimestamp());
+            assertRollupIndexer(indexService, shardNum, task, executeResponse, task.getTotalShardDocCount());
         }
     }
 
+    public void testResumeRollup() throws IOException {
+        // create rollup config and index documents into source index
+        DownsampleConfig config = new DownsampleConfig(randomInterval());
+        SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder()
+            .startObject()
+            .field(FIELD_TIMESTAMP, randomDateForInterval(config.getInterval()))
+            .field(FIELD_DIMENSION_1, randomBoolean() ? "dim1" : "dim2")
+            .field(FIELD_NUMERIC_1, randomDouble())
+            .endObject();
+        bulkIndex(sourceSupplier);
+        prepareSourceIndex(sourceIndex, true);
+
+        IndicesService indexServices = getInstanceFromNode(IndicesService.class);
+        Index srcIndex = resolveIndex(sourceIndex);
+        IndexService indexService = indexServices.indexServiceSafe(srcIndex);
+        int shardNum = randomIntBetween(0, numOfShards - 1);
+        IndexShard shard = indexService.getShard(shardNum);
+
+        RollupShardTask task = new RollupShardTask(
+            randomLong(),
+            "rollup",
+            "action",
+            TaskId.EMPTY_TASK_ID,
+            rollupIndex,
+            indexService.getIndexSettings().getTimestampBounds().startTime(),
+            indexService.getIndexSettings().getTimestampBounds().endTime(),
+            config,
+            emptyMap(),
+            shard.shardId()
+        );
+        task.testInit(mock(PersistentTasksService.class), mock(TaskManager.class), randomAlphaOfLength(5), randomIntBetween(1, 5));
+
+        RollupShardIndexer indexer = new RollupShardIndexer(
+            task,
+            client(),
+            indexService,
+            shard.shardId(),
+            rollupIndex,
+            config,
+            new String[] { FIELD_NUMERIC_1, FIELD_NUMERIC_2 },
+            new String[] {},
+            new RollupShardPersistentTaskState(
+                RollupShardIndexerStatus.STARTED,
+                new BytesRef(
+                    new byte[] {
+                        0x01,
+                        0x0C,
+                        0x64,
+                        0x69,
+                        0x6d,
+                        0x65,
+                        0x6E,
+                        0x73,
+                        0x69,
+                        0x6F,
+                        0x6E,
+                        0x5F,
+                        0x6B,
+                        0x77,
+                        0x73,
+                        0x04,
+                        0x64,
+                        0x69,
+                        0x6D,
+                        0x31 }
+                )
+            )
+        );
+
+        final DownsampleIndexerAction.ShardDownsampleResponse response2 = indexer.execute();
+
+        assertRollupIndexer(indexService, shardNum, task, response2, task.getTotalShardDocCount());
+    }
+
+    public void testResumeRollupPartial() throws IOException {
+        // create rollup config and index documents into source index
+        DownsampleConfig config = new DownsampleConfig(randomInterval());
+        SourceSupplier sourceSupplier = () -> XContentFactory.jsonBuilder()
+            .startObject()
+            .field(FIELD_TIMESTAMP, randomDateForInterval(config.getInterval()))
+            .field(FIELD_DIMENSION_1, randomBoolean() ? "dim1" : "dim2")
+            .field(FIELD_NUMERIC_1, randomDouble())
+            .endObject();
+        bulkIndex(sourceSupplier);
+        prepareSourceIndex(sourceIndex, true);
+
+        IndicesService indexServices = getInstanceFromNode(IndicesService.class);
+        Index srcIndex = resolveIndex(sourceIndex);
+        IndexService indexService = indexServices.indexServiceSafe(srcIndex);
+        int shardNum = randomIntBetween(0, numOfShards - 1);
+        IndexShard shard = indexService.getShard(shardNum);
+
+        RollupShardTask task = new RollupShardTask(
+            randomLong(),
+            "rollup",
+            "action",
+            TaskId.EMPTY_TASK_ID,
+            rollupIndex,
+            indexService.getIndexSettings().getTimestampBounds().startTime(),
+            indexService.getIndexSettings().getTimestampBounds().endTime(),
+            config,
+            emptyMap(),
+            shard.shardId()
+        );
+        task.testInit(mock(PersistentTasksService.class), mock(TaskManager.class), randomAlphaOfLength(5), randomIntBetween(1, 5));
+
+        RollupShardIndexer indexer = new RollupShardIndexer(
+            task,
+            client(),
+            indexService,
+            shard.shardId(),
+            rollupIndex,
+            config,
+            new String[] { FIELD_NUMERIC_1, FIELD_NUMERIC_2 },
+            new String[] {},
+            new RollupShardPersistentTaskState(
+                RollupShardIndexerStatus.STARTED,
+                new BytesRef(
+                    new byte[] {
+                        0x01,
+                        0x0C,
+                        0x64,
+                        0x69,
+                        0x6d,
+                        0x65,
+                        0x6E,
+                        0x73,
+                        0x69,
+                        0x6F,
+                        0x6E,
+                        0x5F,
+                        0x6B,
+                        0x77,
+                        0x73,
+                        0x04,
+                        0x64,
+                        0x69,
+                        0x6D,
+                        0x32 }
+                )
+            )
+        );
+
+        final DownsampleIndexerAction.ShardDownsampleResponse response2 = indexer.execute();
+        int dim2DocCount = client().prepareSearch(sourceIndex)
+            .setQuery(new TermQueryBuilder(FIELD_DIMENSION_1, "dim2"))
+            .setSize(10_000)
+            .get()
+            .getHits()
+            .getHits().length;
+
+        assertRollupIndexer(indexService, shardNum, task, response2, dim2DocCount);
+    }
+
+    private static void assertRollupIndexer(
+        final IndexService indexService,
+        int shardNum,
+        final RollupShardTask task,
+        final DownsampleIndexerAction.ShardDownsampleResponse response,
+        long totalShardDocCount
+    ) {
+        assertEquals(response.getNumIndexed(), task.getNumIndexed());
+        assertEquals(task.getNumReceived(), totalShardDocCount);
+        assertEquals(indexService.getShard(shardNum).docStats().getCount(), task.getTotalShardDocCount());
+        assertEquals(100.0D * task.getNumReceived() / task.getTotalShardDocCount(), task.getDocsProcessedPercentage(), 0.001);
+        assertTrue(task.getRollupBulkInfo().bulkTookSumMillis() >= 0);
+        assertEquals(task.getRollupBulkInfo().bulkIngestSumMillis(), task.getRollupBulkInfo().maxBulkIngestMillis());
+        assertEquals(task.getRollupBulkInfo().bulkIngestSumMillis(), task.getRollupBulkInfo().minBulkIngestMillis());
+        assertTrue(task.getRollupBulkInfo().bulkTookSumMillis() >= 0);
+        assertEquals(task.getRollupBulkInfo().bulkTookSumMillis(), task.getRollupBulkInfo().maxBulkTookMillis());
+        assertEquals(task.getRollupBulkInfo().bulkTookSumMillis(), task.getRollupBulkInfo().minBulkTookMillis());
+        assertEquals(1L, task.getRollupBulkInfo().totalBulkCount());
+        assertEquals(indexService.getIndexSettings().getTimestampBounds().startTime(), task.getIndexStartTimeMillis());
+        assertEquals(indexService.getIndexSettings().getTimestampBounds().endTime(), task.getIndexEndTimeMillis());
+        assertEquals(RollupShardIndexerStatus.COMPLETED, task.getRollupShardIndexerStatus());
+        assertEquals(task.getNumSent(), task.getNumIndexed());
+        assertEquals(task.getNumIndexed(), task.getLastBeforeBulkInfo().numberOfActions());
+        assertTrue(task.getLastBeforeBulkInfo().estimatedSizeInBytes() > 0);
+        assertFalse(task.getLastAfterBulkInfo().hasFailures());
+        assertEquals(RestStatus.OK.getStatus(), task.getLastAfterBulkInfo().restStatusCode());
+        assertTrue(task.getLastAfterBulkInfo().lastTookInMillis() >= 0);
+        assertTrue(indexService.getIndexSettings().getTimestampBounds().startTime() <= task.getLastIndexingTimestamp());
+        assertTrue(indexService.getIndexSettings().getTimestampBounds().startTime() <= task.getLastSourceTimestamp());
+        assertTrue(indexService.getIndexSettings().getTimestampBounds().startTime() <= task.getLastTargetTimestamp());
+        assertTrue(indexService.getIndexSettings().getTimestampBounds().endTime() >= task.getLastIndexingTimestamp());
+        assertTrue(indexService.getIndexSettings().getTimestampBounds().endTime() >= task.getLastSourceTimestamp());
+        assertTrue(indexService.getIndexSettings().getTimestampBounds().endTime() >= task.getLastTargetTimestamp());
+    }
+
     private DateHistogramInterval randomInterval() {
         return ConfigTestHelpers.randomInterval();
     }
@@ -786,10 +1008,10 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
     }
 
     private void bulkIndex(SourceSupplier sourceSupplier) throws IOException {
-        bulkIndex(sourceIndex, sourceSupplier);
+        bulkIndex(sourceIndex, sourceSupplier, docCount);
     }
 
-    private void bulkIndex(String indexName, SourceSupplier sourceSupplier) throws IOException {
+    private void bulkIndex(final String indexName, final SourceSupplier sourceSupplier, int docCount) throws IOException {
         BulkRequestBuilder bulkRequestBuilder = client().prepareBulk();
         bulkRequestBuilder.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE);
         for (int i = 0; i < docCount; i++) {
@@ -817,18 +1039,18 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
         assertHitCount(client().prepareSearch(indexName).setSize(0).get(), docsIndexed);
     }
 
-    private void prepareSourceIndex(String sourceIndex) {
+    private void prepareSourceIndex(final String sourceIndex, boolean blockWrite) {
         // Set the source index to read-only state
         assertAcked(
             indicesAdmin().prepareUpdateSettings(sourceIndex)
-                .setSettings(Settings.builder().put(IndexMetadata.INDEX_BLOCKS_WRITE_SETTING.getKey(), true).build())
+                .setSettings(Settings.builder().put(IndexMetadata.INDEX_BLOCKS_WRITE_SETTING.getKey(), blockWrite).build())
                 .get()
         );
     }
 
     private void rollup(String sourceIndex, String rollupIndex, DownsampleConfig config) {
         assertAcked(
-            client().execute(DownsampleAction.INSTANCE, new DownsampleAction.Request(sourceIndex, rollupIndex, config)).actionGet()
+            client().execute(DownsampleAction.INSTANCE, new DownsampleAction.Request(sourceIndex, rollupIndex, TIMEOUT, config)).actionGet()
         );
     }
 
@@ -1269,9 +1491,8 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
                 .endObject()
                 .endObject();
         };
-        docCount = 512; // Hard code to have 512 documents in the source index, otherwise running this test take too long.
-        bulkIndex(sourceIndex, sourceSupplier);
-        prepareSourceIndex(sourceIndex);
+        bulkIndex(sourceIndex, sourceSupplier, 512);
+        prepareSourceIndex(sourceIndex, true);
 
         int n = randomIntBetween(3, 6);
         final CountDownLatch rollupComplete = new CountDownLatch(n);
@@ -1295,4 +1516,90 @@ public class DownsampleActionSingleNodeTests extends ESSingleNodeTestCase {
             assertRollupIndex(sourceIndex, targets.get(i), config);
         }
     }
+
+    public void testDuplicateRollupRequest() throws IOException, InterruptedException {
+        final DownsampleConfig config = new DownsampleConfig(randomInterval());
+        SourceSupplier sourceSupplier = () -> {
+            String ts = randomDateForInterval(config.getInterval());
+            double labelDoubleValue = DATE_FORMATTER.parseMillis(ts);
+            int labelIntegerValue = randomInt();
+            long labelLongValue = randomLong();
+            String labelIpv4Address = NetworkAddress.format(randomIp(true));
+            String labelIpv6Address = NetworkAddress.format(randomIp(false));
+            Date labelDateValue = randomDate();
+            int keywordArraySize = randomIntBetween(2, 5);
+            String[] keywordArray = new String[keywordArraySize];
+            for (int i = 0; i < keywordArraySize; ++i) {
+                keywordArray[i] = randomAlphaOfLength(10);
+            }
+            int doubleArraySize = randomIntBetween(3, 10);
+            double[] doubleArray = new double[doubleArraySize];
+            for (int i = 0; i < doubleArraySize; ++i) {
+                doubleArray[i] = randomDouble();
+            }
+            return XContentFactory.jsonBuilder()
+                .startObject()
+                .field(FIELD_TIMESTAMP, ts)
+                .field(FIELD_DIMENSION_1, randomFrom(dimensionValues))
+                .field(FIELD_DIMENSION_2, randomIntBetween(1, 10))
+                .field(FIELD_NUMERIC_1, randomInt())
+                .field(FIELD_NUMERIC_2, DATE_FORMATTER.parseMillis(ts))
+                .startObject(FIELD_AGG_METRIC)
+                .field("min", randomDoubleBetween(-2000, -1001, true))
+                .field("max", randomDoubleBetween(-1000, 1000, true))
+                .field("sum", randomIntBetween(100, 10000))
+                .field("value_count", randomIntBetween(100, 1000))
+                .endObject()
+                .field(FIELD_LABEL_DOUBLE, labelDoubleValue)
+                .field(FIELD_METRIC_LABEL_DOUBLE, labelDoubleValue)
+                .field(FIELD_LABEL_INTEGER, labelIntegerValue)
+                .field(FIELD_LABEL_KEYWORD, ts)
+                .field(FIELD_LABEL_UNMAPPED, randomBoolean() ? labelLongValue : labelDoubleValue)
+                .field(FIELD_LABEL_TEXT, ts)
+                .field(FIELD_LABEL_BOOLEAN, randomBoolean())
+                .field(FIELD_LABEL_IPv4_ADDRESS, labelIpv4Address)
+                .field(FIELD_LABEL_IPv6_ADDRESS, labelIpv6Address)
+                .field(FIELD_LABEL_DATE, labelDateValue)
+                .field(FIELD_LABEL_KEYWORD_ARRAY, keywordArray)
+                .field(FIELD_LABEL_DOUBLE_ARRAY, doubleArray)
+                .startObject(FIELD_LABEL_AGG_METRIC)
+                .field("min", randomDoubleBetween(-2000, -1001, true))
+                .field("max", randomDoubleBetween(-1000, 1000, true))
+                .field("sum", Double.valueOf(randomIntBetween(100, 10000)))
+                .field("value_count", randomIntBetween(100, 1000))
+                .endObject()
+                .endObject();
+        };
+        bulkIndex(sourceIndex, sourceSupplier, 512);
+        prepareSourceIndex(sourceIndex, true);
+
+        final CountDownLatch rollupComplete = new CountDownLatch(2);
+        final String targetIndex = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
+        AtomicBoolean firstFailed = new AtomicBoolean(false);
+        AtomicBoolean secondFailed = new AtomicBoolean(false);
+        // NOTE: we expect one thread to run the downsample operation and the other one to fail
+        new Thread(() -> {
+            try {
+                rollup(sourceIndex, targetIndex, config);
+            } catch (ResourceAlreadyExistsException e) {
+                firstFailed.set(true);
+            } finally {
+                rollupComplete.countDown();
+            }
+
+        }).start();
+        new Thread(() -> {
+            try {
+                rollup(sourceIndex, targetIndex, config);
+            } catch (ResourceAlreadyExistsException e) {
+                secondFailed.set(true);
+            } finally {
+                rollupComplete.countDown();
+            }
+        }).start();
+
+        assertTrue(rollupComplete.await(30, TimeUnit.SECONDS));
+        assertFalse(firstFailed.get() ^ secondFailed.get());
+        assertRollupIndex(sourceIndex, targetIndex, config);
+    }
 }

+ 5 - 0
x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/downsample/DownsampleDataStreamTests.java

@@ -32,6 +32,7 @@ import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.Template;
 import org.elasticsearch.common.compress.CompressedXContent;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.datastreams.DataStreamsPlugin;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.index.IndexMode;
@@ -59,12 +60,15 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Locale;
 import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
 
 import static org.elasticsearch.cluster.metadata.MetadataIndexTemplateService.DEFAULT_TIMESTAMP_FIELD;
 import static org.hamcrest.Matchers.equalTo;
 
 public class DownsampleDataStreamTests extends ESSingleNodeTestCase {
 
+    public static final TimeValue TIMEOUT = new TimeValue(1, TimeUnit.MINUTES);
+
     @Override
     protected Collection<Class<? extends Plugin>> getPlugins() {
         return List.of(Rollup.class, DataStreamsPlugin.class);
@@ -94,6 +98,7 @@ public class DownsampleDataStreamTests extends ESSingleNodeTestCase {
         final DownsampleAction.Request downsampleRequest = new DownsampleAction.Request(
             rolloverResponse.getOldIndex(),
             downsampleTargetIndex,
+            TIMEOUT,
             new DownsampleConfig(DateHistogramInterval.HOUR)
         );
         final AcknowledgedResponse downsampleResponse = indicesAdmin().execute(DownsampleAction.INSTANCE, downsampleRequest).actionGet();