From 81f281e92b7fafb32dc9dbd11fff179378aff835 Mon Sep 17 00:00:00 2001 From: YannC <37600690+Skraye@users.noreply.github.com> Date: Wed, 3 Apr 2024 14:33:54 +0200 Subject: [PATCH] feat(): retry-flow (#3392) * feat(): retry-flow * fix: rename behavior * feat: created a metadata props for executions * fix(ui): translate + new metadata prop --- .../core/models/executions/Execution.java | 17 +- .../models/executions/ExecutionMetadata.java | 25 +++ .../core/models/executions/TaskRun.java | 43 +++- .../io/kestra/core/models/flows/Flow.java | 12 +- .../io/kestra/core/models/flows/State.java | 13 +- .../models/tasks/retrys/AbstractRetry.java | 8 + .../kestra/core/runners/ExecutionDelay.java | 19 +- .../kestra/core/runners/ExecutorService.java | 70 +++++-- .../io/kestra/core/runners/RunContext.java | 1 + .../java/io/kestra/core/runners/Worker.java | 24 +-- .../core/services/ExecutionService.java | 32 ++- .../core/tasks/flows/RetryCaseTest.java | 193 ++++++++++++++++++ .../valids/retry-failed-flow-attempts.yml | 12 ++ .../valids/retry-failed-flow-duration.yml | 12 ++ .../valids/retry-failed-task-attempts.yml | 12 ++ .../valids/retry-failed-task-duration.yml | 12 ++ .../retry-new-execution-flow-attempts.yml | 12 ++ .../retry-new-execution-flow-duration.yml | 12 ++ .../retry-new-execution-task-attempts.yml | 12 ++ .../retry-new-execution-task-duration.yml | 12 ++ .../migrations/h2/V1_19__retry_flow.sql | 15 ++ .../migrations/mysql/V1_18__retry_revamp.sql | 2 +- .../migrations/mysql/V1_19__retry_flow.sql | 15 ++ .../migrations/postgres/V1_19__retry_flow.sql | 1 + .../io/kestra/jdbc/runner/JdbcExecutor.java | 35 ++-- .../io/kestra/jdbc/runner/JdbcRunnerTest.java | 49 ++++- .../kestra/runner/memory/MemoryExecutor.java | 2 +- ui/src/components/executions/Overview.vue | 4 +- ui/src/translations/en.json | 5 +- ui/src/translations/fr.json | 5 +- ui/src/utils/state.js | 12 ++ 31 files changed, 603 insertions(+), 95 deletions(-) create mode 100644 core/src/main/java/io/kestra/core/models/executions/ExecutionMetadata.java create mode 100644 core/src/test/java/io/kestra/core/tasks/flows/RetryCaseTest.java create mode 100644 core/src/test/resources/flows/valids/retry-failed-flow-attempts.yml create mode 100644 core/src/test/resources/flows/valids/retry-failed-flow-duration.yml create mode 100644 core/src/test/resources/flows/valids/retry-failed-task-attempts.yml create mode 100644 core/src/test/resources/flows/valids/retry-failed-task-duration.yml create mode 100644 core/src/test/resources/flows/valids/retry-new-execution-flow-attempts.yml create mode 100644 core/src/test/resources/flows/valids/retry-new-execution-flow-duration.yml create mode 100644 core/src/test/resources/flows/valids/retry-new-execution-task-attempts.yml create mode 100644 core/src/test/resources/flows/valids/retry-new-execution-task-duration.yml create mode 100644 jdbc-h2/src/main/resources/migrations/h2/V1_19__retry_flow.sql create mode 100644 jdbc-mysql/src/main/resources/migrations/mysql/V1_19__retry_flow.sql create mode 100644 jdbc-postgres/src/main/resources/migrations/postgres/V1_19__retry_flow.sql diff --git a/core/src/main/java/io/kestra/core/models/executions/Execution.java b/core/src/main/java/io/kestra/core/models/executions/Execution.java index 5194010e857..526b4e27db0 100644 --- a/core/src/main/java/io/kestra/core/models/executions/Execution.java +++ b/core/src/main/java/io/kestra/core/models/executions/Execution.java @@ -93,6 +93,9 @@ public class Execution implements DeletedInterface, TenantInterface { @Builder.Default boolean deleted = false; + @With + ExecutionMetadata metadata; + /** * Factory method for constructing a new {@link Execution} object for the given {@link Flow} and inputs. * @@ -134,6 +137,9 @@ public static Execution newExecution(final Flow flow, public static class ExecutionBuilder { void prebuild() { this.originalId = this.id; + this.metadata = ExecutionMetadata.builder() + .originalCreatedDate(Instant.now()) + .build(); } } @@ -165,7 +171,8 @@ public Execution withState(State.Type state) { this.parentId, this.originalId, this.trigger, - this.deleted + this.deleted, + this.metadata ); } @@ -197,7 +204,8 @@ public Execution withTaskRun(TaskRun taskRun) throws InternalException { this.parentId, this.originalId, this.trigger, - this.deleted + this.deleted, + this.metadata ); } @@ -217,7 +225,8 @@ public Execution childExecution(String childExecutionId, List taskRunLi childExecutionId != null ? this.getId() : null, this.originalId, this.trigger, - this.deleted + this.deleted, + this.metadata ); } @@ -449,7 +458,7 @@ public boolean hasFailedNoRetry(List resolvedTasks, TaskRun parent log.warn("Can't find task for taskRun '{}' in parentTaskRun '{}'", taskRun.getId(), parentTaskRun.getId()); return false; } - return !taskRun.shouldBeRetried(resolvedTask.getTask()) && taskRun.getState().isFailed(); + return !taskRun.shouldBeRetried(resolvedTask.getTask().getRetry()) && taskRun.getState().isFailed(); }); } diff --git a/core/src/main/java/io/kestra/core/models/executions/ExecutionMetadata.java b/core/src/main/java/io/kestra/core/models/executions/ExecutionMetadata.java new file mode 100644 index 00000000000..54b692de0ee --- /dev/null +++ b/core/src/main/java/io/kestra/core/models/executions/ExecutionMetadata.java @@ -0,0 +1,25 @@ +package io.kestra.core.models.executions; + +import lombok.Builder; +import lombok.Getter; +import lombok.Setter; +import lombok.With; + +import java.time.Instant; + +@Builder(toBuilder = true) +@Setter +@Getter +public class ExecutionMetadata { + @Builder.Default + @With + Integer attemptNumber = 1; + + Instant originalCreatedDate; + + public ExecutionMetadata nextAttempt() { + return this.toBuilder() + .attemptNumber(this.attemptNumber + 1) + .build(); + } +} diff --git a/core/src/main/java/io/kestra/core/models/executions/TaskRun.java b/core/src/main/java/io/kestra/core/models/executions/TaskRun.java index b51835a0b09..94ae2181010 100644 --- a/core/src/main/java/io/kestra/core/models/executions/TaskRun.java +++ b/core/src/main/java/io/kestra/core/models/executions/TaskRun.java @@ -3,7 +3,7 @@ import io.kestra.core.models.TenantInterface; import io.kestra.core.models.flows.State; import io.kestra.core.models.tasks.ResolvedTask; -import io.kestra.core.models.tasks.Task; +import io.kestra.core.models.tasks.retrys.AbstractRetry; import io.kestra.core.utils.IdUtils; import io.swagger.v3.oas.annotations.Hidden; import jakarta.validation.constraints.NotNull; @@ -219,19 +219,41 @@ public String toStringState() { } /** + * This method is used when the retry is apply on a task + * but the retry type is NEW_EXECUTION * - * @param task Contains the retry configuration - * @return The next retry date, null if maxAttempt is reached + * @param retry Contains the retry configuration + * @param execution Contains the attempt number and original creation date + * @return The next retry date, null if maxAttempt || maxDuration is reached */ - public Instant nextRetryDate(Task task) { - if (this.attempts == null || this.attempts.isEmpty() || task.getRetry() == null || this.getAttempts().size() >= task.getRetry().getMaxAttempt()) { + public Instant nextRetryDate(AbstractRetry retry, Execution execution) { + if (retry.getMaxAttempt() != null && execution.getMetadata().getAttemptNumber() >= retry.getMaxAttempt()) { return null; } Instant base = this.lastAttempt().getState().maxDate(); + Instant nextDate = retry.nextRetryDate(execution.getMetadata().getAttemptNumber(), base); + if (retry.getMaxDuration() != null && nextDate.isAfter(execution.getMetadata().getOriginalCreatedDate().plus(retry.getMaxDuration()))) { - Instant nextDate = task.getRetry().nextRetryDate(this.attempts.size(), base); - if (task.getRetry().getMaxDuration() != null && nextDate.isAfter(this.lastAttempt().getState().minDate().plus(task.getRetry().getMaxDuration()))) { + return null; + } + + return nextDate; + } + + /** + * This method is used when the Retry definition comes from the flow + * @param retry The retry configuration + * @return The next retry date, null if maxAttempt || maxDuration is reached + */ + public Instant nextRetryDate(AbstractRetry retry) { + if (this.attempts == null || this.attempts.isEmpty() || (retry.getMaxAttempt() != null && this.attemptNumber() >= retry.getMaxAttempt())) { + + return null; + } + Instant base = this.lastAttempt().getState().maxDate(); + Instant nextDate = retry.nextRetryDate(this.attempts.size(), base); + if (retry.getMaxDuration() != null && nextDate.isAfter(this.attempts.get(0).getState().minDate().plus(retry.getMaxDuration()))) { return null; } @@ -239,8 +261,11 @@ public Instant nextRetryDate(Task task) { return nextDate; } - public boolean shouldBeRetried(Task task) { - return this.nextRetryDate(task) != null; + public boolean shouldBeRetried(AbstractRetry retry) { + if (retry == null) { + return false; + } + return this.nextRetryDate(retry) != null; } diff --git a/core/src/main/java/io/kestra/core/models/flows/Flow.java b/core/src/main/java/io/kestra/core/models/flows/Flow.java index 17ee3d48eb7..d6706dc4333 100644 --- a/core/src/main/java/io/kestra/core/models/flows/Flow.java +++ b/core/src/main/java/io/kestra/core/models/flows/Flow.java @@ -16,6 +16,7 @@ import io.kestra.core.models.listeners.Listener; import io.kestra.core.models.tasks.FlowableTask; import io.kestra.core.models.tasks.Task; +import io.kestra.core.models.tasks.retrys.AbstractRetry; import io.kestra.core.models.triggers.AbstractTrigger; import io.kestra.core.models.validations.ManualConstraintViolation; import io.kestra.core.serializers.JacksonMapper; @@ -27,6 +28,10 @@ import io.micronaut.core.annotation.Introspected; import io.swagger.v3.oas.annotations.Hidden; import io.swagger.v3.oas.annotations.media.Schema; +import jakarta.validation.ConstraintViolation; +import jakarta.validation.ConstraintViolationException; +import jakarta.validation.Valid; +import jakarta.validation.constraints.*; import lombok.*; import lombok.experimental.SuperBuilder; import org.slf4j.Logger; @@ -35,10 +40,6 @@ import java.util.*; import java.util.stream.Collectors; import java.util.stream.Stream; -import jakarta.validation.ConstraintViolation; -import jakarta.validation.ConstraintViolationException; -import jakarta.validation.Valid; -import jakarta.validation.constraints.*; @SuperBuilder(toBuilder = true) @Getter @@ -121,6 +122,9 @@ public boolean hasIgnoreMarker(final AnnotatedMember m) { @Valid List outputs; + @Valid + protected AbstractRetry retry; + public Logger logger() { return LoggerFactory.getLogger("flow." + this.id); } diff --git a/core/src/main/java/io/kestra/core/models/flows/State.java b/core/src/main/java/io/kestra/core/models/flows/State.java index e44386f5b0e..39b87da6093 100644 --- a/core/src/main/java/io/kestra/core/models/flows/State.java +++ b/core/src/main/java/io/kestra/core/models/flows/State.java @@ -158,6 +158,11 @@ public boolean isRestartable() { return this.current.isFailed() || this.isPaused(); } + @JsonIgnore + public boolean isResumable() { + return this.current.isPaused() || this.current.isRetrying(); + } + @Introspected public enum Type { @@ -172,10 +177,11 @@ public enum Type { KILLED, CANCELLED, QUEUED, - RETRYING; + RETRYING, + RETRIED; public boolean isTerminated() { - return this == Type.FAILED || this == Type.WARNING || this == Type.SUCCESS || this == Type.KILLED || this == Type.CANCELLED; + return this == Type.FAILED || this == Type.WARNING || this == Type.SUCCESS || this == Type.KILLED || this == Type.CANCELLED || this == Type.RETRIED; } public boolean isCreated() { @@ -195,8 +201,9 @@ public boolean isPaused() { } public boolean isRetrying() { - return this == Type.RETRYING; + return this == Type.RETRYING || this == Type.RETRIED; } + } @Value diff --git a/core/src/main/java/io/kestra/core/models/tasks/retrys/AbstractRetry.java b/core/src/main/java/io/kestra/core/models/tasks/retrys/AbstractRetry.java index 0d39400ef8f..750e3018e45 100644 --- a/core/src/main/java/io/kestra/core/models/tasks/retrys/AbstractRetry.java +++ b/core/src/main/java/io/kestra/core/models/tasks/retrys/AbstractRetry.java @@ -34,6 +34,9 @@ public abstract class AbstractRetry { @Builder.Default private Boolean warningOnRetry = false; + @Builder.Default + private Behavior behavior = Behavior.RETRY_FAILED_TASK; + public abstract Instant nextRetryDate(Integer attemptCount, Instant lastAttempt); public RetryPolicy toPolicy() { @@ -58,4 +61,9 @@ public static RetryPolicy retryPolicy(AbstractRetry retry) { return new RetryPolicy() .withMaxAttempts(1); } + + public enum Behavior { + RETRY_FAILED_TASK, + CREATE_NEW_EXECUTION + } } diff --git a/core/src/main/java/io/kestra/core/runners/ExecutionDelay.java b/core/src/main/java/io/kestra/core/runners/ExecutionDelay.java index d6a61cce502..c567023337b 100644 --- a/core/src/main/java/io/kestra/core/runners/ExecutionDelay.java +++ b/core/src/main/java/io/kestra/core/runners/ExecutionDelay.java @@ -2,14 +2,13 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import io.kestra.core.models.flows.State; +import jakarta.validation.constraints.NotNull; import lombok.AllArgsConstructor; import lombok.Builder; import lombok.Value; import java.time.Instant; -import jakarta.validation.constraints.NotNull; - @Value @AllArgsConstructor @Builder @@ -25,8 +24,24 @@ public class ExecutionDelay { @NotNull State.Type state; + @NotNull DelayType delayType; + @JsonIgnore public String uid() { return String.join("_", executionId, taskRunId); } + + /** + * For previous version, return RESUME_FLOW by default as it was the only case + * @return DelayType representing the action to do when + */ + public DelayType getDelayType() { + return delayType == null ? DelayType.RESUME_FLOW : delayType; + } + + public enum DelayType { + RESUME_FLOW, + RESTART_FAILED_TASK, + RESTART_FAILED_FLOW + } } diff --git a/core/src/main/java/io/kestra/core/runners/ExecutorService.java b/core/src/main/java/io/kestra/core/runners/ExecutorService.java index bf21b4d7256..a90774ca591 100644 --- a/core/src/main/java/io/kestra/core/runners/ExecutorService.java +++ b/core/src/main/java/io/kestra/core/runners/ExecutorService.java @@ -7,7 +7,9 @@ import io.kestra.core.models.flows.Flow; import io.kestra.core.models.flows.State; import io.kestra.core.models.tasks.*; +import io.kestra.core.models.tasks.retrys.AbstractRetry; import io.kestra.core.services.ConditionService; +import io.kestra.core.services.ExecutionService; import io.kestra.core.services.LogService; import io.kestra.core.tasks.flows.Pause; import io.kestra.core.tasks.flows.WorkingDirectory; @@ -47,6 +49,9 @@ public class ExecutorService { protected FlowExecutorInterface flowExecutorInterface; + @Inject + private ExecutionService executionService; + protected FlowExecutorInterface flowExecutorInterface() { // bean is injected late, so we need to wait if (this.flowExecutorInterface == null) { @@ -429,7 +434,7 @@ private Executor handleChildNext(Executor executor) throws InternalException { return executor.withTaskRun(result, "handleChildNext"); } - private Executor handleChildWorkerTaskResult(Executor executor) throws InternalException { + private Executor handleChildWorkerTaskResult(Executor executor) throws Exception { if (executor.getExecution().getTaskRunList() == null) { return executor; } @@ -448,22 +453,55 @@ private Executor handleChildWorkerTaskResult(Executor executor) throws InternalE workerTaskResult.ifPresent(list::add); } - /** + /* * Check if the task is failed and if it has a retry policy */ - if (!executor.getExecution().getState().isRetrying() && taskRun.getState().isFailed() && executor.getFlow().findTaskByTaskId(taskRun.getTaskId()).getRetry() != null) { - Instant nextRetryDate = taskRun.nextRetryDate(executor.getFlow().findTaskByTaskId(taskRun.getTaskId())); + if (!executor.getExecution().getState().isRetrying() && + taskRun.getState().isFailed() && + (executor.getFlow().findTaskByTaskId(taskRun.getTaskId()).getRetry() != null || executor.getFlow().getRetry() != null) + ) { + Instant nextRetryDate; + AbstractRetry retry; + ExecutionDelay.ExecutionDelayBuilder executionDelayBuilder = ExecutionDelay.builder() + .taskRunId(taskRun.getId()) + .executionId(executor.getExecution().getId()); + // Case task has a retry + if (executor.getFlow().findTaskByTaskId(taskRun.getTaskId()).getRetry() != null) { + retry = executor.getFlow().findTaskByTaskId(taskRun.getTaskId()).getRetry(); + AbstractRetry.Behavior behavior = retry.getBehavior(); + nextRetryDate = behavior.equals(AbstractRetry.Behavior.CREATE_NEW_EXECUTION) ? + taskRun.nextRetryDate(retry, executor.getExecution()) : + taskRun.nextRetryDate(retry); + executionDelayBuilder + .date(nextRetryDate) + .state(State.Type.RUNNING) + .delayType(behavior.equals(AbstractRetry.Behavior.CREATE_NEW_EXECUTION) ? + ExecutionDelay.DelayType.RESTART_FAILED_FLOW : + ExecutionDelay.DelayType.RESTART_FAILED_TASK); + } + // Case flow has a retry + else { + retry = executor.getFlow().getRetry(); + AbstractRetry.Behavior behavior = retry.getBehavior(); + nextRetryDate = behavior.equals(AbstractRetry.Behavior.CREATE_NEW_EXECUTION) ? + executionService.nextRetryDate(retry, executor.getExecution()) : + taskRun.nextRetryDate(retry); + + executionDelayBuilder + .date(nextRetryDate) + .state(State.Type.RUNNING) + .delayType(retry.getBehavior().equals(AbstractRetry.Behavior.CREATE_NEW_EXECUTION) ? + ExecutionDelay.DelayType.RESTART_FAILED_FLOW : + ExecutionDelay.DelayType.RESTART_FAILED_TASK); + } if (nextRetryDate != null) { - executionDelays.add( - ExecutionDelay.builder() - .taskRunId(taskRun.getId()) - .executionId(executor.getExecution().getId()) - .date(nextRetryDate) - .state(State.Type.RUNNING) - .build() - ); - executor.withExecution(executor.getExecution().withState(State.Type.RETRYING), "handleRetryTask"); - + executionDelays.add(executionDelayBuilder.build()); + executor.withExecution(executor.getExecution() + .withState(retry.getBehavior().equals(AbstractRetry.Behavior.CREATE_NEW_EXECUTION) ? + State.Type.RETRIED : + State.Type.RETRYING + ), + "handleRetryTask"); } } } @@ -499,6 +537,7 @@ private Executor handlePausedDelay(Executor executor, List wor .executionId(executor.getExecution().getId()) .date(workerTaskResult.getTaskRun().getState().maxDate().plus(pauseTask.getDelay() != null ? pauseTask.getDelay() : pauseTask.getTimeout())) .state(pauseTask.getDelay() != null ? State.Type.RUNNING : State.Type.FAILED) + .delayType(ExecutionDelay.DelayType.RESUME_FLOW) .build(); } } @@ -537,7 +576,6 @@ private Executor handleCreatedKilling(Executor executor) { return executor.withWorkerTaskResults(workerTaskResults, "handleChildWorkerCreatedKilling"); } - private Executor handleListeners(Executor executor) { if (!executor.getExecution().getState().isTerminated()) { return executor; @@ -578,7 +616,6 @@ private Executor handleEnd(Executor executor) { return this.onEnd(executor); } - private Executor handleRestart(Executor executor) { if (executor.getExecution().getState().getCurrent() != State.Type.RESTARTED) { return executor; @@ -608,7 +645,6 @@ private Executor handleKilling(Executor executor) { return executor.withExecution(newExecution, "handleKilling"); } - private Executor handleWorkerTask(Executor executor) throws InternalException { if (executor.getExecution().getTaskRunList() == null || executor.getExecution().getState().getCurrent() == State.Type.KILLING) { return executor; diff --git a/core/src/main/java/io/kestra/core/runners/RunContext.java b/core/src/main/java/io/kestra/core/runners/RunContext.java index 4ad54e45a2d..6bb403ab4b4 100644 --- a/core/src/main/java/io/kestra/core/runners/RunContext.java +++ b/core/src/main/java/io/kestra/core/runners/RunContext.java @@ -306,6 +306,7 @@ protected Map variables(Flow flow, Task task, Execution executio if (execution.getOriginalId() != null) { executionMap.put("originalId", execution.getOriginalId()); } + builder .put("execution", executionMap.build()); diff --git a/core/src/main/java/io/kestra/core/runners/Worker.java b/core/src/main/java/io/kestra/core/runners/Worker.java index 19cb26ad95d..86b6f6a2c8e 100644 --- a/core/src/main/java/io/kestra/core/runners/Worker.java +++ b/core/src/main/java/io/kestra/core/runners/Worker.java @@ -10,11 +10,7 @@ import io.kestra.core.exceptions.TimeoutExceededException; import io.kestra.core.metrics.MetricRegistry; import io.kestra.core.models.Label; -import io.kestra.core.models.executions.Execution; -import io.kestra.core.models.executions.ExecutionKilled; -import io.kestra.core.models.executions.MetricEntry; -import io.kestra.core.models.executions.TaskRun; -import io.kestra.core.models.executions.TaskRunAttempt; +import io.kestra.core.models.executions.*; import io.kestra.core.models.tasks.Output; import io.kestra.core.models.tasks.RunnableTask; import io.kestra.core.models.tasks.Task; @@ -54,14 +50,7 @@ import java.time.Duration; import java.time.ZonedDateTime; import java.time.temporal.ChronoUnit; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.UUID; +import java.util.*; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; @@ -69,12 +58,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; -import static io.kestra.core.models.flows.State.Type.CREATED; -import static io.kestra.core.models.flows.State.Type.FAILED; -import static io.kestra.core.models.flows.State.Type.KILLED; -import static io.kestra.core.models.flows.State.Type.RUNNING; -import static io.kestra.core.models.flows.State.Type.SUCCESS; -import static io.kestra.core.models.flows.State.Type.WARNING; +import static io.kestra.core.models.flows.State.Type.*; import static io.kestra.core.server.Service.ServiceState.TERMINATED_FORCED; import static io.kestra.core.server.Service.ServiceState.TERMINATED_GRACEFULLY; @@ -480,7 +464,7 @@ private WorkerTaskResult run(WorkerTask workerTask, Boolean cleanUp) throws Queu state = WARNING; } - if (workerTask.getTask().isAllowFailure() && !finalWorkerTask.getTaskRun().shouldBeRetried(workerTask.getTask()) && state.isFailed()) { + if (workerTask.getTask().isAllowFailure() && !finalWorkerTask.getTaskRun().shouldBeRetried(workerTask.getTask().getRetry()) && state.isFailed()) { state = WARNING; } diff --git a/core/src/main/java/io/kestra/core/services/ExecutionService.java b/core/src/main/java/io/kestra/core/services/ExecutionService.java index 546014ce308..3e561cd0175 100644 --- a/core/src/main/java/io/kestra/core/services/ExecutionService.java +++ b/core/src/main/java/io/kestra/core/services/ExecutionService.java @@ -12,6 +12,7 @@ import io.kestra.core.models.hierarchies.AbstractGraphTask; import io.kestra.core.models.hierarchies.GraphCluster; import io.kestra.core.models.tasks.Task; +import io.kestra.core.models.tasks.retrys.AbstractRetry; import io.kestra.core.queues.QueueFactoryInterface; import io.kestra.core.queues.QueueInterface; import io.kestra.core.repositories.ExecutionRepositoryInterface; @@ -37,6 +38,7 @@ import java.io.IOException; import java.net.URI; +import java.time.Instant; import java.time.ZonedDateTime; import java.util.*; import java.util.function.Predicate; @@ -76,7 +78,7 @@ public class ExecutionService { * Retry set the given taskRun in created state * and return the execution in running state **/ - public Execution retry(Execution execution, String taskRunId) { + public Execution retryTask(Execution execution, String taskRunId) { List newTaskRuns = execution .getTaskRunList() .stream() @@ -140,6 +142,8 @@ public Execution restart(final Execution execution, @Nullable Integer revision) execution.withState(State.Type.RESTARTED).getState() ); + newExecution = newExecution.withMetadata(execution.getMetadata().nextAttempt()); + return revision != null ? newExecution.withFlowRevision(revision) : newExecution; } @@ -217,6 +221,8 @@ public Execution replay(final Execution execution, @Nullable String taskRunId, @ taskRunId == null ? new State() : execution.withState(State.Type.RESTARTED).getState() ); + newExecution = newExecution.withMetadata(execution.getMetadata().nextAttempt()); + return revision != null ? newExecution.withFlowRevision(revision) : newExecution; } @@ -542,4 +548,28 @@ private Set taskRunWithAncestors(Execution execution, List task .flatMap(throwFunction(taskRun -> this.getAncestors(execution, taskRun).stream())) .collect(Collectors.toSet()); } + + /** + * This method is used to retrieve previous existing execution + * @param retry The retry define in the flow of the failed execution + * @param execution The failed execution + * @return The next retry date, null if maxAttempt || maxDuration is reached + */ + public Instant nextRetryDate(AbstractRetry retry, Execution execution) { + if (retry.getMaxAttempt() != null && execution.getMetadata().getAttemptNumber() >= retry.getMaxAttempt()) { + + return null; + } + + Instant base = execution.getState().maxDate(); + Instant originalCreatedDate = execution.getMetadata().getOriginalCreatedDate(); + Instant nextDate = retry.nextRetryDate(execution.getMetadata().getAttemptNumber(), base); + + if (retry.getMaxDuration() != null && nextDate.isAfter(originalCreatedDate.plus(retry.getMaxDuration()))) { + + return null; + } + + return nextDate; + } } diff --git a/core/src/test/java/io/kestra/core/tasks/flows/RetryCaseTest.java b/core/src/test/java/io/kestra/core/tasks/flows/RetryCaseTest.java new file mode 100644 index 00000000000..191e8bf5eaa --- /dev/null +++ b/core/src/test/java/io/kestra/core/tasks/flows/RetryCaseTest.java @@ -0,0 +1,193 @@ +package io.kestra.core.tasks.flows; + +import io.kestra.core.models.executions.Execution; +import io.kestra.core.models.flows.State; +import io.kestra.core.queues.QueueFactoryInterface; +import io.kestra.core.queues.QueueInterface; +import io.kestra.core.runners.RunnerUtils; +import io.kestra.core.utils.Await; +import jakarta.inject.Inject; +import jakarta.inject.Named; +import jakarta.inject.Singleton; +import lombok.extern.slf4j.Slf4j; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicReference; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.is; + +@Slf4j +@Singleton +public class RetryCaseTest { + + @Inject + @Named(QueueFactoryInterface.EXECUTION_NAMED) + private QueueInterface executionQueue; + + @Inject + protected RunnerUtils runnerUtils; + + public void retryNewExecutionTaskDuration() throws TimeoutException { + CountDownLatch countDownLatch = new CountDownLatch(3); + AtomicReference> stateHistory = new AtomicReference<>(new ArrayList<>()); + + executionQueue.receive(either -> { + Execution execution = either.getLeft(); + if (execution.getFlowId().equals("retry-new-execution-task-duration") && execution.getState().getCurrent().isTerminated()) { + countDownLatch.countDown(); + List stateHistoryList = stateHistory.get(); + stateHistoryList.add(execution.getState().getCurrent()); + stateHistory.set(stateHistoryList); + } + }); + + Execution execution = runnerUtils.runOne( + null, + "io.kestra.tests", + "retry-new-execution-task-duration", + null, + null + ); + + Await.until(() -> countDownLatch.getCount() == 0, Duration.ofSeconds(2), Duration.ofMinutes(1)); + assertThat(stateHistory.get(), containsInAnyOrder(State.Type.RETRIED, State.Type.RETRIED, State.Type.FAILED)); + } + + public void retryNewExecutionTaskAttempts() throws TimeoutException { + CountDownLatch countDownLatch = new CountDownLatch(3); + AtomicReference> stateHistory = new AtomicReference<>(new ArrayList<>()); + + executionQueue.receive(either -> { + Execution execution = either.getLeft(); + if (execution.getFlowId().equals("retry-new-execution-task-attempts") && execution.getState().getCurrent().isTerminated()) { + countDownLatch.countDown(); + List stateHistoryList = stateHistory.get(); + stateHistoryList.add(execution.getState().getCurrent()); + stateHistory.set(stateHistoryList); + } + }); + + Execution execution = runnerUtils.runOne( + null, + "io.kestra.tests", + "retry-new-execution-task-attempts", + null, + null + ); + + Await.until(() -> countDownLatch.getCount() == 0, Duration.ofSeconds(2), Duration.ofMinutes(1)); + assertThat(stateHistory.get(), containsInAnyOrder(State.Type.RETRIED, State.Type.RETRIED, State.Type.FAILED)); + } + + public void retryNewExecutionFlowDuration() throws TimeoutException { + CountDownLatch countDownLatch = new CountDownLatch(3); + AtomicReference> stateHistory = new AtomicReference<>(new ArrayList<>()); + + executionQueue.receive(either -> { + Execution execution = either.getLeft(); + if (execution.getFlowId().equals("retry-new-execution-flow-duration") && execution.getState().getCurrent().isTerminated()) { + countDownLatch.countDown(); + List stateHistoryList = stateHistory.get(); + stateHistoryList.add(execution.getState().getCurrent()); + stateHistory.set(stateHistoryList); + } + }); + + Execution execution = runnerUtils.runOne( + null, + "io.kestra.tests", + "retry-new-execution-flow-duration", + null, + null + ); + + Await.until(() -> countDownLatch.getCount() == 0, Duration.ofSeconds(2), Duration.ofMinutes(1)); + assertThat(stateHistory.get(), containsInAnyOrder(State.Type.RETRIED, State.Type.RETRIED, State.Type.FAILED)); + } + + public void retryNewExecutionFlowAttempts() throws TimeoutException { + CountDownLatch countDownLatch = new CountDownLatch(3); + AtomicReference> stateHistory = new AtomicReference<>(new ArrayList<>()); + + executionQueue.receive(either -> { + Execution execution = either.getLeft(); + if (execution.getFlowId().equals("retry-new-execution-flow-attempts") && execution.getState().getCurrent().isTerminated()) { + countDownLatch.countDown(); + List stateHistoryList = stateHistory.get(); + stateHistoryList.add(execution.getState().getCurrent()); + stateHistory.set(stateHistoryList); + } + }); + + Execution execution = runnerUtils.runOne( + null, + "io.kestra.tests", + "retry-new-execution-flow-attempts", + null, + null + ); + + Await.until(() -> countDownLatch.getCount() == 0, Duration.ofSeconds(2), Duration.ofMinutes(1)); + assertThat(stateHistory.get(), containsInAnyOrder(State.Type.RETRIED, State.Type.RETRIED, State.Type.FAILED)); + } + + public void retryFailedTaskDuration() throws TimeoutException { + Execution execution = runnerUtils.runOne( + null, + "io.kestra.tests", + "retry-failed-task-duration", + null, + null + ); + + assertThat(execution.getState().getCurrent(), is(State.Type.FAILED)); + assertThat(execution.getTaskRunList().get(0).attemptNumber(), is(3)); + } + + public void retryFailedTaskAttempts() throws TimeoutException { + Execution execution = runnerUtils.runOne( + null, + "io.kestra.tests", + "retry-failed-task-attempts", + null, + null, + Duration.ofSeconds(20) + ); + + assertThat(execution.getState().getCurrent(), is(State.Type.FAILED)); + assertThat(execution.getTaskRunList().get(0).attemptNumber(), is(4)); + } + + public void retryFailedFlowDuration() throws TimeoutException { + Execution execution = runnerUtils.runOne( + null, + "io.kestra.tests", + "retry-failed-flow-duration", + null, + null + ); + + assertThat(execution.getState().getCurrent(), is(State.Type.FAILED)); + assertThat(execution.getTaskRunList().get(0).attemptNumber(), is(3)); + } + + public void retryFailedFlowAttempts() throws TimeoutException { + Execution execution = runnerUtils.runOne( + null, + "io.kestra.tests", + "retry-failed-flow-attempts", + null, + null + ); + + assertThat(execution.getState().getCurrent(), is(State.Type.FAILED)); + assertThat(execution.getTaskRunList().get(0).attemptNumber(), is(4)); + } + +} diff --git a/core/src/test/resources/flows/valids/retry-failed-flow-attempts.yml b/core/src/test/resources/flows/valids/retry-failed-flow-attempts.yml new file mode 100644 index 00000000000..db30490fb68 --- /dev/null +++ b/core/src/test/resources/flows/valids/retry-failed-flow-attempts.yml @@ -0,0 +1,12 @@ +id: retry-failed-flow-attempts +namespace: io.kestra.tests +retry: + behavior: RETRY_FAILED_TASK + type: constant + maxAttempt: 4 + interval: PT2S + +tasks: + - id: fail + type: io.kestra.core.tasks.executions.Fail + diff --git a/core/src/test/resources/flows/valids/retry-failed-flow-duration.yml b/core/src/test/resources/flows/valids/retry-failed-flow-duration.yml new file mode 100644 index 00000000000..20054f39e7e --- /dev/null +++ b/core/src/test/resources/flows/valids/retry-failed-flow-duration.yml @@ -0,0 +1,12 @@ +id: retry-failed-flow-duration +namespace: io.kestra.tests +retry: + behavior: RETRY_FAILED_TASK + type: constant + maxDuration: PT15S + interval: PT5S + +tasks: + - id: fail + type: io.kestra.core.tasks.executions.Fail + diff --git a/core/src/test/resources/flows/valids/retry-failed-task-attempts.yml b/core/src/test/resources/flows/valids/retry-failed-task-attempts.yml new file mode 100644 index 00000000000..303d0d6253d --- /dev/null +++ b/core/src/test/resources/flows/valids/retry-failed-task-attempts.yml @@ -0,0 +1,12 @@ +id: retry-failed-task-attempts +namespace: io.kestra.tests + +tasks: + - id: fail + type: io.kestra.core.tasks.executions.Fail + retry: + behavior: RETRY_FAILED_TASK + type: constant + maxAttempt: 4 + interval: PT2S + diff --git a/core/src/test/resources/flows/valids/retry-failed-task-duration.yml b/core/src/test/resources/flows/valids/retry-failed-task-duration.yml new file mode 100644 index 00000000000..9bde34e7f6c --- /dev/null +++ b/core/src/test/resources/flows/valids/retry-failed-task-duration.yml @@ -0,0 +1,12 @@ +id: retry-failed-task-duration +namespace: io.kestra.tests + +tasks: + - id: fail + type: io.kestra.core.tasks.executions.Fail + retry: + behavior: RETRY_FAILED_TASK + type: constant + maxDuration: PT15S + interval: PT5S + diff --git a/core/src/test/resources/flows/valids/retry-new-execution-flow-attempts.yml b/core/src/test/resources/flows/valids/retry-new-execution-flow-attempts.yml new file mode 100644 index 00000000000..f88d53171f4 --- /dev/null +++ b/core/src/test/resources/flows/valids/retry-new-execution-flow-attempts.yml @@ -0,0 +1,12 @@ +id: retry-new-execution-flow-attempts +namespace: io.kestra.tests +retry: + behavior: CREATE_NEW_EXECUTION + type: constant + maxAttempt: 3 + interval: PT2S + +tasks: + - id: fail + type: io.kestra.core.tasks.executions.Fail + diff --git a/core/src/test/resources/flows/valids/retry-new-execution-flow-duration.yml b/core/src/test/resources/flows/valids/retry-new-execution-flow-duration.yml new file mode 100644 index 00000000000..1e88f8b0158 --- /dev/null +++ b/core/src/test/resources/flows/valids/retry-new-execution-flow-duration.yml @@ -0,0 +1,12 @@ +id: retry-new-execution-flow-duration +namespace: io.kestra.tests +retry: + behavior: CREATE_NEW_EXECUTION + type: constant + interval: PT5S + maxDuration: PT15S + +tasks: + - id: fail + type: io.kestra.core.tasks.executions.Fail + diff --git a/core/src/test/resources/flows/valids/retry-new-execution-task-attempts.yml b/core/src/test/resources/flows/valids/retry-new-execution-task-attempts.yml new file mode 100644 index 00000000000..8b9520cc997 --- /dev/null +++ b/core/src/test/resources/flows/valids/retry-new-execution-task-attempts.yml @@ -0,0 +1,12 @@ +id: retry-new-execution-task-attempts +namespace: io.kestra.tests + +tasks: + - id: fail + type: io.kestra.core.tasks.executions.Fail + retry: + behavior: CREATE_NEW_EXECUTION + type: constant + maxAttempt: 3 + interval: PT2S + diff --git a/core/src/test/resources/flows/valids/retry-new-execution-task-duration.yml b/core/src/test/resources/flows/valids/retry-new-execution-task-duration.yml new file mode 100644 index 00000000000..6717579f16e --- /dev/null +++ b/core/src/test/resources/flows/valids/retry-new-execution-task-duration.yml @@ -0,0 +1,12 @@ +id: retry-new-execution-task-duration +namespace: io.kestra.tests + +tasks: + - id: fail + type: io.kestra.core.tasks.executions.Fail + retry: + behavior: CREATE_NEW_EXECUTION + type: constant + maxDuration: PT15S + interval: PT5S + diff --git a/jdbc-h2/src/main/resources/migrations/h2/V1_19__retry_flow.sql b/jdbc-h2/src/main/resources/migrations/h2/V1_19__retry_flow.sql new file mode 100644 index 00000000000..7a9a4d1b33a --- /dev/null +++ b/jdbc-h2/src/main/resources/migrations/h2/V1_19__retry_flow.sql @@ -0,0 +1,15 @@ +ALTER TABLE executions ALTER COLUMN "state_current" ENUM ( + 'CREATED', + 'RUNNING', + 'PAUSED', + 'RESTARTED', + 'KILLING', + 'SUCCESS', + 'WARNING', + 'FAILED', + 'KILLED', + 'CANCELLED', + 'QUEUED', + 'RETRYING', + 'RETRIED' +) NOT NULL GENERATED ALWAYS AS (JQ_STRING("value", '.state.current')); \ No newline at end of file diff --git a/jdbc-mysql/src/main/resources/migrations/mysql/V1_18__retry_revamp.sql b/jdbc-mysql/src/main/resources/migrations/mysql/V1_18__retry_revamp.sql index 7b2fa7ed65d..04292a374fe 100644 --- a/jdbc-mysql/src/main/resources/migrations/mysql/V1_18__retry_revamp.sql +++ b/jdbc-mysql/src/main/resources/migrations/mysql/V1_18__retry_revamp.sql @@ -11,4 +11,4 @@ ALTER TABLE executions MODIFY COLUMN `state_current` ENUM ( 'CANCELLED', 'QUEUED', 'RETRYING' -) GENERATED ALWAYS AS (value ->> '$.state.current') STORED NOT NULL; \ No newline at end of file + ) GENERATED ALWAYS AS (value ->> '$.state.current') STORED NOT NULL; \ No newline at end of file diff --git a/jdbc-mysql/src/main/resources/migrations/mysql/V1_19__retry_flow.sql b/jdbc-mysql/src/main/resources/migrations/mysql/V1_19__retry_flow.sql new file mode 100644 index 00000000000..ee91ba79bc0 --- /dev/null +++ b/jdbc-mysql/src/main/resources/migrations/mysql/V1_19__retry_flow.sql @@ -0,0 +1,15 @@ +ALTER TABLE executions MODIFY COLUMN `state_current` ENUM ( + 'CREATED', + 'RUNNING', + 'PAUSED', + 'RESTARTED', + 'KILLING', + 'SUCCESS', + 'WARNING', + 'FAILED', + 'KILLED', + 'CANCELLED', + 'QUEUED', + 'RETRYING', + 'RETRIED' +) GENERATED ALWAYS AS (value ->> '$.state.current') STORED NOT NULL; \ No newline at end of file diff --git a/jdbc-postgres/src/main/resources/migrations/postgres/V1_19__retry_flow.sql b/jdbc-postgres/src/main/resources/migrations/postgres/V1_19__retry_flow.sql new file mode 100644 index 00000000000..25a70d5d23b --- /dev/null +++ b/jdbc-postgres/src/main/resources/migrations/postgres/V1_19__retry_flow.sql @@ -0,0 +1 @@ +ALTER TYPE state_type ADD VALUE IF NOT EXISTS 'RETRIED'; diff --git a/jdbc/src/main/java/io/kestra/jdbc/runner/JdbcExecutor.java b/jdbc/src/main/java/io/kestra/jdbc/runner/JdbcExecutor.java index 769db4a980f..799ed9828bb 100644 --- a/jdbc/src/main/java/io/kestra/jdbc/runner/JdbcExecutor.java +++ b/jdbc/src/main/java/io/kestra/jdbc/runner/JdbcExecutor.java @@ -22,23 +22,7 @@ import io.kestra.core.queues.QueueFactoryInterface; import io.kestra.core.queues.QueueInterface; import io.kestra.core.repositories.FlowRepositoryInterface; -import io.kestra.core.runners.ExecutableUtils; -import io.kestra.core.runners.ExecutionQueued; -import io.kestra.core.runners.ExecutionRunning; -import io.kestra.core.runners.Executor; -import io.kestra.core.runners.ExecutorInterface; -import io.kestra.core.runners.ExecutorService; -import io.kestra.core.runners.ExecutorState; -import io.kestra.core.runners.RunContext; -import io.kestra.core.runners.RunContextFactory; -import io.kestra.core.runners.SubflowExecution; -import io.kestra.core.runners.SubflowExecutionResult; -import io.kestra.core.runners.WorkerJob; -import io.kestra.core.runners.WorkerTask; -import io.kestra.core.runners.WorkerTaskResult; -import io.kestra.core.runners.WorkerTaskRunning; -import io.kestra.core.runners.WorkerTrigger; -import io.kestra.core.runners.WorkerTriggerRunning; +import io.kestra.core.runners.*; import io.kestra.core.server.Service; import io.kestra.core.server.ServiceStateChangeEvent; import io.kestra.core.services.AbstractFlowTriggerService; @@ -855,7 +839,7 @@ private void executionDelaySend() { try { // Handle paused tasks - if (executor.getExecution().findTaskRunByTaskRunId(executionDelay.getTaskRunId()).getState().getCurrent() == State.Type.PAUSED) { + if (executionDelay.getDelayType().equals(ExecutionDelay.DelayType.RESUME_FLOW)) { Execution markAsExecution = executionService.markAs( pair.getKey(), @@ -864,14 +848,19 @@ private void executionDelaySend() { ); executor = executor.withExecution(markAsExecution, "pausedRestart"); - // Handle failed tasks - } else if (executor.getExecution().findTaskRunByTaskRunId(executionDelay.getTaskRunId()).getState().getCurrent().equals(State.Type.FAILED)) { - Execution newAttempt = executionService.retry( + } + // Handle failed tasks + else if (executionDelay.getDelayType().equals(ExecutionDelay.DelayType.RESTART_FAILED_TASK)) { + Execution newAttempt = executionService.retryTask( pair.getKey(), executionDelay.getTaskRunId() ); - - executor = executor.withExecution(newAttempt, "failedRetry"); + executor = executor.withExecution(newAttempt, "retryFailedTask"); + } + // Handle failed flow + else if (executionDelay.getDelayType().equals(ExecutionDelay.DelayType.RESTART_FAILED_FLOW)) { + Execution newExecution = executionService.replay(executor.getExecution(), null, null); + executor = executor.withExecution(newExecution, "retryFailedFlow"); } } catch (Exception e) { executor = handleFailedExecutionFromExecutor(executor, e); diff --git a/jdbc/src/test/java/io/kestra/jdbc/runner/JdbcRunnerTest.java b/jdbc/src/test/java/io/kestra/jdbc/runner/JdbcRunnerTest.java index f6359d3305c..4c86d3c2aac 100644 --- a/jdbc/src/test/java/io/kestra/jdbc/runner/JdbcRunnerTest.java +++ b/jdbc/src/test/java/io/kestra/jdbc/runner/JdbcRunnerTest.java @@ -9,11 +9,7 @@ import io.kestra.core.queues.QueueInterface; import io.kestra.core.repositories.LocalFlowRepositoryLoader; import io.kestra.core.runners.*; -import io.kestra.core.tasks.flows.EachSequentialTest; -import io.kestra.core.tasks.flows.FlowCaseTest; -import io.kestra.core.tasks.flows.ForEachItemCaseTest; -import io.kestra.core.tasks.flows.PauseTest; -import io.kestra.core.tasks.flows.WorkingDirectoryTest; +import io.kestra.core.tasks.flows.*; import io.kestra.core.utils.TestsUtils; import io.kestra.jdbc.JdbcTestUtils; import io.micronaut.test.extensions.junit5.annotation.MicronautTest; @@ -83,6 +79,9 @@ public abstract class JdbcRunnerTest { @Inject private FlowConcurrencyCaseTest flowConcurrencyCaseTest; + @Inject + private RetryCaseTest retryCaseTest; + @BeforeAll void init() throws IOException, URISyntaxException { jdbcTestUtils.drop(); @@ -321,4 +320,44 @@ void badExecutable() throws TimeoutException { assertThat(execution.getTaskRunList().get(0).getState().getCurrent(), is(State.Type.FAILED)); assertThat(execution.getState().getCurrent(), is(State.Type.FAILED)); } + + @Test + void retryNewExecutionTaskDuration() throws TimeoutException { + retryCaseTest.retryNewExecutionTaskDuration(); + } + + @Test + void retryNewExecutionTaskAttempts() throws TimeoutException { + retryCaseTest.retryNewExecutionTaskAttempts(); + } + + @Test + void retryNewExecutionFlowDuration() throws TimeoutException { + retryCaseTest.retryNewExecutionFlowDuration(); + } + + @Test + void retryNewExecutionFlowAttempts() throws TimeoutException { + retryCaseTest.retryNewExecutionFlowAttempts(); + } + + @Test + void retryFailedTaskDuration() throws TimeoutException { + retryCaseTest.retryFailedTaskDuration(); + } + + @Test + void retryFailedTaskAttempts() throws TimeoutException { + retryCaseTest.retryFailedTaskAttempts(); + } + + @Test + void retryFailedFlowDuration() throws TimeoutException { + retryCaseTest.retryFailedFlowDuration(); + } + + @Test + void retryFailedFlowAttempts() throws TimeoutException { + retryCaseTest.retryFailedFlowAttempts(); + } } diff --git a/runner-memory/src/main/java/io/kestra/runner/memory/MemoryExecutor.java b/runner-memory/src/main/java/io/kestra/runner/memory/MemoryExecutor.java index 24db1737fd3..392d956c877 100644 --- a/runner-memory/src/main/java/io/kestra/runner/memory/MemoryExecutor.java +++ b/runner-memory/src/main/java/io/kestra/runner/memory/MemoryExecutor.java @@ -232,7 +232,7 @@ private void handleExecution(ExecutionState state) { EXECUTIONS.put(workerTaskResultDelay.getExecutionId(), executionState.from(markAsExecution)); executionQueue.emit(markAsExecution); } else if (executionState.execution.findTaskRunByTaskRunId(workerTaskResultDelay.getTaskRunId()).getState().getCurrent().equals(State.Type.FAILED)) { - Execution newAttempt = executionService.retry( + Execution newAttempt = executionService.retryTask( executionState.execution, workerTaskResultDelay.getTaskRunId() ); diff --git a/ui/src/components/executions/Overview.vue b/ui/src/components/executions/Overview.vue index bec0681b072..857ffc3a971 100644 --- a/ui/src/components/executions/Overview.vue +++ b/ui/src/components/executions/Overview.vue @@ -140,7 +140,9 @@ {key: this.$t("created date"), value: this.execution.state.histories[0].date, date: true}, {key: this.$t("updated date"), value: this.stop(), date: true}, {key: this.$t("duration"), value: this.execution.state.histories, duration: true}, - {key: this.$t("steps"), value: stepCount} + {key: this.$t("steps"), value: stepCount}, + {key: this.$t("attempt"), value: this.execution.metadata.attemptNumber}, + {key: this.$t("originalCreatedDate"), value: this.execution.metadata.originalCreatedDate, date: true}, ]; if (this.execution.parentId) { diff --git a/ui/src/translations/en.json b/ui/src/translations/en.json index ae191bfee6c..56c81f87241 100644 --- a/ui/src/translations/en.json +++ b/ui/src/translations/en.json @@ -606,7 +606,7 @@ }, "export": "Export as a ZIP file" }, - "continue backfill":"Continue the backfill", + "continue backfill": "Continue the backfill", "delete backfill": "Delete the backfill", "pause backfill": "Pause the backfill", "backfill executions": "Backfill executions", @@ -634,6 +634,7 @@ "false": "False", "undefined": "Undefined", "execution-include-non-terminated": "Include non-terminated executions?", - "execution-warn-deleting-still-running": "Executions still running will not be stopped and must be killed before being deleting." + "execution-warn-deleting-still-running": "Executions still running will not be stopped and must be killed before being deleting.", + "originalCreatedDate": "Original creation date" } } \ No newline at end of file diff --git a/ui/src/translations/fr.json b/ui/src/translations/fr.json index 8c8574bdf14..e6208b0c5a5 100644 --- a/ui/src/translations/fr.json +++ b/ui/src/translations/fr.json @@ -101,7 +101,7 @@ "add flow": "Ajouter un flow", "from": "De", "to": "à", - "steps": "étapes", + "steps": "Étapes", "state": "État", "search term in message": "Chercher dans le message", "search": "Chercher", @@ -610,6 +610,7 @@ "false": "Faux", "undefined": "Non défini", "execution-include-non-terminated": "Inclure les exécutions non terminées ?", - "execution-warn-deleting-still-running": "Les exécutions en cours ne seront pas arrêtées et doivent être stoppées avant d'être supprimées." + "execution-warn-deleting-still-running": "Les exécutions en cours ne seront pas arrêtées et doivent être stoppées avant d'être supprimées.", + "originalCreatedDate": "Date de création originale" } } \ No newline at end of file diff --git a/ui/src/utils/state.js b/ui/src/utils/state.js index c2a847d22c5..6936a6ed6b6 100644 --- a/ui/src/utils/state.js +++ b/ui/src/utils/state.js @@ -109,6 +109,14 @@ const STATE = Object.freeze({ isRunning: false, isKillable: true, isFailed: false + }, + RETRIED: { + name: "RETRIED", + colorClass: "gray", + icon: Refresh, + isRunning: false, + isKillable: false, + isFailed: false } }); @@ -161,6 +169,10 @@ export default class State { return STATE.RETRYING.name; } + static get RETRIED() { + return STATE.RETRIED.name; + } + static isRunning(state) { return STATE[state] && STATE[state].isRunning; }