mirror of
https://github.com/kestra-io/kestra.git
synced 2025-12-19 18:05:41 -05:00
committed by
Florian Hussonnois
parent
66a030a487
commit
015659ae1b
@@ -160,9 +160,9 @@ kestra:
|
||||
flowtopologies:
|
||||
table: "flow_topologies"
|
||||
cls: io.kestra.core.models.topologies.FlowTopology
|
||||
workerinstance:
|
||||
cls: io.kestra.core.runners.WorkerInstance
|
||||
table: "worker_instance"
|
||||
serviceinstance:
|
||||
cls: io.kestra.core.server.ServiceInstance
|
||||
table: "service_instance"
|
||||
workerjobrunning:
|
||||
cls: io.kestra.core.runners.WorkerJobRunning
|
||||
table: "worker_job_running"
|
||||
@@ -214,13 +214,22 @@ kestra:
|
||||
preview:
|
||||
initial-rows: 100
|
||||
max-rows: 5000
|
||||
|
||||
# The expected time a server to complete all of its
|
||||
# tasks before initiating a graceful shutdown.
|
||||
terminationGracePeriod: 5m
|
||||
# Configuration for Liveness and Heartbeat mechanism between servers.
|
||||
liveness:
|
||||
enabled: true
|
||||
# The expected time between liveness probe.
|
||||
interval: 5s
|
||||
# The timeout used to detect service failures.
|
||||
timeout: 45s
|
||||
# The time to wait before executing a liveness probe.
|
||||
initialDelay: 30s
|
||||
# The expected time between service heartbeats.
|
||||
heartbeatInterval: 3s
|
||||
anonymous-usage-report:
|
||||
enabled: true
|
||||
uri: https://api.kestra.io/v1/reports/usages
|
||||
initial-delay: 5m
|
||||
fixed-delay: 1h
|
||||
|
||||
heartbeat:
|
||||
frequency: 10s
|
||||
heartbeat-missed: 3
|
||||
@@ -18,7 +18,9 @@ kestra:
|
||||
url: https://repo.maven.apache.org/maven2/
|
||||
sonatype:
|
||||
url: https://s01.oss.sonatype.org/content/repositories/snapshots/
|
||||
|
||||
server:
|
||||
liveness:
|
||||
enabled: false
|
||||
micronaut:
|
||||
http:
|
||||
services:
|
||||
|
||||
101
core/src/main/java/io/kestra/core/contexts/KestraContext.java
Normal file
101
core/src/main/java/io/kestra/core/contexts/KestraContext.java
Normal file
@@ -0,0 +1,101 @@
|
||||
package io.kestra.core.contexts;
|
||||
|
||||
import io.kestra.core.models.ServerType;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import io.micronaut.context.annotation.Context;
|
||||
import io.micronaut.context.annotation.Requires;
|
||||
import io.micronaut.context.env.Environment;
|
||||
import jakarta.annotation.PreDestroy;
|
||||
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
/**
|
||||
* Utility class for retrieving common information about a Kestra Server at runtime.
|
||||
*/
|
||||
public abstract class KestraContext {
|
||||
|
||||
private static final AtomicReference<KestraContext> INSTANCE = new AtomicReference<>();
|
||||
|
||||
// Properties
|
||||
private static final String KESTRA_SERVER_TYPE = "kestra.server-type";
|
||||
|
||||
/**
|
||||
* Gets the current {@link KestraContext}.
|
||||
*
|
||||
* @return The context.
|
||||
* @throws IllegalStateException if not context is initialized.
|
||||
*/
|
||||
public static KestraContext getContext() {
|
||||
return Optional.ofNullable(INSTANCE.get())
|
||||
.orElseThrow(() -> new IllegalStateException("Kestra context not initialized"));
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the current {@link KestraContext}.
|
||||
*
|
||||
* @param context The context.
|
||||
*/
|
||||
public static void setContext(final KestraContext context) {
|
||||
KestraContext.INSTANCE.set(context);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the current {@link ServerType}.
|
||||
*
|
||||
* @return The {@link ServerType}.
|
||||
*/
|
||||
public ServerType getServerType() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/**
|
||||
* Stops Kestra.
|
||||
*/
|
||||
public void exit(int status) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/**
|
||||
* Kestra context initializer
|
||||
*/
|
||||
@Context
|
||||
@Requires(missingBeans = KestraContext.class)
|
||||
public static class Initializer extends KestraContext {
|
||||
|
||||
private final ApplicationContext applicationContext;
|
||||
private final Environment environment;
|
||||
|
||||
/**
|
||||
* Creates a new {@link KestraContext} instance.
|
||||
*
|
||||
* @param applicationContext The {@link ApplicationContext}.
|
||||
* @param environment The {@link Environment}.
|
||||
*/
|
||||
public Initializer(ApplicationContext applicationContext, Environment environment) {
|
||||
this.applicationContext = applicationContext;
|
||||
this.environment = environment;
|
||||
KestraContext.setContext(this);
|
||||
}
|
||||
|
||||
/** {@inheritDoc} **/
|
||||
@Override
|
||||
public ServerType getServerType() {
|
||||
return Optional.ofNullable(environment)
|
||||
.flatMap(env -> env.getProperty(KESTRA_SERVER_TYPE, ServerType.class))
|
||||
.orElseThrow(() -> new IllegalStateException("Cannot found required environment property '" + KESTRA_SERVER_TYPE + "'."));
|
||||
}
|
||||
|
||||
/** {@inheritDoc} **/
|
||||
@Override
|
||||
public void exit(int status) {
|
||||
applicationContext.close();
|
||||
Runtime.getRuntime().exit(status);
|
||||
}
|
||||
|
||||
@PreDestroy
|
||||
public void dispose() {
|
||||
setContext(null);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -8,6 +8,7 @@ import io.kestra.core.models.topologies.FlowTopology;
|
||||
import io.kestra.core.models.triggers.Trigger;
|
||||
import io.kestra.core.models.triggers.multipleflows.MultipleConditionWindow;
|
||||
import io.kestra.core.runners.*;
|
||||
import io.kestra.core.server.ServiceInstance;
|
||||
import jakarta.inject.Singleton;
|
||||
|
||||
@Singleton
|
||||
@@ -20,7 +21,7 @@ public class QueueService {
|
||||
} else if (object.getClass() == WorkerTaskRunning.class) {
|
||||
return ((WorkerTaskRunning) object).getTaskRun().getId();
|
||||
} else if (object.getClass() == WorkerInstance.class) {
|
||||
return ((WorkerInstance) object).getWorkerUuid().toString();
|
||||
return ((WorkerInstance) object).getWorkerUuid();
|
||||
} else if (object.getClass() == WorkerTaskResult.class) {
|
||||
return ((WorkerTaskResult) object).getTaskRun().getId();
|
||||
} else if (object.getClass() == LogEntry.class) {
|
||||
@@ -59,6 +60,8 @@ public class QueueService {
|
||||
return ((WorkerTriggerResult) object).getTriggerContext().uid();
|
||||
} else if (object.getClass() == ExecutionQueued.class) {
|
||||
return ((ExecutionQueued) object).uid();
|
||||
} else if (object.getClass() == ServiceInstance.class) {
|
||||
return ((ServiceInstance) object).id();
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unknown type '" + object.getClass().getName() + "'");
|
||||
}
|
||||
|
||||
@@ -12,5 +12,4 @@ public interface WorkerJobQueueInterface extends Closeable {
|
||||
|
||||
void pause();
|
||||
|
||||
void cleanup();
|
||||
}
|
||||
|
||||
@@ -0,0 +1,120 @@
|
||||
package io.kestra.core.repositories;
|
||||
|
||||
import io.kestra.core.server.Service;
|
||||
import io.kestra.core.server.ServiceInstance;
|
||||
import io.kestra.core.server.ServiceStateTransition;
|
||||
import org.apache.commons.lang3.tuple.ImmutablePair;
|
||||
|
||||
import java.time.Instant;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* Repository service for storing service instance.
|
||||
*
|
||||
* @see io.kestra.core.server.ServerInstance
|
||||
*/
|
||||
public interface ServiceInstanceRepositoryInterface {
|
||||
|
||||
/**
|
||||
* Finds the service instance for the given id.
|
||||
*
|
||||
* @param id The service's ID. cannot be {@code null}.
|
||||
* @return an {@link Optional} of {@link ServiceInstance}, or {@link Optional#empty()}
|
||||
*/
|
||||
Optional<ServiceInstance> findById(String id);
|
||||
|
||||
/**
|
||||
* Finds all service instance.
|
||||
*
|
||||
* @return a list of {@link ServiceInstance}.
|
||||
*/
|
||||
List<ServiceInstance> findAll();
|
||||
|
||||
/**
|
||||
* Deletes the given service instance.
|
||||
*
|
||||
* @param service The service to be deleted.
|
||||
*/
|
||||
void delete(ServiceInstance service);
|
||||
|
||||
/**
|
||||
* Saves the given service instance.
|
||||
*
|
||||
* @param service The service to be saved.
|
||||
* @return The saved instance.
|
||||
*/
|
||||
ServiceInstance save(ServiceInstance service);
|
||||
|
||||
/**
|
||||
* Finds all running service instances which have not sent a state update for more than their associated timeout,
|
||||
* and thus should be considered in failure.
|
||||
*
|
||||
* @param now the time instant to be used for querying.
|
||||
* @return the list of {@link ServiceInstance}.
|
||||
*/
|
||||
default List<ServiceInstance> findAllTimeoutRunningInstances(final Instant now) {
|
||||
return findAllInstancesInStates(List.of(Service.ServiceState.CREATED, Service.ServiceState.RUNNING))
|
||||
.stream()
|
||||
.filter(instance -> instance.isSessionTimeoutElapsed(now))
|
||||
.toList();
|
||||
}
|
||||
|
||||
/**
|
||||
* Finds all service instances which are in the given state.
|
||||
*
|
||||
* @return the list of {@link ServiceInstance}.
|
||||
*/
|
||||
List<ServiceInstance> findAllInstancesInState(final Service.ServiceState state);
|
||||
|
||||
/**
|
||||
* Finds all service instances which are in the given state.
|
||||
*
|
||||
* @return the list of {@link ServiceInstance}.
|
||||
*/
|
||||
List<ServiceInstance> findAllInstancesInStates(final List<Service.ServiceState> states);
|
||||
|
||||
/**
|
||||
* Attempt to transition the state of a given service to given new state.
|
||||
* This method may not update the service if the transition is not valid.
|
||||
*
|
||||
* @param instance the service instance.
|
||||
* @param newState the new state of the service.
|
||||
* @return an optional of the {@link ServiceInstance} or {@link Optional#empty()} if the service is not running.
|
||||
*/
|
||||
default ServiceStateTransition.Response mayTransitionServiceTo(final ServiceInstance instance,
|
||||
final Service.ServiceState newState) {
|
||||
return mayTransitionServiceTo(instance, newState, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Attempt to transition the state of a given service to given new state.
|
||||
* This method may not update the service if the transition is not valid.
|
||||
*
|
||||
* @param instance the service instance.
|
||||
* @param newState the new state of the service.
|
||||
* @param reason the human-readable reason of the state transition
|
||||
* @return an optional of the {@link ServiceInstance} or {@link Optional#empty()} if the service is not running.
|
||||
*/
|
||||
default ServiceStateTransition.Response mayTransitionServiceTo(final ServiceInstance instance,
|
||||
final Service.ServiceState newState,
|
||||
final String reason) {
|
||||
// This default method is not transactional and may lead to inconsistent state transition.
|
||||
synchronized (this) {
|
||||
Optional<ServiceInstance> optional = findById(instance.id());
|
||||
final ImmutablePair<ServiceInstance, ServiceInstance> beforeAndAfter;
|
||||
// UNKNOWN service
|
||||
if (optional.isEmpty()) {
|
||||
beforeAndAfter = null;
|
||||
// VALID service transition
|
||||
} else if (optional.get().state().isValidTransition(newState)) {
|
||||
ServiceInstance updated = optional.get().updateState(newState, Instant.now(), reason);
|
||||
beforeAndAfter = new ImmutablePair<>(optional.get(), save(updated));
|
||||
// INVALID service transition
|
||||
} else {
|
||||
beforeAndAfter = new ImmutablePair<>(optional.get(), null);
|
||||
}
|
||||
return ServiceStateTransition.logTransitionAndGetResponse(instance, newState, beforeAndAfter);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,16 +0,0 @@
|
||||
package io.kestra.core.repositories;
|
||||
|
||||
import io.kestra.core.runners.WorkerInstance;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
||||
public interface WorkerInstanceRepositoryInterface {
|
||||
Optional<WorkerInstance> findByWorkerUuid(String workerUuid);
|
||||
|
||||
List<WorkerInstance> findAll();
|
||||
|
||||
void delete(WorkerInstance workerInstance);
|
||||
|
||||
WorkerInstance save(WorkerInstance workerInstance);
|
||||
}
|
||||
@@ -1,30 +0,0 @@
|
||||
package io.kestra.core.runners;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import io.micronaut.context.annotation.Value;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
import jakarta.validation.constraints.NotNull;
|
||||
import lombok.Getter;
|
||||
|
||||
import java.util.Objects;
|
||||
import java.util.UUID;
|
||||
|
||||
/**
|
||||
* Runtime information about a Kestra server.
|
||||
*/
|
||||
public record ServerInstance(@NotNull UUID id) {
|
||||
private static final ServerInstance INSTANCE = new ServerInstance(UUID.randomUUID());
|
||||
|
||||
public ServerInstance {
|
||||
Objects.requireNonNull(id, "id cannot be null");
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the local {@link ServerInstance}.
|
||||
*/
|
||||
public static ServerInstance getInstance() {
|
||||
return INSTANCE;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -10,8 +10,11 @@ import io.kestra.core.exceptions.DeserializationException;
|
||||
import io.kestra.core.exceptions.TimeoutExceededException;
|
||||
import io.kestra.core.metrics.MetricRegistry;
|
||||
import io.kestra.core.models.Label;
|
||||
import io.kestra.core.models.executions.*;
|
||||
import io.kestra.core.models.flows.State;
|
||||
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.tasks.Output;
|
||||
import io.kestra.core.models.tasks.RunnableTask;
|
||||
import io.kestra.core.models.tasks.Task;
|
||||
@@ -23,6 +26,9 @@ import io.kestra.core.queues.QueueFactoryInterface;
|
||||
import io.kestra.core.queues.QueueInterface;
|
||||
import io.kestra.core.queues.WorkerJobQueueInterface;
|
||||
import io.kestra.core.serializers.JacksonMapper;
|
||||
import io.kestra.core.server.ServerConfig;
|
||||
import io.kestra.core.server.Service;
|
||||
import io.kestra.core.server.ServiceStateChangeEvent;
|
||||
import io.kestra.core.services.LogService;
|
||||
import io.kestra.core.services.WorkerGroupService;
|
||||
import io.kestra.core.tasks.flows.WorkingDirectory;
|
||||
@@ -30,8 +36,10 @@ import io.kestra.core.utils.Await;
|
||||
import io.kestra.core.utils.ExecutorsUtils;
|
||||
import io.kestra.core.utils.Hashing;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import io.micronaut.context.event.ApplicationEventPublisher;
|
||||
import io.micronaut.core.annotation.Introspected;
|
||||
import io.micronaut.inject.qualifiers.Qualifiers;
|
||||
import jakarta.inject.Inject;
|
||||
import lombok.Getter;
|
||||
import lombok.Synchronized;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
@@ -45,7 +53,14 @@ import java.io.IOException;
|
||||
import java.time.Duration;
|
||||
import java.time.ZonedDateTime;
|
||||
import java.time.temporal.ChronoUnit;
|
||||
import java.util.*;
|
||||
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.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
@@ -53,19 +68,28 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import static io.kestra.core.models.flows.State.Type.*;
|
||||
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.SUCCESS;
|
||||
import static io.kestra.core.models.flows.State.Type.WARNING;
|
||||
import static io.kestra.core.server.Service.ServiceState.TERMINATED_FORCED;
|
||||
import static io.kestra.core.server.Service.ServiceState.TERMINATED_GRACEFULLY;
|
||||
|
||||
@Slf4j
|
||||
@Introspected
|
||||
public class Worker implements Runnable, AutoCloseable {
|
||||
public class Worker implements Service, Runnable, AutoCloseable {
|
||||
private final static ObjectMapper MAPPER = JacksonMapper.ofJson();
|
||||
private static final String SERVICE_PROS_WORKER_GROUP = "workerGroup";
|
||||
|
||||
private final ApplicationContext applicationContext;
|
||||
private final WorkerJobQueueInterface workerJobQueue;
|
||||
private final QueueInterface<WorkerTaskResult> workerTaskResultQueue;
|
||||
|
||||
private final QueueInterface<WorkerTriggerResult> workerTriggerResultQueue;
|
||||
private final QueueInterface<ExecutionKilled> executionKilledQueue;
|
||||
private final QueueInterface<MetricEntry> metricEntryQueue;
|
||||
private final MetricRegistry metricRegistry;
|
||||
private final ServerConfig serverConfig;
|
||||
|
||||
private final Set<String> killedExecution = ConcurrentHashMap.newKeySet();
|
||||
|
||||
@@ -81,15 +105,32 @@ public class Worker implements Runnable, AutoCloseable {
|
||||
|
||||
private final List<WorkerThread> workerThreadReferences = new ArrayList<>();
|
||||
|
||||
private final ApplicationEventPublisher<ServiceStateChangeEvent> eventPublisher;
|
||||
|
||||
private final AtomicBoolean skipGracefulTermination = new AtomicBoolean(false);
|
||||
|
||||
@Getter
|
||||
private final String workerGroup;
|
||||
|
||||
private final LogService logService;
|
||||
|
||||
private final String id;
|
||||
private final AtomicReference<ServiceState> state = new AtomicReference<>();
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Inject
|
||||
public Worker(ApplicationContext applicationContext, int thread, String workerGroupKey) {
|
||||
// FIXME: For backward-compatibility with Kestra 0.15.x and earliest we still used UUID for Worker ID instead of IdUtils
|
||||
this(applicationContext, thread, workerGroupKey, UUID.randomUUID().toString());
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public Worker(ApplicationContext applicationContext, int thread, String workerGroupKey, String id) {
|
||||
this.id = id;
|
||||
this.applicationContext = applicationContext;
|
||||
this.workerJobQueue = applicationContext.getBean(WorkerJobQueueInterface.class);
|
||||
this.eventPublisher = applicationContext.getBean(ApplicationEventPublisher.class);
|
||||
|
||||
this.workerTaskResultQueue = (QueueInterface<WorkerTaskResult>) applicationContext.getBean(
|
||||
QueueInterface.class,
|
||||
Qualifiers.byName(QueueFactoryInterface.WORKERTASKRESULT_NAMED)
|
||||
@@ -115,10 +156,14 @@ public class Worker implements Runnable, AutoCloseable {
|
||||
this.workerGroup = workerGroupService.resolveGroupFromKey(workerGroupKey);
|
||||
|
||||
this.logService = applicationContext.getBean(LogService.class);
|
||||
|
||||
this.serverConfig = applicationContext.getBean(ServerConfig.class);
|
||||
setState(ServiceState.CREATED);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
setState(ServiceState.RUNNING);
|
||||
this.executionKilledQueue.receive(executionKilled -> {
|
||||
if(executionKilled == null || !executionKilled.isLeft()) {
|
||||
return;
|
||||
@@ -160,6 +205,13 @@ public class Worker implements Runnable, AutoCloseable {
|
||||
);
|
||||
}
|
||||
|
||||
private void setState(final ServiceState state) {
|
||||
this.state.set(state);
|
||||
Map<String, Object> properties = new HashMap<>();
|
||||
properties.put(SERVICE_PROS_WORKER_GROUP, workerGroup);
|
||||
eventPublisher.publishEvent(new ServiceStateChangeEvent(this, properties));
|
||||
}
|
||||
|
||||
private void handleDeserializationError(DeserializationException deserializationException) {
|
||||
if (deserializationException.getRecord() != null) {
|
||||
try {
|
||||
@@ -316,7 +368,7 @@ public class Worker implements Runnable, AutoCloseable {
|
||||
.counter(MetricRegistry.METRIC_WORKER_STARTED_COUNT, metricRegistry.tags(workerTask, workerGroup))
|
||||
.increment();
|
||||
|
||||
if (workerTask.getTaskRun().getState().getCurrent() == State.Type.CREATED) {
|
||||
if (workerTask.getTaskRun().getState().getCurrent() == CREATED) {
|
||||
metricRegistry
|
||||
.timer(MetricRegistry.METRIC_WORKER_QUEUED_DURATION, metricRegistry.tags(workerTask, workerGroup))
|
||||
.record(Duration.between(
|
||||
@@ -325,7 +377,7 @@ public class Worker implements Runnable, AutoCloseable {
|
||||
}
|
||||
|
||||
if (killedExecution.contains(workerTask.getTaskRun().getExecutionId())) {
|
||||
workerTask = workerTask.withTaskRun(workerTask.getTaskRun().withState(State.Type.KILLED));
|
||||
workerTask = workerTask.withTaskRun(workerTask.getTaskRun().withState(KILLED));
|
||||
|
||||
WorkerTaskResult workerTaskResult = new WorkerTaskResult(workerTask);
|
||||
this.workerTaskResultQueue.emit(workerTaskResult);
|
||||
@@ -345,7 +397,7 @@ public class Worker implements Runnable, AutoCloseable {
|
||||
workerTask.getTask().getClass().getSimpleName()
|
||||
);
|
||||
|
||||
workerTask = workerTask.withTaskRun(workerTask.getTaskRun().withState(State.Type.RUNNING));
|
||||
workerTask = workerTask.withTaskRun(workerTask.getTaskRun().withState(RUNNING));
|
||||
this.workerTaskResultQueue.emit(new WorkerTaskResult(workerTask));
|
||||
|
||||
AtomicReference<WorkerTask> current = new AtomicReference<>(workerTask);
|
||||
@@ -370,7 +422,7 @@ public class Worker implements Runnable, AutoCloseable {
|
||||
WorkerTask finalWorkerTask = Failsafe
|
||||
.with(workerTaskRetryPolicy
|
||||
.handleResultIf(result -> result.getTaskRun().lastAttempt() != null &&
|
||||
result.getTaskRun().lastAttempt().getState().getCurrent() == State.Type.FAILED &&
|
||||
result.getTaskRun().lastAttempt().getState().getCurrent() == FAILED &&
|
||||
!killedExecution.contains(result.getTaskRun().getExecutionId())
|
||||
)
|
||||
.onRetry(e -> {
|
||||
@@ -418,18 +470,18 @@ public class Worker implements Runnable, AutoCloseable {
|
||||
finalWorkerTask.getTaskRun().toString(true) + "'"
|
||||
);
|
||||
}
|
||||
State.Type state = lastAttempt.getState().getCurrent();
|
||||
io.kestra.core.models.flows.State.Type state = lastAttempt.getState().getCurrent();
|
||||
|
||||
if (workerTask.getTask().getRetry() != null &&
|
||||
workerTask.getTask().getRetry().getWarningOnRetry() &&
|
||||
finalWorkerTask.getTaskRun().attemptNumber() > 1 &&
|
||||
state == State.Type.SUCCESS
|
||||
state == SUCCESS
|
||||
) {
|
||||
state = State.Type.WARNING;
|
||||
state = WARNING;
|
||||
}
|
||||
|
||||
if (workerTask.getTask().isAllowFailure() && state.isFailed()) {
|
||||
state = State.Type.WARNING;
|
||||
state = WARNING;
|
||||
}
|
||||
|
||||
// emit
|
||||
@@ -504,8 +556,8 @@ public class Worker implements Runnable, AutoCloseable {
|
||||
|
||||
if (!(workerTask.getTask() instanceof RunnableTask<?> task)) {
|
||||
// This should never happen but better to deal with it than crashing the Worker
|
||||
var state = workerTask.getTask().isAllowFailure() ? State.Type.WARNING : State.Type.FAILED;
|
||||
TaskRunAttempt attempt = TaskRunAttempt.builder().state(new State().withState(state)).build();
|
||||
var state = workerTask.getTask().isAllowFailure() ? WARNING : FAILED;
|
||||
TaskRunAttempt attempt = TaskRunAttempt.builder().state(new io.kestra.core.models.flows.State().withState(state)).build();
|
||||
List<TaskRunAttempt> attempts = this.addAttempt(workerTask, attempt);
|
||||
TaskRun taskRun = workerTask.getTaskRun().withAttempts(attempts);
|
||||
logger.error("Unable to execute the task '" + workerTask.getTask().getId() +
|
||||
@@ -514,7 +566,7 @@ public class Worker implements Runnable, AutoCloseable {
|
||||
}
|
||||
|
||||
TaskRunAttempt.TaskRunAttemptBuilder builder = TaskRunAttempt.builder()
|
||||
.state(new State().withState(State.Type.RUNNING));
|
||||
.state(new io.kestra.core.models.flows.State().withState(RUNNING));
|
||||
|
||||
AtomicInteger metricRunningCount = getMetricRunningCount(workerTask);
|
||||
|
||||
@@ -531,7 +583,7 @@ public class Worker implements Runnable, AutoCloseable {
|
||||
));
|
||||
|
||||
// run it
|
||||
State.Type state;
|
||||
io.kestra.core.models.flows.State.Type state;
|
||||
try {
|
||||
synchronized (this) {
|
||||
workerThreadReferences.add(workerThread);
|
||||
@@ -541,7 +593,7 @@ public class Worker implements Runnable, AutoCloseable {
|
||||
state = workerThread.getTaskState();
|
||||
} catch (InterruptedException e) {
|
||||
logger.error("Failed to join WorkerThread {}", e.getMessage(), e);
|
||||
state = workerTask.getTask().isAllowFailure() ? State.Type.WARNING : State.Type.FAILED;;
|
||||
state = workerTask.getTask().isAllowFailure() ? WARNING : FAILED;
|
||||
} finally {
|
||||
synchronized (this) {
|
||||
workerThreadReferences.remove(workerThread);
|
||||
@@ -597,17 +649,36 @@ public class Worker implements Runnable, AutoCloseable {
|
||||
|
||||
@Override
|
||||
public void close() throws Exception {
|
||||
closeWorker(Duration.ofMinutes(5));
|
||||
closeWorker(serverConfig.terminationGracePeriod());
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void closeWorker(Duration awaitDuration) throws Exception {
|
||||
public void closeWorker(Duration timeout) throws Exception {
|
||||
log.info("Terminating.");
|
||||
setState(ServiceState.TERMINATING);
|
||||
workerJobQueue.pause();
|
||||
|
||||
final boolean terminatedGracefully;
|
||||
if (!skipGracefulTermination.get()) {
|
||||
terminatedGracefully = waitForTasksCompletion(timeout);
|
||||
} else {
|
||||
log.info("Terminating now and skip waiting for tasks completions.");
|
||||
this.executors.shutdownNow();
|
||||
closeWorkerTaskResultQueue();
|
||||
terminatedGracefully = false;
|
||||
}
|
||||
|
||||
ServiceState state = terminatedGracefully ? TERMINATED_GRACEFULLY : TERMINATED_FORCED;
|
||||
setState(state);
|
||||
log.info("Worker closed ({}).", state.name().toLowerCase());
|
||||
}
|
||||
|
||||
private boolean waitForTasksCompletion(final Duration timeout) {
|
||||
new Thread(
|
||||
() -> {
|
||||
try {
|
||||
this.executors.shutdown();
|
||||
this.executors.awaitTermination(awaitDuration.toMillis(), TimeUnit.MILLISECONDS);
|
||||
this.executors.awaitTermination(timeout.toMillis(), TimeUnit.MILLISECONDS);
|
||||
} catch (InterruptedException e) {
|
||||
log.error("Fail to shutdown the worker", e);
|
||||
}
|
||||
@@ -615,21 +686,15 @@ public class Worker implements Runnable, AutoCloseable {
|
||||
"worker-shutdown"
|
||||
).start();
|
||||
|
||||
AtomicBoolean cleanShutdown = new AtomicBoolean(false);
|
||||
|
||||
final AtomicBoolean cleanShutdown = new AtomicBoolean(false);
|
||||
Await.until(
|
||||
() -> {
|
||||
if (this.executors.isTerminated() || this.workerThreadReferences.isEmpty()) {
|
||||
log.info("No more worker threads busy, shutting down!");
|
||||
|
||||
// we ensure that last produce message are send
|
||||
try {
|
||||
this.workerTaskResultQueue.close();
|
||||
} catch (IOException e) {
|
||||
log.error("Failed to close the workerTaskResultQueue", e);
|
||||
}
|
||||
|
||||
cleanShutdown.set(true);;
|
||||
closeWorkerTaskResultQueue();
|
||||
cleanShutdown.set(true);
|
||||
return true;
|
||||
}
|
||||
|
||||
@@ -642,9 +707,14 @@ public class Worker implements Runnable, AutoCloseable {
|
||||
},
|
||||
Duration.ofSeconds(1)
|
||||
);
|
||||
return cleanShutdown.get();
|
||||
}
|
||||
|
||||
if (cleanShutdown.get()) {
|
||||
workerJobQueue.cleanup();
|
||||
private void closeWorkerTaskResultQueue() {
|
||||
try {
|
||||
this.workerTaskResultQueue.close();
|
||||
} catch (IOException e) {
|
||||
log.error("Failed to close the workerTaskResultQueue", e);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -710,7 +780,7 @@ public class Worker implements Runnable, AutoCloseable {
|
||||
taskOutput = task.run(runContext);
|
||||
}
|
||||
|
||||
taskState = io.kestra.core.models.flows.State.Type.SUCCESS;
|
||||
taskState = SUCCESS;
|
||||
if (taskOutput != null && taskOutput.finalState().isPresent()) {
|
||||
taskState = taskOutput.finalState().get();
|
||||
}
|
||||
@@ -724,15 +794,50 @@ public class Worker implements Runnable, AutoCloseable {
|
||||
@Synchronized
|
||||
public void kill() {
|
||||
this.killed = true;
|
||||
taskState = io.kestra.core.models.flows.State.Type.KILLED;
|
||||
taskState = KILLED;
|
||||
this.interrupt();
|
||||
}
|
||||
|
||||
private void exceptionHandler(Thread t, Throwable e) {
|
||||
if (!this.killed) {
|
||||
logger.error(e.getMessage(), e);
|
||||
taskState = io.kestra.core.models.flows.State.Type.FAILED;
|
||||
taskState = FAILED;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Specify whether to skip graceful termination on shutdown.
|
||||
*
|
||||
* @param skipGracefulTermination {@code true} to skip graceful termination on shutdown.
|
||||
*/
|
||||
@Override
|
||||
public void skipGracefulTermination(final boolean skipGracefulTermination) {
|
||||
this.skipGracefulTermination.set(skipGracefulTermination);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public ServiceType getType() {
|
||||
return ServiceType.WORKER;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public ServiceState getState() {
|
||||
return state.get();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -1,49 +1,30 @@
|
||||
package io.kestra.core.runners;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import lombok.*;
|
||||
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
|
||||
import jakarta.validation.constraints.NotNull;
|
||||
import lombok.Builder;
|
||||
import lombok.Data;
|
||||
import lombok.Getter;
|
||||
import lombok.NoArgsConstructor;
|
||||
import lombok.ToString;
|
||||
import lombok.experimental.SuperBuilder;
|
||||
|
||||
import jakarta.validation.constraints.NotNull;
|
||||
import java.time.Instant;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
|
||||
@Data
|
||||
@SuperBuilder(toBuilder = true)
|
||||
@ToString
|
||||
@NoArgsConstructor
|
||||
@Getter
|
||||
@JsonIgnoreProperties(ignoreUnknown = true)
|
||||
public class WorkerInstance {
|
||||
@NotNull
|
||||
private UUID workerUuid;
|
||||
|
||||
@NotNull
|
||||
private String hostname;
|
||||
private Integer port;
|
||||
private Integer managementPort;
|
||||
private String workerUuid;
|
||||
|
||||
private String workerGroup;
|
||||
|
||||
@Builder.Default
|
||||
private List<Integer> partitions = new ArrayList<>();
|
||||
|
||||
@Builder.Default
|
||||
@JsonInclude
|
||||
private Status status = Status.UP;
|
||||
|
||||
@Builder.Default
|
||||
private Instant heartbeatDate = Instant.now();
|
||||
|
||||
/**
|
||||
* The Kestra server owning the worker.
|
||||
*/
|
||||
@Builder.Default
|
||||
private ServerInstance server = ServerInstance.getInstance();
|
||||
|
||||
public enum Status {
|
||||
UP, DEAD
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -18,11 +18,14 @@ import io.kestra.core.queues.QueueFactoryInterface;
|
||||
import io.kestra.core.queues.QueueInterface;
|
||||
import io.kestra.core.queues.WorkerTriggerResultQueueInterface;
|
||||
import io.kestra.core.runners.*;
|
||||
import io.kestra.core.server.Service;
|
||||
import io.kestra.core.server.ServiceStateChangeEvent;
|
||||
import io.kestra.core.services.*;
|
||||
import io.kestra.core.utils.Await;
|
||||
import io.kestra.core.utils.IdUtils;
|
||||
import io.kestra.core.utils.ListUtils;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import io.micronaut.context.event.ApplicationEventPublisher;
|
||||
import io.micronaut.inject.qualifiers.Qualifiers;
|
||||
import jakarta.annotation.PreDestroy;
|
||||
import jakarta.inject.Inject;
|
||||
@@ -42,13 +45,14 @@ import java.time.ZonedDateTime;
|
||||
import java.time.temporal.ChronoUnit;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.*;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.BiConsumer;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
@Slf4j
|
||||
@Singleton
|
||||
public abstract class AbstractScheduler implements Scheduler {
|
||||
public abstract class AbstractScheduler implements Scheduler, Service {
|
||||
protected final ApplicationContext applicationContext;
|
||||
private final QueueInterface<Execution> executionQueue;
|
||||
private final QueueInterface<Trigger> triggerQueue;
|
||||
@@ -75,6 +79,10 @@ public abstract class AbstractScheduler implements Scheduler {
|
||||
@Getter
|
||||
private volatile Map<String, FlowWithPollingTriggerNextDate> schedulableNextDate = new ConcurrentHashMap<>();
|
||||
|
||||
private final String id = IdUtils.create();
|
||||
private final AtomicReference<ServiceState> state = new AtomicReference<>();
|
||||
private final ApplicationEventPublisher<ServiceStateChangeEvent> eventPublisher;
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Inject
|
||||
public AbstractScheduler(
|
||||
@@ -93,6 +101,8 @@ public abstract class AbstractScheduler implements Scheduler {
|
||||
this.taskDefaultService = applicationContext.getBean(TaskDefaultService.class);
|
||||
this.workerGroupService = applicationContext.getBean(WorkerGroupService.class);
|
||||
this.logService = applicationContext.getBean(LogService.class);
|
||||
this.eventPublisher = applicationContext.getBean(ApplicationEventPublisher.class);
|
||||
setState(ServiceState.CREATED);
|
||||
}
|
||||
|
||||
protected boolean isReady() {
|
||||
@@ -181,6 +191,7 @@ public abstract class AbstractScheduler implements Scheduler {
|
||||
}
|
||||
}
|
||||
);
|
||||
setState(ServiceState.RUNNING);
|
||||
}
|
||||
|
||||
// Initialized local trigger state,
|
||||
@@ -677,7 +688,9 @@ public abstract class AbstractScheduler implements Scheduler {
|
||||
@Override
|
||||
@PreDestroy
|
||||
public void close() {
|
||||
setState(ServiceState.TERMINATING);
|
||||
this.scheduleExecutor.shutdown();
|
||||
setState(ServiceState.TERMINATED_GRACEFULLY);
|
||||
}
|
||||
|
||||
@SuperBuilder(toBuilder = true)
|
||||
@@ -748,4 +761,27 @@ public abstract class AbstractScheduler implements Scheduler {
|
||||
return Trigger.uid(flow, AbstractTrigger);
|
||||
}
|
||||
}
|
||||
|
||||
private void setState(final ServiceState state) {
|
||||
this.state.set(state);
|
||||
eventPublisher.publishEvent(new ServiceStateChangeEvent(this));
|
||||
}
|
||||
|
||||
/** {@inheritDoc} **/
|
||||
@Override
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} **/
|
||||
@Override
|
||||
public ServiceType getType() {
|
||||
return ServiceType.SCHEDULER;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} **/
|
||||
@Override
|
||||
public ServiceState getState() {
|
||||
return state.get();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,132 @@
|
||||
package io.kestra.core.server;
|
||||
|
||||
import io.kestra.core.repositories.ServiceInstanceRepositoryInterface;
|
||||
import io.micronaut.core.annotation.Introspected;
|
||||
import jakarta.inject.Inject;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.random.RandomGenerator;
|
||||
|
||||
import static io.kestra.core.server.Service.ServiceState.CREATED;
|
||||
import static io.kestra.core.server.Service.ServiceState.RUNNING;
|
||||
|
||||
/**
|
||||
* Base class for coordinating service liveness.
|
||||
*/
|
||||
@Introspected
|
||||
@Slf4j
|
||||
public abstract class AbstractServiceLivenessCoordinator extends AbstractServiceLivenessTask {
|
||||
|
||||
private final static int DEFAULT_SCHEDULE_JITTER_MAX_MS = 500;
|
||||
|
||||
private static final String TASK_NAME = "service-liveness-coordinator-task";
|
||||
|
||||
protected final ServiceInstanceRepositoryInterface serviceInstanceRepository;
|
||||
|
||||
// mutable for testing purpose
|
||||
protected String serverId = ServerInstance.INSTANCE_ID;
|
||||
|
||||
/**
|
||||
* Creates a new {@link AbstractServiceLivenessCoordinator} instance.
|
||||
*
|
||||
* @param serviceInstanceRepository The {@link ServiceInstanceRepositoryInterface}.
|
||||
* @param serverConfig The server configuration.
|
||||
*/
|
||||
@Inject
|
||||
public AbstractServiceLivenessCoordinator(final ServiceInstanceRepositoryInterface serviceInstanceRepository,
|
||||
final ServerConfig serverConfig) {
|
||||
super(TASK_NAME, serverConfig);
|
||||
this.serviceInstanceRepository = serviceInstanceRepository;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
**/
|
||||
@Override
|
||||
protected Duration getScheduleInterval() {
|
||||
// Multiple Executors can be running in parallel. We add a jitter to
|
||||
// help distributing the load more evenly among the ServiceLivenessCoordinator.
|
||||
// This is also used to prevent all ServiceLivenessCoordinator from attempting to query the repository simultaneously.
|
||||
RandomGenerator r = RandomGenerator.getDefault();
|
||||
int jitter = r.nextInt(DEFAULT_SCHEDULE_JITTER_MAX_MS);
|
||||
return serverConfig.liveness().interval().plus(Duration.ofMillis(jitter));
|
||||
}
|
||||
|
||||
/**
|
||||
* Transitions to the DISCONNECTED state all non-local services for which liveness
|
||||
* is enabled and are detected as non-responsive .
|
||||
*
|
||||
* @param now the instant.
|
||||
*/
|
||||
protected void transitionAllNonRespondingService(final Instant now) {
|
||||
|
||||
// Detect and handle non-responding services.
|
||||
List<ServiceInstance> nonRespondingServices = serviceInstanceRepository
|
||||
// gets all non-responding services.
|
||||
.findAllTimeoutRunningInstances(now)
|
||||
.stream()
|
||||
// keep only services with liveness enabled.
|
||||
.filter(instance -> instance.config().liveness().enabled())
|
||||
// exclude any service running on the same server as the executor, to prevent the latter from shutting down.
|
||||
.filter(instance -> !instance.server().id().equals(serverId))
|
||||
// only keep services eligible for liveness probe
|
||||
.filter(instance -> {
|
||||
final Instant minInstantForLivenessProbe = now.minus(instance.config().liveness().initialDelay());
|
||||
return instance.createdAt().isBefore(minInstantForLivenessProbe);
|
||||
})
|
||||
// warn
|
||||
.peek(instance -> log.warn("Detected non-responding service [id={}, type={}, hostname={}] after timeout ({}ms).",
|
||||
instance.id(),
|
||||
instance.type(),
|
||||
instance.server().hostname(),
|
||||
now.toEpochMilli() - instance.updatedAt().toEpochMilli()
|
||||
))
|
||||
.toList();
|
||||
|
||||
// Attempt to transit all non-responding services to DISCONNECTED.
|
||||
nonRespondingServices.forEach(instance -> this.safelyTransitionServiceTo(
|
||||
instance,
|
||||
Service.ServiceState.DISCONNECTED,
|
||||
"The service was detected as non-responsive after the session timeout. Service was transitioned to the 'DISCONNECTED' state."
|
||||
));
|
||||
}
|
||||
|
||||
protected void mayDetectAndLogNewConnectedServices() {
|
||||
if (log.isInfoEnabled()) {
|
||||
// Log the newly-connected services (useful for troubleshooting).
|
||||
serviceInstanceRepository.findAllInstancesInStates(List.of(CREATED, RUNNING))
|
||||
.stream()
|
||||
.filter(instance -> instance.createdAt().isAfter(lastScheduledExecution()))
|
||||
.forEach(instance -> {
|
||||
log.info("Detected new service [id={}, type={}, hostname={}] (started at: {}).",
|
||||
instance.id(),
|
||||
instance.type(),
|
||||
instance.server().hostname(),
|
||||
instance.createdAt()
|
||||
);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
protected void safelyTransitionServiceTo(final ServiceInstance instance,
|
||||
final Service.ServiceState state,
|
||||
final String reason) {
|
||||
try {
|
||||
serviceInstanceRepository.mayTransitionServiceTo(instance, state, reason);
|
||||
} catch (Exception e) {
|
||||
// Log and ignore exception - it's safe to ignore error because the run() method is supposed to schedule at fix rate.
|
||||
log.error("Unexpected error while service [id={}, type={}, hostname={}] transition from {} to {}. Error: {}",
|
||||
instance.id(),
|
||||
instance.type(),
|
||||
instance.server().hostname(),
|
||||
instance.state(),
|
||||
state,
|
||||
e.getMessage()
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,140 @@
|
||||
package io.kestra.core.server;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import io.micronaut.core.annotation.Introspected;
|
||||
import jakarta.annotation.PostConstruct;
|
||||
import jakarta.annotation.PreDestroy;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
/**
|
||||
* Base class for scheduling a task that operate on Worker liveness.
|
||||
*/
|
||||
@Introspected
|
||||
@Slf4j
|
||||
public abstract class AbstractServiceLivenessTask implements Runnable, AutoCloseable {
|
||||
|
||||
private final String name;
|
||||
protected final ServerConfig serverConfig;
|
||||
private final AtomicBoolean isStopped = new AtomicBoolean(false);
|
||||
private ScheduledExecutorService scheduledExecutorService;
|
||||
private Instant lastScheduledExecution;
|
||||
|
||||
/**
|
||||
* Creates a new {@link AbstractServiceLivenessTask} instance.
|
||||
*
|
||||
* @param name the task name.
|
||||
* @param configuration the liveness configuration.
|
||||
*/
|
||||
protected AbstractServiceLivenessTask(final String name,
|
||||
final ServerConfig configuration) {
|
||||
this.name = Objects.requireNonNull(name, "name cannot be null");
|
||||
this.serverConfig = Objects.requireNonNull(configuration, "serverConfig cannot be null");
|
||||
this.lastScheduledExecution = Instant.now();
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
**/
|
||||
@Override
|
||||
public void run() {
|
||||
final Instant now = Instant.now();
|
||||
run(now);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void run(final Instant now) {
|
||||
try {
|
||||
long elapsed = getElapsedMilliSinceLastSchedule(now);
|
||||
long timeout = serverConfig.liveness().timeout().toMillis();
|
||||
if (elapsed > timeout) {
|
||||
// useful for debugging unexpected heartbeat timeout
|
||||
log.warn("Thread starvation or clock leap detected (elapsed since previous schedule {}", elapsed);
|
||||
}
|
||||
onSchedule(now);
|
||||
} catch (Exception e) {
|
||||
log.error("Unexpected error while executing '{}'. Error: {}", now, e.getMessage());
|
||||
} finally {
|
||||
lastScheduledExecution = now;
|
||||
}
|
||||
}
|
||||
|
||||
protected Instant lastScheduledExecution() {
|
||||
return lastScheduledExecution;
|
||||
}
|
||||
|
||||
protected long getElapsedMilliSinceLastSchedule(final Instant now) {
|
||||
return now.toEpochMilli() - lastScheduledExecution.toEpochMilli();
|
||||
}
|
||||
|
||||
/**
|
||||
* The callback method invoked on each schedule.
|
||||
*
|
||||
* @param now the time of the execution.
|
||||
* @throws Exception when something goes wrong during the execution.
|
||||
*/
|
||||
protected abstract void onSchedule(final Instant now) throws Exception;
|
||||
|
||||
/**
|
||||
* Starts this task.
|
||||
*/
|
||||
@PostConstruct
|
||||
public void start() {
|
||||
if (!isLivenessEnabled()) {
|
||||
log.warn(
|
||||
"Server liveness is currently disabled (`kestra.server.liveness.enabled=false`) " +
|
||||
"If you are running in production environment, please ensure this property is configured to `true`. "
|
||||
);
|
||||
}
|
||||
if (scheduledExecutorService == null && !isStopped.get()) {
|
||||
scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(r -> new Thread(r, name));
|
||||
Duration scheduleInterval = getScheduleInterval();
|
||||
log.debug("Scheduling '{}' at fixed rate {}.", name, scheduleInterval);
|
||||
Duration initialDelay = serverConfig.liveness().initialDelay();
|
||||
scheduledExecutorService.scheduleAtFixedRate(
|
||||
this,
|
||||
initialDelay.toSeconds(),
|
||||
scheduleInterval.toSeconds(),
|
||||
TimeUnit.SECONDS
|
||||
);
|
||||
} else {
|
||||
throw new IllegalStateException(
|
||||
"The task '" + name + "' is either already started or already stopped, cannot re-start");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks whether liveness is enabled.
|
||||
*
|
||||
* @return {@code true} if liveness is enabled.
|
||||
*/
|
||||
protected Boolean isLivenessEnabled() {
|
||||
return serverConfig.liveness().enabled();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the fixed rate duration for scheduling this task.
|
||||
*
|
||||
* @return a {@link Duration}.
|
||||
*/
|
||||
protected abstract Duration getScheduleInterval();
|
||||
|
||||
/**
|
||||
* Closes this task.
|
||||
*/
|
||||
@PreDestroy
|
||||
@Override
|
||||
public void close() {
|
||||
if (isStopped.compareAndSet(false, true) && scheduledExecutorService != null) {
|
||||
scheduledExecutorService.shutdown();
|
||||
log.debug("Stopped scheduled '{}' task.", name);
|
||||
}
|
||||
}
|
||||
}
|
||||
46
core/src/main/java/io/kestra/core/server/ServerConfig.java
Normal file
46
core/src/main/java/io/kestra/core/server/ServerConfig.java
Normal file
@@ -0,0 +1,46 @@
|
||||
package io.kestra.core.server;
|
||||
|
||||
import io.micronaut.context.annotation.ConfigurationProperties;
|
||||
import io.micronaut.core.bind.annotation.Bindable;
|
||||
import jakarta.validation.constraints.NotNull;
|
||||
|
||||
import java.time.Duration;
|
||||
|
||||
/**
|
||||
* Server configuration.
|
||||
*
|
||||
* @param terminationGracePeriod The expected time a worker to complete all of its
|
||||
* tasks before initiating a graceful shutdown.
|
||||
*/
|
||||
@ConfigurationProperties("kestra.server")
|
||||
public record ServerConfig(
|
||||
@NotNull
|
||||
@Bindable(defaultValue = "5m")
|
||||
Duration terminationGracePeriod,
|
||||
|
||||
Liveness liveness
|
||||
|
||||
) {
|
||||
/**
|
||||
* Configuration for Liveness and Heartbeat mechanism between Kestra Services, and Executor.
|
||||
*
|
||||
* @param interval The expected time between liveness probe.
|
||||
* @param timeout The timeout used to detect service failures.
|
||||
* Kestra services sends periodic heartbeats to indicate their liveness.
|
||||
* For Workers, if no heartbeats are received by the executor before the expiration of this session timeout,
|
||||
* then the executor will remove any timeout workers from the cluster and eventually resubmit all their tasks.
|
||||
* @param initialDelay The time to wait before executing a liveness probe for a service.
|
||||
* @param heartbeatInterval The expected time between heartbeats.
|
||||
*/
|
||||
@ConfigurationProperties("liveness")
|
||||
public record Liveness(
|
||||
@NotNull
|
||||
@Bindable(defaultValue = "true") Boolean enabled,
|
||||
@NotNull @Bindable(defaultValue = "5s")
|
||||
Duration interval,
|
||||
@NotNull @Bindable(defaultValue = "45s") Duration timeout,
|
||||
@NotNull @Bindable(defaultValue = "30s") Duration initialDelay,
|
||||
@NotNull @Bindable(defaultValue = "3s") Duration heartbeatInterval
|
||||
) {
|
||||
}
|
||||
}
|
||||
42
core/src/main/java/io/kestra/core/server/ServerInstance.java
Normal file
42
core/src/main/java/io/kestra/core/server/ServerInstance.java
Normal file
@@ -0,0 +1,42 @@
|
||||
package io.kestra.core.server;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import io.kestra.core.utils.IdUtils;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Runtime information about a Kestra server (i.e. JVM).
|
||||
*
|
||||
* @param id The server instance ID.
|
||||
* @param type The server type.
|
||||
* @param version The server version.
|
||||
* @param props The server properties - an opaque map of key/value properties.
|
||||
*/
|
||||
@JsonInclude(JsonInclude.Include.ALWAYS)
|
||||
public record ServerInstance(
|
||||
String id,
|
||||
Type type,
|
||||
String version,
|
||||
String hostname,
|
||||
Map<String, Object> props
|
||||
|
||||
) {
|
||||
/// Static JVM Instance UUID
|
||||
public static final String INSTANCE_ID = IdUtils.create();
|
||||
|
||||
/**
|
||||
* Creates a new {@link ServerInstance} using the static local instance ID.
|
||||
*/
|
||||
public ServerInstance(final Type type,
|
||||
final String version,
|
||||
final String hostname,
|
||||
final Map<String, Object> props) {
|
||||
this(INSTANCE_ID, type, version, hostname, props);
|
||||
}
|
||||
|
||||
public enum Type {
|
||||
SERVER, STANDALONE;
|
||||
}
|
||||
|
||||
}
|
||||
132
core/src/main/java/io/kestra/core/server/Service.java
Normal file
132
core/src/main/java/io/kestra/core/server/Service.java
Normal file
@@ -0,0 +1,132 @@
|
||||
package io.kestra.core.server;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Interface for Kestra's Service
|
||||
*/
|
||||
public interface Service {
|
||||
|
||||
/**
|
||||
* Gets the unique identifier for this service.
|
||||
*
|
||||
* @return the string id.
|
||||
*/
|
||||
String getId();
|
||||
|
||||
/**
|
||||
* Gets the service type.
|
||||
*
|
||||
* @return the type.
|
||||
*/
|
||||
ServiceType getType();
|
||||
|
||||
/**
|
||||
* Gets the service state.
|
||||
*
|
||||
* @return the state.
|
||||
*/
|
||||
ServiceState getState();
|
||||
|
||||
/**
|
||||
* Specify whether to skip graceful termination on shutdown.
|
||||
*
|
||||
* @param skipGracefulTermination {@code true} to skip graceful termination on shutdown.
|
||||
*/
|
||||
default void skipGracefulTermination(final boolean skipGracefulTermination) {
|
||||
// noop
|
||||
}
|
||||
|
||||
/**
|
||||
* Supported service types.
|
||||
*/
|
||||
enum ServiceType {
|
||||
EXECUTOR,
|
||||
INDEXER,
|
||||
SCHEDULER,
|
||||
WEBSERVER,
|
||||
WORKER,
|
||||
}
|
||||
|
||||
/**
|
||||
* {@link ServiceState} are the possible states that a Kestra's Service can be in.
|
||||
* An instance must only be in one state at a time.
|
||||
* The expected state transition with the following defined states is:
|
||||
*
|
||||
* <pre>
|
||||
*
|
||||
* +--------------+
|
||||
* | Created (0) |------------->+
|
||||
* +------+-------+ |
|
||||
* | |
|
||||
* v |
|
||||
* +--------------+ |
|
||||
* +<----- | Running (1) | ------------>+
|
||||
* | +------+-------+ |
|
||||
* +----+----+ | |
|
||||
* | Error(2)| | |
|
||||
* +----+----+ | |
|
||||
* | v v
|
||||
* | +------+----------+ +------+------------+
|
||||
* +-----> | Terminating (4) |<---| Disconnected (3) |
|
||||
* +------+----------+ +-------------------+
|
||||
* | |
|
||||
* v v
|
||||
* +------+-------+ +------+-------+
|
||||
* | Terminated | | Terminated |
|
||||
* | Graceful (5) | | Forced (6) |
|
||||
* +--------------+ +--------------+
|
||||
* | |
|
||||
* v v
|
||||
* +------+-------+
|
||||
* | Not |
|
||||
* | Running (7) |
|
||||
* +------+-------+
|
||||
* |
|
||||
* v
|
||||
* +------+-------+
|
||||
* | Empty (8) |
|
||||
* +------+-------+
|
||||
* </pre>
|
||||
*/
|
||||
enum ServiceState {
|
||||
CREATED(1, 2, 3), // 0
|
||||
RUNNING(2, 3, 4), // 1
|
||||
ERROR(4), // 2
|
||||
DISCONNECTED(4, 7), // 3
|
||||
TERMINATING(5, 6, 7), // 4
|
||||
TERMINATED_GRACEFULLY(7), // 5
|
||||
TERMINATED_FORCED(7), // 6
|
||||
NOT_RUNNING(8), // 7
|
||||
EMPTY(); // 8
|
||||
|
||||
private final Set<Integer> validTransitions = new HashSet<>();
|
||||
|
||||
ServiceState(final Integer... validTransitions) {
|
||||
this.validTransitions.addAll(Arrays.asList(validTransitions));
|
||||
}
|
||||
|
||||
public boolean isValidTransition(final ServiceState newState) {
|
||||
return validTransitions.contains(newState.ordinal()) || equals(newState);
|
||||
}
|
||||
|
||||
public boolean isRunning() {
|
||||
return equals(CREATED)
|
||||
|| equals(RUNNING);
|
||||
}
|
||||
|
||||
public boolean isDisconnectedOrTerminating() {
|
||||
return equals(TERMINATING)
|
||||
|| equals(DISCONNECTED);
|
||||
}
|
||||
|
||||
public boolean hasCompletedTermination() {
|
||||
return equals(TERMINATED_GRACEFULLY)
|
||||
|| equals(TERMINATED_FORCED)
|
||||
|| equals(NOT_RUNNING)
|
||||
|| equals(EMPTY);
|
||||
}
|
||||
}
|
||||
}
|
||||
150
core/src/main/java/io/kestra/core/server/ServiceInstance.java
Normal file
150
core/src/main/java/io/kestra/core/server/ServiceInstance.java
Normal file
@@ -0,0 +1,150 @@
|
||||
package io.kestra.core.server;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import io.kestra.core.server.Service.ServiceState;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Runtime information about a Kestra's service (e.g., WORKER, EXECUTOR, etc.).
|
||||
*
|
||||
* @param id The service unique identifier.
|
||||
* @param type The service type.
|
||||
* @param state The state of the service.
|
||||
* @param server The server running this service.
|
||||
* @param createdAt Instant when this service was created.
|
||||
* @param updatedAt Instant when this service was updated.
|
||||
* @param events The last of events attached to this service - used to provide some contextual information about a state changed.
|
||||
* @param config The server configuration and liveness.
|
||||
* @param props The server additional properties - an opaque map of key/value pairs.
|
||||
* @param seqId A monolithic sequence id which is incremented each time the service instance is updated.
|
||||
* Used to detect non-transactional update of the instance.
|
||||
*/
|
||||
@JsonInclude(JsonInclude.Include.ALWAYS)
|
||||
public record ServiceInstance(
|
||||
String id,
|
||||
Service.ServiceType type,
|
||||
ServiceState state,
|
||||
ServerInstance server,
|
||||
Instant createdAt,
|
||||
Instant updatedAt,
|
||||
List<TimestampedEvent> events,
|
||||
ServerConfig config,
|
||||
Map<String, Object> props,
|
||||
long seqId
|
||||
) {
|
||||
|
||||
public ServiceInstance(
|
||||
String id,
|
||||
Service.ServiceType type,
|
||||
ServiceState state,
|
||||
ServerInstance server,
|
||||
Instant createdAt,
|
||||
Instant updatedAt,
|
||||
List<TimestampedEvent> events,
|
||||
ServerConfig config,
|
||||
Map<String, Object> props
|
||||
) {
|
||||
this(id, type, state, server, createdAt, updatedAt, events, config, props, 0L);
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks service type.
|
||||
*
|
||||
* @param type the type to check.
|
||||
* @return {@code true} if this instance is of the given type.
|
||||
*/
|
||||
public boolean is(final Service.ServiceType type) {
|
||||
return this.type.equals(type);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check service state.
|
||||
*
|
||||
* @param state the state to check.
|
||||
* @return {@code true} if this instance is in the given state.
|
||||
*/
|
||||
public boolean is(final ServiceState state) {
|
||||
return this.state.equals(state);
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates this service instance with the given state and instant.
|
||||
*
|
||||
* @param newState The new state.
|
||||
* @param updatedAt The update instant
|
||||
* @return a new {@link ServiceInstance}.
|
||||
*/
|
||||
public ServiceInstance updateState(final ServiceState newState,
|
||||
final Instant updatedAt) {
|
||||
return updateState(newState, updatedAt, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates this service instance with the given state and instant.
|
||||
*
|
||||
* @param newState The new state.
|
||||
* @param updatedAt The update instant
|
||||
* @param reason The human-readable reason of the update.
|
||||
* @return a new {@link ServiceInstance}.
|
||||
*/
|
||||
public ServiceInstance updateState(final ServiceState newState,
|
||||
final Instant updatedAt,
|
||||
final String reason) {
|
||||
|
||||
List<TimestampedEvent> events = this.events;
|
||||
if (reason != null) {
|
||||
events = new ArrayList<>(events);
|
||||
events.add(new TimestampedEvent(updatedAt, reason));
|
||||
}
|
||||
|
||||
long nextSeqId = seqId + 1;
|
||||
return new ServiceInstance(
|
||||
id,
|
||||
type,
|
||||
newState,
|
||||
server,
|
||||
createdAt,
|
||||
updatedAt,
|
||||
events,
|
||||
config,
|
||||
props,
|
||||
nextSeqId
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks whether the session timeout elapsed for this service.
|
||||
*
|
||||
* @param now The instant.
|
||||
* @return {@code true} if the session for this service has timeout, otherwise {@code false}.
|
||||
*/
|
||||
public boolean isSessionTimeoutElapsed(final Instant now) {
|
||||
Duration timeout = this.config.liveness().timeout();
|
||||
return this.state.isRunning() && updatedAt().plus(timeout).isBefore(now);
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks whether the termination grace period elapsed for this service.
|
||||
*
|
||||
* @param now The instant.
|
||||
* @return {@code true} if the termination grace period elapsed, otherwise {@code false}.
|
||||
*/
|
||||
public boolean isTerminationGracePeriodElapsed(final Instant now) {
|
||||
Duration terminationGracePeriod = this.config.terminationGracePeriod();
|
||||
return this.updatedAt().plus(terminationGracePeriod).isBefore(now);
|
||||
}
|
||||
|
||||
/**
|
||||
* A timestamped event value.
|
||||
*
|
||||
* @param ts The instant of this event.
|
||||
* @param value The value of this event.
|
||||
*/
|
||||
public record TimestampedEvent(Instant ts, String value) {
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,95 @@
|
||||
package io.kestra.core.server;
|
||||
|
||||
import io.kestra.core.models.ServerType;
|
||||
import io.kestra.core.utils.Network;
|
||||
import io.kestra.core.utils.VersionProvider;
|
||||
import io.micronaut.context.env.Environment;
|
||||
import jakarta.annotation.Nullable;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
import jakarta.validation.constraints.NotNull;
|
||||
|
||||
import java.time.Instant;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* Default factory service for constructing {@link ServiceInstance} objects.
|
||||
*/
|
||||
@Singleton
|
||||
public class ServiceInstanceFactory {
|
||||
|
||||
private final ServerConfig serverConfig;
|
||||
private final VersionProvider versionProvider;
|
||||
private final Environment environment;
|
||||
|
||||
@Inject
|
||||
public ServiceInstanceFactory(final ServerConfig serverConfig,
|
||||
final VersionProvider versionProvider,
|
||||
final Environment environment) {
|
||||
this.serverConfig = serverConfig;
|
||||
this.versionProvider = versionProvider;
|
||||
this.environment = environment;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new {@link ServiceInstance} for the given type.
|
||||
*
|
||||
* @param service The service.
|
||||
* @return a new {@link ServiceInstance}.
|
||||
*/
|
||||
public ServiceInstance newServiceInstance(@NotNull final Service service,
|
||||
@Nullable final Map<String, Object> properties) {
|
||||
Objects.requireNonNull(service, "Cannot create ServiceInstance for null service");
|
||||
|
||||
final Instant now = Instant.now();
|
||||
return new ServiceInstance(
|
||||
service.getId(),
|
||||
service.getType(),
|
||||
service.getState(),
|
||||
newServerInstance(),
|
||||
now,
|
||||
now,
|
||||
List.of(),
|
||||
serverConfig,
|
||||
Optional.ofNullable(properties).orElse(Map.of())
|
||||
);
|
||||
}
|
||||
|
||||
private ServerInstance newServerInstance() {
|
||||
return new ServerInstance(
|
||||
getInstanceType(),
|
||||
Optional.ofNullable(versionProvider).map(VersionProvider::getVersion).orElse(null),
|
||||
Network.localHostname(),
|
||||
Map.of(
|
||||
"server.port", getServerPort(),
|
||||
"server.management.port", getServerManagementPort()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
private ServerInstance.Type getInstanceType() {
|
||||
return getServerType() == ServerType.STANDALONE ?
|
||||
ServerInstance.Type.STANDALONE :
|
||||
ServerInstance.Type.SERVER;
|
||||
}
|
||||
private ServerType getServerType() {
|
||||
return Optional.ofNullable(environment)
|
||||
.flatMap(env -> env.getProperty("kestra.server-type", ServerType.class))
|
||||
.orElse(ServerType.STANDALONE);
|
||||
}
|
||||
|
||||
private int getServerPort() {
|
||||
return Optional.ofNullable(environment)
|
||||
.flatMap(env -> env.getProperty("micronaut.server.port", Integer.class))
|
||||
.orElse(8080);
|
||||
}
|
||||
|
||||
private int getServerManagementPort() {
|
||||
return Optional.ofNullable(environment)
|
||||
.flatMap(env -> env.getProperty("endpoints.all.port", Integer.class))
|
||||
.orElse(8081);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,356 @@
|
||||
package io.kestra.core.server;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import io.kestra.core.contexts.KestraContext;
|
||||
import io.kestra.core.repositories.ServiceInstanceRepositoryInterface;
|
||||
import io.kestra.core.server.ServiceStateTransition.Result;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import static io.kestra.core.server.ServiceLivenessManager.OnStateTransitionFailureCallback.NOOP;
|
||||
|
||||
/**
|
||||
* Service responsible for managing the state of local Kestra's Services.
|
||||
* Moreover, this class periodically send state updates (a.k.a. heartbeats) to indicate service's liveness.
|
||||
*/
|
||||
public class ServiceLivenessManager extends AbstractServiceLivenessTask {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(ServiceLivenessManager.class);
|
||||
|
||||
private static final String TASK_NAME = "service-liveness-manager-task";
|
||||
private final ServiceInstanceFactory serviceInstanceFactory;
|
||||
private final ServiceInstanceRepositoryInterface serviceRepository;
|
||||
private final ReentrantLock stateLock = new ReentrantLock();
|
||||
protected final OnStateTransitionFailureCallback onStateTransitionFailureCallback;
|
||||
private final ServiceRegistry serviceRegistry;
|
||||
|
||||
private Instant lastSucceedStateUpdated;
|
||||
|
||||
public ServiceLivenessManager(final ServerConfig configuration,
|
||||
final ServiceRegistry serviceRegistry,
|
||||
final ServiceInstanceFactory serviceInstanceFactory,
|
||||
final ServiceInstanceRepositoryInterface repository,
|
||||
final OnStateTransitionFailureCallback onStateTransitionFailureCallback) {
|
||||
super(TASK_NAME, configuration);
|
||||
this.serviceRegistry = serviceRegistry;
|
||||
this.serviceInstanceFactory = serviceInstanceFactory;
|
||||
this.serviceRepository = repository;
|
||||
this.onStateTransitionFailureCallback = onStateTransitionFailureCallback;
|
||||
}
|
||||
|
||||
/**
|
||||
* Handles the given state change event.
|
||||
*
|
||||
* @param event The state change event.
|
||||
*/
|
||||
public void onServiceStateChangeEvent(final ServiceStateChangeEvent event) {
|
||||
|
||||
final Service.ServiceState newState = event.getService().getState();
|
||||
|
||||
if (newState == null) {
|
||||
return; // invalid service event.
|
||||
}
|
||||
|
||||
// Check whether the state for this service is updatable.
|
||||
// A service (e.g., Worker) is not updatable when its state has already been transitioned to
|
||||
// a completed state (e.g., NOT_RUNNING) by an external service (e.g. Executor).
|
||||
ServiceRegistry.LocalServiceState holder = serviceRegistry.get(event.getService().getType());
|
||||
if (holder != null && !holder.isStateUpdatable().get()) {
|
||||
ServiceInstance instance = holder.instance();
|
||||
log.debug(
|
||||
"[Service id={}, type={}, hostname={}] Service state is not updatable. StateChangeEvent[{}] skipped.",
|
||||
instance.id(),
|
||||
instance.type(),
|
||||
instance.server().hostname(),
|
||||
instance.state()
|
||||
);
|
||||
return;
|
||||
}
|
||||
|
||||
switch (newState) {
|
||||
case CREATED:
|
||||
onCreateState(event);
|
||||
break;
|
||||
case RUNNING, TERMINATING, TERMINATED_GRACEFULLY, TERMINATED_FORCED:
|
||||
updateServiceInstanceState(Instant.now(), event.getService(), newState, NOOP);
|
||||
break;
|
||||
default:
|
||||
log.warn("Unsupported service state: {}. Ignored.", newState);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Handles {@link Service.ServiceState#CREATED}.
|
||||
*/
|
||||
private void onCreateState(final ServiceStateChangeEvent event) {
|
||||
Service service = event.getService();
|
||||
ServiceInstance instance = serviceRepository.save(serviceInstanceFactory.newServiceInstance(
|
||||
service,
|
||||
event.properties()
|
||||
));
|
||||
this.serviceRegistry.register(new ServiceRegistry.LocalServiceState(service, instance));
|
||||
log.info("[Service id={}, type='{}', hostname='{}'] Connected.",
|
||||
instance.id(),
|
||||
instance.type(),
|
||||
instance.server().hostname()
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
**/
|
||||
@Override
|
||||
protected Duration getScheduleInterval() {
|
||||
return serverConfig.liveness().heartbeatInterval();
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
**/
|
||||
@Override
|
||||
protected void onSchedule(final Instant now) {
|
||||
|
||||
if (serviceRegistry.isEmpty()) {
|
||||
log.trace("No service registered yet. Skip service state update.");
|
||||
return;
|
||||
}
|
||||
|
||||
// Try to update the state of each service.
|
||||
serviceRegistry.all().stream()
|
||||
.filter(localServiceState -> localServiceState.isStateUpdatable().get())
|
||||
.forEach(localServiceState -> {
|
||||
final long start = System.currentTimeMillis();
|
||||
final ServiceInstance instance = localServiceState.instance();
|
||||
final Service service = localServiceState.service();
|
||||
|
||||
// Execute the before hook.
|
||||
if (!beforeScheduledStateUpdate(now, service, instance)) {
|
||||
return;
|
||||
}
|
||||
|
||||
// Execute state update for current service (i.e., heartbeat).
|
||||
updateServiceInstanceState(now, service, instance.state(), onStateTransitionFailureCallback);
|
||||
log.trace("[Service id={}, type={}, hostname='{}'] Completed scheduled state update: '{}' ({}ms).",
|
||||
instance.id(),
|
||||
instance.type(),
|
||||
instance.server().hostname(),
|
||||
instance.state(),
|
||||
System.currentTimeMillis() - start
|
||||
);
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* This method can be overridden to provide a hook before executing a scheduled service state update.
|
||||
*
|
||||
* @return {@code true} for continuing scheduled update.
|
||||
*/
|
||||
protected boolean beforeScheduledStateUpdate(final Instant now,
|
||||
final Service service,
|
||||
final ServiceInstance instance) {
|
||||
return true; // noop
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the Instant of the last time the state was successfully updated.
|
||||
*
|
||||
* @return the {@link Instant}.
|
||||
*/
|
||||
protected Instant lastSucceedStateUpdated() {
|
||||
return lastSucceedStateUpdated;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Updates the state of the local worker instance.
|
||||
*
|
||||
* @param newState the new worker state.
|
||||
* @param onStateChangeError the callback to invoke if the state cannot be changed.
|
||||
*/
|
||||
protected ServiceInstance updateServiceInstanceState(final Instant now,
|
||||
final Service service,
|
||||
final Service.ServiceState newState,
|
||||
final OnStateTransitionFailureCallback onStateChangeError) {
|
||||
|
||||
// Pre-check the state transition validation with the known local state.
|
||||
ServiceInstance localInstance = localService(service).instance();
|
||||
if (!localInstance.state().isValidTransition(newState)) {
|
||||
log.warn("Failed to transition service [id={}, type={}, hostname={}] from {} to {}. Cause: {}.",
|
||||
localInstance.id(),
|
||||
localInstance.type(),
|
||||
localInstance.server().hostname(),
|
||||
localInstance.state(),
|
||||
newState,
|
||||
"Invalid transition"
|
||||
);
|
||||
mayDisableStateUpdate(service, localInstance);
|
||||
return localInstance;
|
||||
}
|
||||
|
||||
// Ensure only one thread can update any instance at a time.
|
||||
stateLock.lock();
|
||||
// Optional callback to be executed at the end.
|
||||
Runnable returnCallback = null;
|
||||
try {
|
||||
localInstance = localService(service).instance();
|
||||
ServiceStateTransition.Response response = serviceRepository.mayTransitionServiceTo(localInstance, newState);
|
||||
|
||||
ServiceInstance remoteInstance = response.instance();
|
||||
|
||||
boolean isStateTransitionSucceed = response.is(Result.SUCCEED);
|
||||
|
||||
if (response.is(Result.ABORTED)) {
|
||||
// Force state transition due to inconsistent state; remote state does not exist (yet).
|
||||
remoteInstance = serviceRepository.save(localInstance.updateState(newState, now));
|
||||
isStateTransitionSucceed = true;
|
||||
}
|
||||
|
||||
if (response.is(Result.FAILED)) {
|
||||
if (remoteInstance.seqId() < localInstance.seqId()) {
|
||||
// Force state transition due to inconsistent state; remote state is not up-to-date.
|
||||
remoteInstance = serviceRepository.save(localInstance.updateState(newState, now));
|
||||
isStateTransitionSucceed = true;
|
||||
} else {
|
||||
mayDisableStateUpdate(service, remoteInstance);
|
||||
|
||||
// Register the OnStateTransitionFailureCallback
|
||||
final ServiceInstance instance = remoteInstance;
|
||||
returnCallback = () -> {
|
||||
Optional<ServiceInstance> result = onStateChangeError.execute(now, service, instance, isLivenessEnabled());
|
||||
if (result.isPresent()) {
|
||||
// Optionally recover from state-transition failure
|
||||
this.serviceRegistry.register(localService(service).with(serviceRepository.save(result.get())));
|
||||
this.lastSucceedStateUpdated = now;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
if (isStateTransitionSucceed) {
|
||||
this.lastSucceedStateUpdated = now;
|
||||
}
|
||||
// Update the local instance
|
||||
this.serviceRegistry.register(localService(service).with(remoteInstance));
|
||||
} catch (Exception e) {
|
||||
log.error("[Service id={}, type='{}', hostname='{}'] Failed to update state. Error: {}",
|
||||
localInstance.id(),
|
||||
localInstance.type(),
|
||||
localInstance.server().hostname(),
|
||||
e.getMessage()
|
||||
);
|
||||
} finally {
|
||||
stateLock.unlock();
|
||||
// Because the callback may trigger a new thread that will update
|
||||
// the service instance we must ensure that we run it after calling unlock.
|
||||
if (returnCallback != null) {
|
||||
returnCallback.run();
|
||||
}
|
||||
}
|
||||
return localService(service).instance();
|
||||
}
|
||||
|
||||
private void mayDisableStateUpdate(final Service service, final ServiceInstance instance) {
|
||||
Service.ServiceState actualState = instance.state();
|
||||
if (actualState.hasCompletedTermination()) {
|
||||
log.error(
|
||||
"[Service id={}, type={}, hostname={}] Termination already completed ({}). " +
|
||||
"This error may occur if the service has already been evicted by Kestra due to a prior error.",
|
||||
instance.id(),
|
||||
instance.type(),
|
||||
instance.server().hostname(),
|
||||
actualState
|
||||
);
|
||||
// Mark the service has not updatable to prevent any unnecessary state transition issues.
|
||||
localService(service).isStateUpdatable().set(false);
|
||||
}
|
||||
}
|
||||
|
||||
private ServiceRegistry.LocalServiceState localService(final Service service) {
|
||||
return serviceRegistry.get(service.getType());
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Callback to be invoked when a state transition failed.
|
||||
*/
|
||||
@FunctionalInterface
|
||||
public interface OnStateTransitionFailureCallback {
|
||||
|
||||
OnStateTransitionFailureCallback NOOP = (now, service, instance, isLivenessEnabled) -> Optional.empty();
|
||||
|
||||
/**
|
||||
* The callback method.
|
||||
*
|
||||
* @param service the service.
|
||||
* @param instance the service instance.
|
||||
*
|
||||
* @return an optional {@link ServiceInstance} that be used to force a state transition.
|
||||
*/
|
||||
Optional<ServiceInstance> execute(Instant now,
|
||||
Service service,
|
||||
ServiceInstance instance,
|
||||
boolean isLivenessEnabled);
|
||||
}
|
||||
|
||||
public static final class DefaultStateTransitionFailureCallback implements OnStateTransitionFailureCallback {
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
**/
|
||||
@Override
|
||||
public Optional<ServiceInstance> execute(final Instant now,
|
||||
final Service service,
|
||||
final ServiceInstance instance,
|
||||
final boolean isLivenessEnabled) {
|
||||
// Never shutdown STANDALONE server or WEB_SERVER service.
|
||||
if (instance.server().type().equals(ServerInstance.Type.STANDALONE) ||
|
||||
instance.is(Service.ServiceType.WEBSERVER)) {
|
||||
// Force the RUNNING state.
|
||||
return Optional.of(instance.updateState(Service.ServiceState.RUNNING, now, null));
|
||||
}
|
||||
|
||||
if (isLivenessEnabled || instance.is(Service.ServiceState.ERROR)) {
|
||||
log.error("[Service id={}, type={}, hostname='{}'] Terminating server.",
|
||||
instance.id(),
|
||||
instance.type(),
|
||||
instance.server().hostname()
|
||||
);
|
||||
Service.ServiceState state = instance.state();
|
||||
// Skip graceful termination if the service was already considered being NOT_RUNNING by the coordinator.
|
||||
// More especially, this handles the case where a WORKER is configured with a short gracefulTerminationPeriod
|
||||
// and the JVM was unresponsive for more than this period.
|
||||
// In this context, the worker's tasks have already been resubmitted by the executor; the worker must therefore stop immediately.
|
||||
if (state.equals(Service.ServiceState.NOT_RUNNING) || state.equals(Service.ServiceState.EMPTY)) {
|
||||
service.skipGracefulTermination(true);
|
||||
}
|
||||
KestraContext.getContext().exit(1);
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
// This should not happen, but let's log a WARN to keep a trace.
|
||||
log.warn("[Service id={}, type={}, hostname='{}'] Received unexpected state [{}] transition error [bug].",
|
||||
instance.id(),
|
||||
instance.type(),
|
||||
instance.state(),
|
||||
instance.server().hostname()
|
||||
);
|
||||
return Optional.empty();
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public List<ServiceInstance> allServiceInstances() {
|
||||
return serviceRegistry.all().stream().map(ServiceRegistry.LocalServiceState::instance).toList();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void updateServiceInstance(final Service service, final ServiceInstance instance) {
|
||||
this.serviceRegistry.register(new ServiceRegistry.LocalServiceState(service, instance));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,92 @@
|
||||
package io.kestra.core.server;
|
||||
|
||||
import io.kestra.core.utils.Await;
|
||||
import jakarta.inject.Singleton;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
/**
|
||||
* Service for registering local service states.
|
||||
*
|
||||
* @see Service
|
||||
*/
|
||||
@Singleton
|
||||
public final class ServiceRegistry {
|
||||
|
||||
private final ConcurrentHashMap<Service.ServiceType, LocalServiceState> services = new ConcurrentHashMap<>();
|
||||
|
||||
/**
|
||||
* Registers or update a {@link LocalServiceState}.
|
||||
*
|
||||
* @param service The {@link LocalServiceState}.
|
||||
*/
|
||||
public void register(final LocalServiceState service) {
|
||||
services.put(service.service.getType(), service);
|
||||
}
|
||||
|
||||
public boolean containsService(final Service.ServiceType type) {
|
||||
return services.containsKey(type);
|
||||
}
|
||||
|
||||
public Service getServiceByType(final Service.ServiceType type) {
|
||||
return services.get(type).service();
|
||||
}
|
||||
|
||||
public Service waitForServiceAndGet(final Service.ServiceType type) {
|
||||
Await.until(() -> containsService(type));
|
||||
return getServiceByType(type);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the {@link LocalServiceState} for the given service type.
|
||||
*
|
||||
* @param type The service type.
|
||||
* @return The {@link LocalServiceState} or {@code null}.
|
||||
*/
|
||||
public LocalServiceState get(final Service.ServiceType type) {
|
||||
return services.get(type);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets all the registered {@link LocalServiceState}.
|
||||
*
|
||||
* @return The list of {@link LocalServiceState}.
|
||||
*/
|
||||
public List<LocalServiceState> all() {
|
||||
return new ArrayList<>(services.values());
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks whether this registry is empty.
|
||||
*
|
||||
* @return {@code} true if no service is registered.
|
||||
*/
|
||||
public boolean isEmpty() {
|
||||
return services.isEmpty();
|
||||
}
|
||||
|
||||
/**
|
||||
* Immutable class holding the local state of a service.
|
||||
*
|
||||
* @param service The service bean.
|
||||
* @param instance The service instance.
|
||||
* @param isStateUpdatable Flag indicating whether the service's state is updatable or not.
|
||||
*/
|
||||
public record LocalServiceState(Service service,
|
||||
ServiceInstance instance,
|
||||
AtomicBoolean isStateUpdatable) {
|
||||
|
||||
public LocalServiceState(Service service,
|
||||
ServiceInstance instance) {
|
||||
this(service, instance, new AtomicBoolean(true));
|
||||
}
|
||||
|
||||
public LocalServiceState with(final ServiceInstance instance) {
|
||||
return new LocalServiceState(service, instance, isStateUpdatable);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,54 @@
|
||||
package io.kestra.core.server;
|
||||
|
||||
import io.micronaut.context.event.ApplicationEvent;
|
||||
import jakarta.annotation.Nullable;
|
||||
import jakarta.validation.constraints.NotNull;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Event fired when a Service's state is changing.
|
||||
*/
|
||||
public final class ServiceStateChangeEvent extends ApplicationEvent {
|
||||
|
||||
/**
|
||||
* Opaque map of properties associated to a service.
|
||||
*/
|
||||
private final Map<String, Object> properties;
|
||||
|
||||
/**
|
||||
* Creates a new {@link ServiceStateChangeEvent} instance.
|
||||
*
|
||||
* @param source The object on which the Event initially occurred.
|
||||
* @throws IllegalArgumentException if source is null.
|
||||
*/
|
||||
public ServiceStateChangeEvent(@NotNull final Service source) {
|
||||
this(source, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new {@link ServiceStateChangeEvent} instance.
|
||||
*
|
||||
* @param source The object on which the Event initially occurred.
|
||||
* @param properties The properties to pass the event listeners.
|
||||
* @throws IllegalArgumentException if source is null.
|
||||
*/
|
||||
public ServiceStateChangeEvent(@NotNull final Service source,
|
||||
@Nullable final Map<String, Object> properties) {
|
||||
super(source);
|
||||
this.properties = properties;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the properties attached to the service.
|
||||
* @return a map of key/value pairs.
|
||||
*/
|
||||
public Map<String, Object> properties() {
|
||||
return properties;
|
||||
}
|
||||
|
||||
public Service getService() {
|
||||
return (Service) getSource();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,101 @@
|
||||
package io.kestra.core.server;
|
||||
|
||||
import jakarta.annotation.Nullable;
|
||||
import jakarta.validation.constraints.NotNull;
|
||||
import org.apache.commons.lang3.tuple.ImmutablePair;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.Optional;
|
||||
|
||||
public final class ServiceStateTransition {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(ServiceStateTransition.class);
|
||||
|
||||
/**
|
||||
* Helpers method to get a convenient response from a service state transition.
|
||||
*
|
||||
* @param initial the initial or local {@link ServiceInstance}.
|
||||
* @param newState The new service state.
|
||||
* @param result The service transition result. An {@link Optional} of {@link ImmutablePair} holding the old (left),
|
||||
* and new {@link ServiceInstance} or {@code null} if transition failed (right).
|
||||
* Otherwise, an {@link Optional#empty()} if the no service can be found.
|
||||
* @return an optional {@link Response}.
|
||||
*/
|
||||
public static Response logTransitionAndGetResponse(@NotNull final ServiceInstance initial,
|
||||
@NotNull final Service.ServiceState newState,
|
||||
@Nullable final ImmutablePair<ServiceInstance, ServiceInstance> result) {
|
||||
if (result == null) {
|
||||
LOG.debug("Failed to transition service [id={}, type={}, hostname={}] to {}. Cause: {}",
|
||||
initial.id(),
|
||||
initial.type(),
|
||||
initial.server().hostname(),
|
||||
newState,
|
||||
"Invalid service."
|
||||
);
|
||||
return new Response(Result.ABORTED);
|
||||
}
|
||||
|
||||
final ServiceInstance oldInstance = result.getLeft();
|
||||
final ServiceInstance newInstance = result.getRight();
|
||||
|
||||
if (newInstance == null) {
|
||||
LOG.warn("Failed to transition service [id={}, type={}, hostname={}] from {} to {}. Cause: {}.",
|
||||
initial.id(),
|
||||
initial.type(),
|
||||
initial.server().hostname(),
|
||||
oldInstance.state(),
|
||||
newState,
|
||||
"Invalid transition"
|
||||
);
|
||||
return new Response(Result.FAILED, oldInstance);
|
||||
}
|
||||
|
||||
// Logs if the state was changed, otherwise this method called for heartbeat purpose.
|
||||
if (!oldInstance.state().equals(newInstance.state())) {
|
||||
LOG.info("Service [id={}, type={}, hostname={}] transition from {} to {}.",
|
||||
initial.id(),
|
||||
initial.type(),
|
||||
initial.server().hostname(),
|
||||
oldInstance.state(),
|
||||
newInstance.state()
|
||||
);
|
||||
}
|
||||
return new Response(Result.SUCCEED, newInstance);
|
||||
}
|
||||
|
||||
/**
|
||||
* Wraps a service instance and a transition result.
|
||||
*
|
||||
* @param instance The service.
|
||||
* @param result The transition result.
|
||||
*/
|
||||
public record Response(Result result, @Nullable ServiceInstance instance) {
|
||||
|
||||
public Response(Result result) {
|
||||
this(result, null);
|
||||
}
|
||||
|
||||
public boolean is(final Result result) {
|
||||
return this.result.equals(result);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Represents the result of a service state transition.
|
||||
*/
|
||||
public enum Result {
|
||||
/**
|
||||
* State transition succeed.
|
||||
*/
|
||||
SUCCEED,
|
||||
/**
|
||||
* State transition failed due to invalid state transition.
|
||||
*/
|
||||
FAILED,
|
||||
/**
|
||||
* State transition cannot be executed; service does not exist.
|
||||
*/
|
||||
ABORTED,
|
||||
}
|
||||
}
|
||||
22
core/src/main/java/io/kestra/core/utils/Network.java
Normal file
22
core/src/main/java/io/kestra/core/utils/Network.java
Normal file
@@ -0,0 +1,22 @@
|
||||
package io.kestra.core.utils;
|
||||
|
||||
import java.net.InetAddress;
|
||||
import java.net.UnknownHostException;
|
||||
|
||||
public final class Network {
|
||||
|
||||
private static final String HOSTNAME;
|
||||
|
||||
static {
|
||||
try {
|
||||
HOSTNAME = InetAddress.getLocalHost().getHostName();
|
||||
} catch (UnknownHostException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public static String localHostname() {
|
||||
return HOSTNAME;
|
||||
}
|
||||
private Network() {}
|
||||
}
|
||||
@@ -0,0 +1,25 @@
|
||||
package io.kestra.core.server;
|
||||
|
||||
import io.micronaut.test.extensions.junit5.annotation.MicronautTest;
|
||||
import jakarta.inject.Inject;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.time.Duration;
|
||||
|
||||
@MicronautTest
|
||||
class ServerConfigTest {
|
||||
|
||||
@Inject
|
||||
ServerConfig config;
|
||||
|
||||
@Test
|
||||
void test() {
|
||||
Assertions.assertNotNull(config);
|
||||
Assertions.assertEquals(config.liveness().interval(), Duration.ofSeconds(5));
|
||||
Assertions.assertNotNull(config.liveness().initialDelay());
|
||||
Assertions.assertNotNull(config.liveness().timeout());
|
||||
Assertions.assertNotNull(config.liveness().heartbeatInterval());
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,86 @@
|
||||
package io.kestra.core.server;
|
||||
|
||||
import io.kestra.core.utils.IdUtils;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
import java.util.List;
|
||||
|
||||
class ServiceInstanceTest {
|
||||
|
||||
public static final ServerConfig CONFIG = new ServerConfig(Duration.ZERO,
|
||||
new ServerConfig.Liveness(
|
||||
true,
|
||||
Duration.ZERO,
|
||||
Duration.ofSeconds(10), // timeout
|
||||
Duration.ZERO,
|
||||
Duration.ZERO
|
||||
)
|
||||
);
|
||||
|
||||
@Test
|
||||
void shouldGetFalseForRunningAndNotTimeout() {
|
||||
// Given
|
||||
Instant now = Instant.now();
|
||||
ServiceInstance instance = new ServiceInstance(
|
||||
IdUtils.create(),
|
||||
Service.ServiceType.WORKER,
|
||||
Service.ServiceState.RUNNING,
|
||||
null,
|
||||
now.minus(Duration.ofSeconds(5)),
|
||||
now.minus(Duration.ofSeconds(5)),
|
||||
null,
|
||||
CONFIG,
|
||||
null
|
||||
);
|
||||
|
||||
// When - Then
|
||||
Assertions.assertFalse(instance.isSessionTimeoutElapsed(now));
|
||||
}
|
||||
|
||||
@Test
|
||||
void shouldGetTrueForRunningAndTimeout() {
|
||||
// Given
|
||||
Instant now = Instant.now();
|
||||
ServiceInstance instance = new ServiceInstance(
|
||||
IdUtils.create(),
|
||||
Service.ServiceType.WORKER,
|
||||
Service.ServiceState.RUNNING,
|
||||
null,
|
||||
now.minus(Duration.ofSeconds(20)),
|
||||
now.minus(Duration.ofSeconds(20)),
|
||||
null,
|
||||
CONFIG,
|
||||
null
|
||||
);
|
||||
|
||||
// When - Then
|
||||
Assertions.assertTrue(instance.isSessionTimeoutElapsed(now));
|
||||
}
|
||||
|
||||
@Test
|
||||
void shouldUpdateGivenReason() {
|
||||
// Given
|
||||
Instant now = Instant.now();
|
||||
ServiceInstance instance = new ServiceInstance(
|
||||
IdUtils.create(),
|
||||
Service.ServiceType.WORKER,
|
||||
Service.ServiceState.RUNNING,
|
||||
null,
|
||||
now,
|
||||
now,
|
||||
List.of(),
|
||||
CONFIG,
|
||||
null
|
||||
);
|
||||
|
||||
// When
|
||||
ServiceInstance result = instance.updateState(Service.ServiceState.DISCONNECTED, now, "Disconnected");
|
||||
|
||||
// Then
|
||||
Assertions.assertNotEquals(instance, result);
|
||||
Assertions.assertEquals(result.events(), List.of(new ServiceInstance.TimestampedEvent(now, "Disconnected")));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,203 @@
|
||||
package io.kestra.core.server;
|
||||
|
||||
import io.kestra.core.repositories.ServiceInstanceRepositoryInterface;
|
||||
import io.kestra.core.utils.IdUtils;
|
||||
import io.kestra.core.utils.Network;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.mockito.ArgumentCaptor;
|
||||
import org.mockito.Captor;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.junit.jupiter.MockitoExtension;
|
||||
import org.mockito.junit.jupiter.MockitoSettings;
|
||||
import org.mockito.quality.Strictness;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
import java.time.temporal.ChronoUnit;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static io.kestra.core.server.ServiceStateTransition.Result.ABORTED;
|
||||
import static io.kestra.core.server.ServiceStateTransition.Result.FAILED;
|
||||
import static io.kestra.core.server.ServiceStateTransition.Result.SUCCEED;
|
||||
|
||||
@ExtendWith({MockitoExtension.class})
|
||||
@MockitoSettings(strictness = Strictness.LENIENT)
|
||||
public class ServiceLivenessManagerTest {
|
||||
|
||||
private static final Duration DEFAULT_DURATION = Duration.ofSeconds(5);
|
||||
|
||||
@Mock
|
||||
public ServiceInstanceRepositoryInterface repository;
|
||||
|
||||
@Captor
|
||||
ArgumentCaptor<ServiceInstance> workerInstanceCaptor;
|
||||
|
||||
private ServiceLivenessManager serviceLivenessManager;
|
||||
|
||||
@Mock
|
||||
private ServiceLivenessManager.OnStateTransitionFailureCallback onStateTransitionFailureCallback;
|
||||
|
||||
@BeforeEach
|
||||
void beforeEach() {
|
||||
ServerConfig config = new ServerConfig(Duration.ZERO,
|
||||
new ServerConfig.Liveness(
|
||||
true,
|
||||
Duration.ZERO,
|
||||
DEFAULT_DURATION, // timeout
|
||||
DEFAULT_DURATION,
|
||||
DEFAULT_DURATION
|
||||
)
|
||||
);
|
||||
|
||||
this.serviceLivenessManager = new ServiceLivenessManager(
|
||||
config,
|
||||
new ServiceRegistry(),
|
||||
new ServiceInstanceFactory(config, null, null),
|
||||
repository,
|
||||
onStateTransitionFailureCallback
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
void shouldSaveWorkerInstanceOnRunningStateChange() {
|
||||
// Given
|
||||
Service service = newServiceForState(Service.ServiceState.CREATED);
|
||||
ServiceInstance instance = serviceInstanceFor(service);
|
||||
final ServiceStateChangeEvent event = new ServiceStateChangeEvent(service);
|
||||
Mockito.when(repository.save(Mockito.any(ServiceInstance.class))).thenReturn(instance);
|
||||
|
||||
// When
|
||||
serviceLivenessManager.onServiceStateChangeEvent(event);
|
||||
|
||||
// Then
|
||||
Mockito.verify(repository, Mockito.only()).save(workerInstanceCaptor.capture());
|
||||
|
||||
ServiceInstance value = workerInstanceCaptor.getValue();
|
||||
Assertions.assertEquals(Service.ServiceState.CREATED, value.state());
|
||||
Assertions.assertEquals(instance, serviceLivenessManager.allServiceInstances().get(0));
|
||||
}
|
||||
|
||||
@Test
|
||||
void shouldUpdateStateOnScheduleForSucceedTransition() {
|
||||
// Given
|
||||
Service running = newServiceForState(Service.ServiceState.RUNNING);
|
||||
serviceLivenessManager.updateServiceInstance(running, serviceInstanceFor(running));
|
||||
|
||||
Service terminating = newServiceForState(Service.ServiceState.TERMINATING);
|
||||
ServiceInstance instance = serviceInstanceFor(terminating);
|
||||
final ServiceStateTransition.Response response = new ServiceStateTransition.Response(
|
||||
SUCCEED,
|
||||
instance
|
||||
);
|
||||
|
||||
// mock the state transition result
|
||||
Mockito
|
||||
.when(repository.mayTransitionServiceTo(Mockito.any(ServiceInstance.class), Mockito.any(Service.ServiceState.class)))
|
||||
.thenReturn(response);
|
||||
|
||||
// When
|
||||
serviceLivenessManager.onSchedule(Instant.now());
|
||||
|
||||
// Then
|
||||
Assertions.assertEquals(instance, serviceLivenessManager.allServiceInstances().get(0));
|
||||
Mockito.verify(onStateTransitionFailureCallback, Mockito.never())
|
||||
.execute(Mockito.any(), Mockito.any(), Mockito.any(), Mockito.eq(true));
|
||||
}
|
||||
|
||||
@Test
|
||||
void shouldRunOnStateTransitionFailureForFailedTransition() {
|
||||
// Given
|
||||
Service running = newServiceForState(Service.ServiceState.RUNNING);
|
||||
serviceLivenessManager.updateServiceInstance(running, serviceInstanceFor(running));
|
||||
|
||||
Service disconnecting = newServiceForState(Service.ServiceState.TERMINATING);
|
||||
ServiceInstance instance = serviceInstanceFor(disconnecting);
|
||||
final ServiceStateTransition.Response response = new ServiceStateTransition.Response(
|
||||
FAILED,
|
||||
instance
|
||||
);
|
||||
|
||||
// mock the state transition result
|
||||
Mockito
|
||||
.when(repository.mayTransitionServiceTo(Mockito.any(ServiceInstance.class), Mockito.any(Service.ServiceState.class)))
|
||||
.thenReturn(response);
|
||||
|
||||
// When
|
||||
serviceLivenessManager.onSchedule(Instant.now());
|
||||
|
||||
// Then
|
||||
Assertions.assertEquals(instance, serviceLivenessManager.allServiceInstances().get(0));
|
||||
Mockito.verify(onStateTransitionFailureCallback, Mockito.only())
|
||||
.execute(Mockito.any(), Mockito.any(), Mockito.any(), Mockito.eq(true));
|
||||
}
|
||||
|
||||
@Test
|
||||
void shouldNotRunOnStateTransitionFailureForAbortedTransition() {
|
||||
// Given
|
||||
Service running = newServiceForState(Service.ServiceState.RUNNING);
|
||||
serviceLivenessManager.updateServiceInstance(running, serviceInstanceFor(running));
|
||||
|
||||
// mock the state transition result
|
||||
Mockito
|
||||
.when(repository.mayTransitionServiceTo(Mockito.any(ServiceInstance.class), Mockito.any(Service.ServiceState.class)))
|
||||
.thenReturn(new ServiceStateTransition.Response(ABORTED));
|
||||
|
||||
// When
|
||||
serviceLivenessManager.onSchedule(Instant.now());
|
||||
|
||||
// Then
|
||||
Mockito.verify(onStateTransitionFailureCallback, Mockito.never())
|
||||
.execute(Mockito.any(), Mockito.any(), Mockito.any(), Mockito.eq(true));
|
||||
}
|
||||
|
||||
public static Service newServiceForState(final Service.ServiceState state) {
|
||||
return new Service() {
|
||||
@Override
|
||||
public String getId() {
|
||||
return IdUtils.create();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServiceType getType() {
|
||||
return ServiceType.WORKER;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServiceState getState() {
|
||||
return state;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
public static ServiceInstance serviceInstanceFor(final Service service) {
|
||||
ServerConfig config = new ServerConfig(Duration.ZERO,
|
||||
new ServerConfig.Liveness(
|
||||
true,
|
||||
Duration.ZERO,
|
||||
Duration.ofSeconds(10), // timeout
|
||||
Duration.ZERO,
|
||||
Duration.ZERO
|
||||
)
|
||||
);
|
||||
return new ServiceInstance(
|
||||
service.getId(),
|
||||
service.getType(),
|
||||
service.getState(),
|
||||
new ServerInstance(
|
||||
ServerInstance.Type.SERVER,
|
||||
"N/A",
|
||||
Network.localHostname(), Map.of()
|
||||
),
|
||||
Instant.now().truncatedTo(ChronoUnit.MILLIS),
|
||||
Instant.now().truncatedTo(ChronoUnit.MILLIS),
|
||||
List.of(),
|
||||
config,
|
||||
Map.of()
|
||||
);
|
||||
}
|
||||
}
|
||||
69
core/src/test/java/io/kestra/core/server/ServiceTest.java
Normal file
69
core/src/test/java/io/kestra/core/server/ServiceTest.java
Normal file
@@ -0,0 +1,69 @@
|
||||
package io.kestra.core.server;
|
||||
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
class ServiceTest {
|
||||
|
||||
@Test
|
||||
void shouldReturnValidTransitionForRunning() {
|
||||
List<Service.ServiceState> states = List.of(
|
||||
Service.ServiceState.RUNNING,
|
||||
Service.ServiceState.DISCONNECTED,
|
||||
Service.ServiceState.TERMINATING
|
||||
);
|
||||
states.forEach(status -> Assertions.assertTrue(Service.ServiceState.RUNNING.isValidTransition(status)));
|
||||
}
|
||||
|
||||
@Test
|
||||
void shouldReturnValidTransitionForDisconnected() {
|
||||
List<Service.ServiceState> states = List.of(
|
||||
Service.ServiceState.DISCONNECTED,
|
||||
Service.ServiceState.TERMINATING,
|
||||
Service.ServiceState.NOT_RUNNING
|
||||
);
|
||||
states.forEach(status -> Assertions.assertTrue(Service.ServiceState.DISCONNECTED.isValidTransition(status)));
|
||||
}
|
||||
|
||||
@Test
|
||||
void shouldReturnValidTransitionForPendingShutdown() {
|
||||
List<Service.ServiceState> states = List.of(
|
||||
Service.ServiceState.TERMINATING,
|
||||
Service.ServiceState.TERMINATED_FORCED,
|
||||
Service.ServiceState.TERMINATED_GRACEFULLY
|
||||
);
|
||||
states.forEach(status -> Assertions.assertTrue(Service.ServiceState.TERMINATING.isValidTransition(status)));
|
||||
}
|
||||
|
||||
@Test
|
||||
void shouldReturnValidTransitionForForcedShutdown() {
|
||||
List<Service.ServiceState> states = List.of(
|
||||
Service.ServiceState.TERMINATED_FORCED,
|
||||
Service.ServiceState.NOT_RUNNING
|
||||
);
|
||||
states.forEach(status -> Assertions.assertTrue(Service.ServiceState.TERMINATED_FORCED.isValidTransition(status)));
|
||||
}
|
||||
|
||||
@Test
|
||||
void shouldReturnValidTransitionForGracefulShutdown() {
|
||||
List<Service.ServiceState> states = List.of(
|
||||
Service.ServiceState.TERMINATED_GRACEFULLY,
|
||||
Service.ServiceState.NOT_RUNNING
|
||||
);
|
||||
states.forEach(status -> Assertions.assertTrue(Service.ServiceState.TERMINATED_GRACEFULLY.isValidTransition(status)));
|
||||
}
|
||||
|
||||
@Test
|
||||
void shouldReturnValidTransitionForNotRunning() {
|
||||
List<Service.ServiceState> states = List.of(Service.ServiceState.EMPTY);
|
||||
states.forEach(status -> Assertions.assertTrue(Service.ServiceState.NOT_RUNNING.isValidTransition(status)));
|
||||
}
|
||||
|
||||
@Test
|
||||
void shouldReturnTrueForDisconnectedOrPendingShutDown() {
|
||||
Assertions.assertTrue(Service.ServiceState.DISCONNECTED.isDisconnectedOrTerminating());
|
||||
Assertions.assertTrue(Service.ServiceState.TERMINATING.isDisconnectedOrTerminating());
|
||||
}
|
||||
}
|
||||
@@ -1,12 +1,12 @@
|
||||
package io.kestra.core.services;
|
||||
|
||||
import io.kestra.core.utils.IdUtils;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import io.kestra.core.runners.WorkerInstance;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
@@ -24,7 +24,7 @@ class WorkerInstanceServiceTest {
|
||||
);
|
||||
|
||||
assertThat(workerInstance.size(), is(1));
|
||||
assertThat(workerInstance.get(0).getWorkerUuid().toString(), is(first.getWorkerUuid().toString()));
|
||||
assertThat(workerInstance.get(0).getWorkerUuid(), is(first.getWorkerUuid()));
|
||||
assertThat(workerInstance.get(0).getPartitions().size(), is(0));
|
||||
}
|
||||
|
||||
@@ -38,7 +38,7 @@ class WorkerInstanceServiceTest {
|
||||
);
|
||||
|
||||
assertThat(workerInstance.size(), is(1));
|
||||
assertThat(workerInstance.get(0).getWorkerUuid().toString(), is(first.getWorkerUuid().toString()));
|
||||
assertThat(workerInstance.get(0).getWorkerUuid(), is(first.getWorkerUuid()));
|
||||
assertThat(workerInstance.get(0).getWorkerGroup(), is("workerGroup"));
|
||||
assertThat(workerInstance.get(0).getPartitions().size(), is(0));
|
||||
}
|
||||
@@ -55,11 +55,11 @@ class WorkerInstanceServiceTest {
|
||||
|
||||
List<WorkerInstance> workerInstance = WorkerInstanceService.removeEvictedPartitions(
|
||||
workerInstanceStream,
|
||||
workerInstance(Arrays.asList(1, 2, 3, 4, 5, 6), willBeUpdated.getWorkerUuid())
|
||||
workerInstance(Arrays.asList(1, 2, 3, 4, 5, 6), willBeUpdated.getWorkerUuid(), null)
|
||||
);
|
||||
|
||||
assertThat(workerInstance.size(), is(1));
|
||||
assertThat(workerInstance.get(0).getWorkerUuid().toString(), is(first.getWorkerUuid().toString()));
|
||||
assertThat(workerInstance.get(0).getWorkerUuid(), is(first.getWorkerUuid()));
|
||||
assertThat(workerInstance.get(0).getPartitions().size(), is(0));
|
||||
}
|
||||
|
||||
@@ -79,7 +79,7 @@ class WorkerInstanceServiceTest {
|
||||
);
|
||||
|
||||
assertThat(workerInstance.size(), is(1));
|
||||
assertThat(workerInstance.get(0).getWorkerUuid().toString(), is(first.getWorkerUuid().toString()));
|
||||
assertThat(workerInstance.get(0).getWorkerUuid(), is(first.getWorkerUuid()));
|
||||
assertThat(workerInstance.get(0).getWorkerGroup(), is("workerGroup"));
|
||||
assertThat(workerInstance.get(0).getPartitions().size(), is(0));
|
||||
}
|
||||
@@ -97,15 +97,15 @@ class WorkerInstanceServiceTest {
|
||||
|
||||
List<WorkerInstance> workerInstance = WorkerInstanceService.removeEvictedPartitions(
|
||||
workerInstanceStream,
|
||||
workerInstance(Arrays.asList(2, 3, 4), UUID.randomUUID())
|
||||
workerInstance(Arrays.asList(2, 3, 4))
|
||||
);
|
||||
|
||||
assertThat(workerInstance.size(), is(2));
|
||||
assertThat(workerInstance.get(0).getWorkerUuid().toString(), is(first.getWorkerUuid().toString()));
|
||||
assertThat(workerInstance.get(0).getWorkerUuid(), is(first.getWorkerUuid()));
|
||||
assertThat(workerInstance.get(0).getPartitions().size(), is(1));
|
||||
assertThat(workerInstance.get(0).getPartitions(), contains(1));
|
||||
|
||||
assertThat(workerInstance.get(1).getWorkerUuid().toString(), is(second.getWorkerUuid().toString()));
|
||||
assertThat(workerInstance.get(1).getWorkerUuid(), is(second.getWorkerUuid()));
|
||||
assertThat(workerInstance.get(1).getPartitions().size(), is(2));
|
||||
assertThat(workerInstance.get(1).getPartitions(), contains(5, 6));
|
||||
}
|
||||
@@ -122,16 +122,16 @@ class WorkerInstanceServiceTest {
|
||||
|
||||
List<WorkerInstance> workerInstance = WorkerInstanceService.removeEvictedPartitions(
|
||||
workerInstanceStream,
|
||||
workerInstance(Arrays.asList(2, 3, 4), UUID.randomUUID(), "workerGroup")
|
||||
workerInstance(Arrays.asList(2, 3, 4), "workerGroup")
|
||||
);
|
||||
|
||||
assertThat(workerInstance.size(), is(2));
|
||||
assertThat(workerInstance.get(0).getWorkerUuid().toString(), is(first.getWorkerUuid().toString()));
|
||||
assertThat(workerInstance.get(0).getWorkerUuid(), is(first.getWorkerUuid()));
|
||||
assertThat(workerInstance.get(0).getWorkerGroup(), is("workerGroup"));
|
||||
assertThat(workerInstance.get(0).getPartitions().size(), is(1));
|
||||
assertThat(workerInstance.get(0).getPartitions(), contains(1));
|
||||
|
||||
assertThat(workerInstance.get(1).getWorkerUuid().toString(), is(second.getWorkerUuid().toString()));
|
||||
assertThat(workerInstance.get(1).getWorkerUuid(), is(second.getWorkerUuid()));
|
||||
assertThat(workerInstance.get(1).getWorkerGroup(), is("workerGroup"));
|
||||
assertThat(workerInstance.get(1).getPartitions().size(), is(2));
|
||||
assertThat(workerInstance.get(1).getPartitions(), contains(5, 6));
|
||||
@@ -155,8 +155,7 @@ class WorkerInstanceServiceTest {
|
||||
WorkerInstance first = WorkerInstance
|
||||
.builder()
|
||||
.partitions(null)
|
||||
.workerUuid(UUID.randomUUID())
|
||||
.hostname("unit-test")
|
||||
.workerUuid(IdUtils.create())
|
||||
.build();
|
||||
|
||||
List<WorkerInstance> workerInstance = WorkerInstanceService.removeEvictedPartitions(
|
||||
@@ -175,16 +174,11 @@ class WorkerInstanceServiceTest {
|
||||
return workerInstance(partitions, null, workerGroup);
|
||||
}
|
||||
|
||||
private WorkerInstance workerInstance(List<Integer> partitions, UUID uuid) {
|
||||
return workerInstance(partitions, uuid, null);
|
||||
}
|
||||
|
||||
private static WorkerInstance workerInstance(List<Integer> partitions, UUID uuid, String workerGroup) {
|
||||
private static WorkerInstance workerInstance(List<Integer> partitions, String uuid, String workerGroup) {
|
||||
return WorkerInstance
|
||||
.builder()
|
||||
.partitions(new ArrayList<>(partitions))
|
||||
.workerUuid(uuid == null ? UUID.randomUUID() : uuid)
|
||||
.hostname("unit-test")
|
||||
.workerUuid(uuid == null ? IdUtils.create() : uuid)
|
||||
.workerGroup(workerGroup)
|
||||
.build();
|
||||
}
|
||||
|
||||
@@ -0,0 +1,16 @@
|
||||
package io.kestra.repository.h2;
|
||||
|
||||
import io.kestra.core.server.ServiceInstance;
|
||||
import io.kestra.jdbc.repository.AbstractJdbcServiceInstanceRepository;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
|
||||
@Singleton
|
||||
@H2RepositoryEnabled
|
||||
public class H2ServiceInstanceRepository extends AbstractJdbcServiceInstanceRepository {
|
||||
@Inject
|
||||
public H2ServiceInstanceRepository(ApplicationContext applicationContext) {
|
||||
super(new H2Repository<>(ServiceInstance.class, applicationContext));
|
||||
}
|
||||
}
|
||||
@@ -1,16 +0,0 @@
|
||||
package io.kestra.repository.h2;
|
||||
|
||||
import io.kestra.core.runners.WorkerInstance;
|
||||
import io.kestra.jdbc.repository.AbstractJdbcWorkerInstanceRepository;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
|
||||
@Singleton
|
||||
@H2RepositoryEnabled
|
||||
public class H2WorkerInstanceRepository extends AbstractJdbcWorkerInstanceRepository {
|
||||
@Inject
|
||||
public H2WorkerInstanceRepository(ApplicationContext applicationContext) {
|
||||
super(new H2Repository<>(WorkerInstance.class, applicationContext));
|
||||
}
|
||||
}
|
||||
@@ -28,11 +28,6 @@ public class H2WorkerJobQueue implements WorkerJobQueueInterface {
|
||||
jdbcworkerjobQueueService.pause();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cleanup() {
|
||||
jdbcworkerjobQueueService.cleanup();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
jdbcworkerjobQueueService.close();
|
||||
|
||||
@@ -0,0 +1,11 @@
|
||||
/* ----------------------- ServiceInstance ----------------------- */
|
||||
CREATE TABLE IF NOT EXISTS service_instance
|
||||
(
|
||||
"key" VARCHAR(250) NOT NULL PRIMARY KEY,
|
||||
"value" VARCHAR(1000) NOT NULL,
|
||||
"service_id" VARCHAR(36) NOT NULL GENERATED ALWAYS AS (JQ_STRING("value", '.id')),
|
||||
"service_type" VARCHAR(36) NOT NULL GENERATED ALWAYS AS (JQ_STRING("value", '.type')),
|
||||
"state" VARCHAR(36) NOT NULL GENERATED ALWAYS AS (JQ_STRING("value", '.state')),
|
||||
"created_at" TIMESTAMP NOT NULL GENERATED ALWAYS AS (PARSEDATETIME(JQ_STRING("value", '.createdAt'), 'yyyy-MM-dd''T''HH:mm:ss.SSSXXX')),
|
||||
"updated_at" TIMESTAMP NOT NULL GENERATED ALWAYS AS (PARSEDATETIME(JQ_STRING("value", '.updatedAt'), 'yyyy-MM-dd''T''HH:mm:ss.SSSXXX'))
|
||||
);
|
||||
@@ -0,0 +1,6 @@
|
||||
package io.kestra.repository.h2;
|
||||
|
||||
import io.kestra.jdbc.repository.AbstractJdbcServiceInstanceRepositoryTest;
|
||||
|
||||
public class H2ServiceInstanceRepositoryTest extends AbstractJdbcServiceInstanceRepositoryTest {
|
||||
}
|
||||
@@ -1,6 +0,0 @@
|
||||
package io.kestra.repository.h2;
|
||||
|
||||
import io.kestra.jdbc.repository.AbstractJdbcWorkerInstanceRepositoryTest;
|
||||
|
||||
public class H2WorkerInstanceRepositoryTest extends AbstractJdbcWorkerInstanceRepositoryTest {
|
||||
}
|
||||
@@ -1,7 +0,0 @@
|
||||
package io.kestra.runner.h2;
|
||||
|
||||
import io.kestra.jdbc.runner.JdbcHeartbeatTest;
|
||||
|
||||
class H2HeartbeatTest extends JdbcHeartbeatTest {
|
||||
|
||||
}
|
||||
@@ -0,0 +1,7 @@
|
||||
package io.kestra.runner.h2;
|
||||
|
||||
import io.kestra.jdbc.runner.JdbcServiceLivenessCoordinatorTest;
|
||||
|
||||
class H2ServiceLivenessHandlerTest extends JdbcServiceLivenessCoordinatorTest {
|
||||
|
||||
}
|
||||
9
jdbc-h2/src/test/resources/application-liveness.yml
Normal file
9
jdbc-h2/src/test/resources/application-liveness.yml
Normal file
@@ -0,0 +1,9 @@
|
||||
kestra:
|
||||
server:
|
||||
terminationGracePeriod: 0s
|
||||
liveness:
|
||||
enabled: true
|
||||
interval: 1s
|
||||
timeout: 5s
|
||||
initialDelay: 0s
|
||||
heartbeatInterval: 1s
|
||||
@@ -16,9 +16,6 @@ flyway:
|
||||
|
||||
kestra:
|
||||
server-type: STANDALONE
|
||||
heartbeat:
|
||||
frequency: 10s
|
||||
heartbeat-missed: 3
|
||||
queue:
|
||||
type: h2
|
||||
repository:
|
||||
@@ -27,7 +24,6 @@ kestra:
|
||||
type: local
|
||||
local:
|
||||
base-path: /tmp/unittest
|
||||
|
||||
jdbc:
|
||||
tables:
|
||||
queues:
|
||||
@@ -68,9 +64,9 @@ kestra:
|
||||
flowtopologies:
|
||||
table: "flow_topologies"
|
||||
cls: io.kestra.core.models.topologies.FlowTopology
|
||||
workerinstance:
|
||||
cls: io.kestra.core.runners.WorkerInstance
|
||||
table: "worker_instance"
|
||||
serviceinstance:
|
||||
cls: io.kestra.core.server.ServiceInstance
|
||||
table: "service_instance"
|
||||
workerjobrunning:
|
||||
cls: io.kestra.core.runners.WorkerJobRunning
|
||||
table: "worker_job_running"
|
||||
@@ -82,3 +78,6 @@ kestra:
|
||||
min-poll-interval: 10ms
|
||||
max-poll-interval: 100ms
|
||||
poll-switch-interval: 5s
|
||||
worker:
|
||||
liveness:
|
||||
enabled: false
|
||||
@@ -0,0 +1,17 @@
|
||||
package io.kestra.repository.mysql;
|
||||
|
||||
import io.kestra.core.server.ServiceInstance;
|
||||
import io.kestra.jdbc.repository.AbstractJdbcServiceInstanceRepository;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
|
||||
@Singleton
|
||||
@MysqlRepositoryEnabled
|
||||
public class MysqlServiceInstanceRepository extends AbstractJdbcServiceInstanceRepository {
|
||||
@Inject
|
||||
public MysqlServiceInstanceRepository(ApplicationContext applicationContext) {
|
||||
super(new MysqlRepository<>(ServiceInstance.class, applicationContext));
|
||||
}
|
||||
|
||||
}
|
||||
@@ -1,16 +0,0 @@
|
||||
package io.kestra.repository.mysql;
|
||||
|
||||
import io.kestra.core.runners.WorkerInstance;
|
||||
import io.kestra.jdbc.repository.AbstractJdbcWorkerInstanceRepository;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
|
||||
@Singleton
|
||||
@MysqlRepositoryEnabled
|
||||
public class MysqlWorkerInstanceRepository extends AbstractJdbcWorkerInstanceRepository {
|
||||
@Inject
|
||||
public MysqlWorkerInstanceRepository(ApplicationContext applicationContext) {
|
||||
super(new MysqlRepository<>(WorkerInstance.class, applicationContext));
|
||||
}
|
||||
}
|
||||
@@ -28,11 +28,6 @@ public class MysqlWorkerJobQueue implements WorkerJobQueueInterface {
|
||||
jdbcworkerjobQueueService.pause();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cleanup() {
|
||||
jdbcworkerjobQueueService.cleanup();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
jdbcworkerjobQueueService.close();
|
||||
|
||||
@@ -0,0 +1,11 @@
|
||||
/* ----------------------- service_instance ----------------------- */
|
||||
CREATE TABLE IF NOT EXISTS service_instance
|
||||
(
|
||||
`key` VARCHAR(250) NOT NULL PRIMARY KEY,
|
||||
`value` JSON NOT NULL,
|
||||
`service_id` VARCHAR(36) GENERATED ALWAYS AS (`value` ->> '$.id') STORED NOT NULL,
|
||||
`service_type` VARCHAR(36) GENERATED ALWAYS AS (`value` ->> '$.type') STORED NOT NULL,
|
||||
`state` VARCHAR(36) GENERATED ALWAYS AS (`value` ->> '$.state') STORED NOT NULL,
|
||||
`created_at` DATETIME(6) GENERATED ALWAYS AS (STR_TO_DATE(value ->> '$.createdAt' , '%Y-%m-%dT%H:%i:%s.%fZ')) STORED NOT NULL,
|
||||
`updated_at` DATETIME(6) GENERATED ALWAYS AS (STR_TO_DATE(value ->> '$.updatedAt' , '%Y-%m-%dT%H:%i:%s.%fZ')) STORED NOT NULL
|
||||
);
|
||||
@@ -0,0 +1,7 @@
|
||||
package io.kestra.repository.mysql;
|
||||
|
||||
import io.kestra.jdbc.repository.AbstractJdbcServiceInstanceRepositoryTest;
|
||||
|
||||
public class MysqlServiceInstanceRepositoryTest extends AbstractJdbcServiceInstanceRepositoryTest {
|
||||
|
||||
}
|
||||
@@ -1,7 +0,0 @@
|
||||
package io.kestra.repository.mysql;
|
||||
|
||||
import io.kestra.jdbc.repository.AbstractJdbcWorkerInstanceRepositoryTest;
|
||||
|
||||
public class MysqlWorkerInstanceRepositoryTest extends AbstractJdbcWorkerInstanceRepositoryTest {
|
||||
|
||||
}
|
||||
@@ -1,7 +0,0 @@
|
||||
package io.kestra.runner.mysql;
|
||||
|
||||
import io.kestra.jdbc.runner.JdbcHeartbeatTest;
|
||||
|
||||
class MysqlHeartbeatTest extends JdbcHeartbeatTest {
|
||||
|
||||
}
|
||||
@@ -0,0 +1,7 @@
|
||||
package io.kestra.runner.mysql;
|
||||
|
||||
import io.kestra.jdbc.runner.JdbcServiceLivenessCoordinatorTest;
|
||||
|
||||
class MysqlServiceLivenessHandlerTest extends JdbcServiceLivenessCoordinatorTest {
|
||||
|
||||
}
|
||||
9
jdbc-mysql/src/test/resources/application-liveness.yml
Normal file
9
jdbc-mysql/src/test/resources/application-liveness.yml
Normal file
@@ -0,0 +1,9 @@
|
||||
kestra:
|
||||
server:
|
||||
terminationGracePeriod: 0s
|
||||
liveness:
|
||||
enabled: true
|
||||
interval: 1s
|
||||
timeout: 5s
|
||||
initialDelay: 0s
|
||||
heartbeatInterval: 1s
|
||||
@@ -25,10 +25,6 @@ kestra:
|
||||
type: local
|
||||
local:
|
||||
base-path: /tmp/unittest
|
||||
heartbeat:
|
||||
frequency: 10s
|
||||
heartbeat-missed: 3
|
||||
|
||||
jdbc:
|
||||
tables:
|
||||
queues:
|
||||
@@ -69,9 +65,9 @@ kestra:
|
||||
flowtopologies:
|
||||
table: "flow_topologies"
|
||||
cls: io.kestra.core.models.topologies.FlowTopology
|
||||
workerinstance:
|
||||
cls: io.kestra.core.runners.WorkerInstance
|
||||
table: "worker_instance"
|
||||
serviceinstance:
|
||||
cls: io.kestra.core.server.ServiceInstance
|
||||
table: "service_instance"
|
||||
workerjobrunning:
|
||||
cls: io.kestra.core.runners.WorkerJobRunning
|
||||
table: "worker_job_running"
|
||||
@@ -83,3 +79,6 @@ kestra:
|
||||
min-poll-interval: 10ms
|
||||
max-poll-interval: 100ms
|
||||
poll-switch-interval: 5s
|
||||
worker:
|
||||
liveness:
|
||||
enabled: false
|
||||
@@ -0,0 +1,16 @@
|
||||
package io.kestra.repository.postgres;
|
||||
|
||||
import io.kestra.core.server.ServiceInstance;
|
||||
import io.kestra.jdbc.repository.AbstractJdbcServiceInstanceRepository;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
|
||||
@Singleton
|
||||
@PostgresRepositoryEnabled
|
||||
public class PostgresServiceInstanceRepository extends AbstractJdbcServiceInstanceRepository {
|
||||
@Inject
|
||||
public PostgresServiceInstanceRepository(ApplicationContext applicationContext) {
|
||||
super(new PostgresRepository<>(ServiceInstance.class, applicationContext));
|
||||
}
|
||||
}
|
||||
@@ -1,16 +0,0 @@
|
||||
package io.kestra.repository.postgres;
|
||||
|
||||
import io.kestra.core.runners.WorkerInstance;
|
||||
import io.kestra.jdbc.repository.AbstractJdbcWorkerInstanceRepository;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
|
||||
@Singleton
|
||||
@PostgresRepositoryEnabled
|
||||
public class PostgresWorkerInstanceRepository extends AbstractJdbcWorkerInstanceRepository {
|
||||
@Inject
|
||||
public PostgresWorkerInstanceRepository(ApplicationContext applicationContext) {
|
||||
super(new PostgresRepository<>(WorkerInstance.class, applicationContext));
|
||||
}
|
||||
}
|
||||
@@ -28,11 +28,6 @@ public class PostgresWorkerJobQueue implements WorkerJobQueueInterface {
|
||||
jdbcworkerjobQueueService.pause();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cleanup() {
|
||||
jdbcworkerjobQueueService.cleanup();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
jdbcworkerjobQueueService.close();
|
||||
|
||||
@@ -0,0 +1,11 @@
|
||||
/* ----------------------- service_instance ----------------------- */
|
||||
CREATE TABLE IF NOT EXISTS service_instance
|
||||
(
|
||||
key VARCHAR(250) NOT NULL PRIMARY KEY,
|
||||
value JSONB NOT NULL,
|
||||
service_id VARCHAR(36) NOT NULL GENERATED ALWAYS AS (value ->> 'id') STORED,
|
||||
service_type VARCHAR(36) NOT NULL GENERATED ALWAYS AS (value ->> 'type') STORED,
|
||||
state VARCHAR(36) NOT NULL GENERATED ALWAYS AS (value ->> 'state') STORED,
|
||||
created_at TIMESTAMPTZ NOT NULL GENERATED ALWAYS AS (PARSE_ISO8601_DATETIME(value ->> 'createdAt')) STORED,
|
||||
updated_at TIMESTAMPTZ NOT NULL GENERATED ALWAYS AS (PARSE_ISO8601_DATETIME(value ->> 'updatedAt')) STORED
|
||||
);
|
||||
@@ -0,0 +1,7 @@
|
||||
package io.kestra.repository.postgres;
|
||||
|
||||
import io.kestra.jdbc.repository.AbstractJdbcServiceInstanceRepositoryTest;
|
||||
|
||||
public class PostgresServiceInstanceRepositoryTest extends AbstractJdbcServiceInstanceRepositoryTest {
|
||||
|
||||
}
|
||||
@@ -1,7 +0,0 @@
|
||||
package io.kestra.repository.postgres;
|
||||
|
||||
import io.kestra.jdbc.repository.AbstractJdbcWorkerInstanceRepositoryTest;
|
||||
|
||||
public class PostgresWorkerInstanceRepositoryTest extends AbstractJdbcWorkerInstanceRepositoryTest {
|
||||
|
||||
}
|
||||
@@ -1,7 +0,0 @@
|
||||
package io.kestra.runner.postgres;
|
||||
|
||||
import io.kestra.jdbc.runner.JdbcHeartbeatTest;
|
||||
|
||||
class PostgresHeartbeatTest extends JdbcHeartbeatTest {
|
||||
|
||||
}
|
||||
@@ -0,0 +1,7 @@
|
||||
package io.kestra.runner.postgres;
|
||||
|
||||
import io.kestra.jdbc.runner.JdbcServiceLivenessCoordinatorTest;
|
||||
|
||||
class PostgresServiceLivenessHandlerTest extends JdbcServiceLivenessCoordinatorTest {
|
||||
|
||||
}
|
||||
@@ -0,0 +1,9 @@
|
||||
kestra:
|
||||
server:
|
||||
terminationGracePeriod: 0s
|
||||
liveness:
|
||||
enabled: true
|
||||
interval: 1s
|
||||
timeout: 5s
|
||||
initialDelay: 0s
|
||||
heartbeatInterval: 1s
|
||||
@@ -26,10 +26,6 @@ kestra:
|
||||
type: local
|
||||
local:
|
||||
base-path: /tmp/unittest
|
||||
heartbeat:
|
||||
frequency: 10s
|
||||
heartbeat-missed: 3
|
||||
|
||||
jdbc:
|
||||
tables:
|
||||
queues:
|
||||
@@ -70,9 +66,9 @@ kestra:
|
||||
flowtopologies:
|
||||
table: "flow_topologies"
|
||||
cls: io.kestra.core.models.topologies.FlowTopology
|
||||
workerinstance:
|
||||
cls: io.kestra.core.runners.WorkerInstance
|
||||
table: "worker_instance"
|
||||
serviceinstance:
|
||||
cls: io.kestra.core.server.ServiceInstance
|
||||
table: "service_instance"
|
||||
workerjobrunning:
|
||||
cls: io.kestra.core.runners.WorkerJobRunning
|
||||
table: "worker_job_running"
|
||||
@@ -84,7 +80,9 @@ kestra:
|
||||
min-poll-interval: 10ms
|
||||
max-poll-interval: 100ms
|
||||
poll-switch-interval: 5s
|
||||
|
||||
tasks:
|
||||
subflow:
|
||||
allow-parameter-outputs: true
|
||||
server:
|
||||
liveness:
|
||||
enabled: false
|
||||
@@ -3,11 +3,11 @@ package io.kestra.jdbc;
|
||||
import io.kestra.core.exceptions.DeserializationException;
|
||||
import io.kestra.core.queues.QueueFactoryInterface;
|
||||
import io.kestra.core.queues.QueueInterface;
|
||||
import io.kestra.core.repositories.WorkerInstanceRepositoryInterface;
|
||||
import io.kestra.core.runners.*;
|
||||
import io.kestra.core.server.Service;
|
||||
import io.kestra.core.server.ServiceRegistry;
|
||||
import io.kestra.core.utils.Either;
|
||||
import io.kestra.jdbc.repository.AbstractJdbcWorkerJobRunningRepository;
|
||||
import io.kestra.jdbc.runner.JdbcHeartbeat;
|
||||
import io.kestra.jdbc.runner.JdbcQueue;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import io.micronaut.inject.qualifiers.Qualifiers;
|
||||
@@ -20,9 +20,10 @@ import java.util.function.Consumer;
|
||||
@Slf4j
|
||||
public class JdbcWorkerJobQueueService {
|
||||
private final JdbcQueue<WorkerJob> workerTaskQueue;
|
||||
private final JdbcHeartbeat jdbcHeartbeat;
|
||||
private final AbstractJdbcWorkerJobRunningRepository jdbcWorkerJobRunningRepository;
|
||||
private final WorkerInstanceRepositoryInterface workerInstanceRepository;
|
||||
|
||||
private final ServiceRegistry serviceRegistry;
|
||||
|
||||
private Runnable queueStop;
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@@ -31,15 +32,21 @@ public class JdbcWorkerJobQueueService {
|
||||
QueueInterface.class,
|
||||
Qualifiers.byName(QueueFactoryInterface.WORKERJOB_NAMED)
|
||||
);
|
||||
this.jdbcHeartbeat = applicationContext.getBean(JdbcHeartbeat.class);
|
||||
this.serviceRegistry = applicationContext.getBean(ServiceRegistry.class);
|
||||
this.jdbcWorkerJobRunningRepository = applicationContext.getBean(AbstractJdbcWorkerJobRunningRepository.class);
|
||||
this.workerInstanceRepository = applicationContext.getBean(WorkerInstanceRepositoryInterface.class);
|
||||
}
|
||||
|
||||
public Runnable receive(String consumerGroup, Class<?> queueType, Consumer<Either<WorkerJob, DeserializationException>> consumer) {
|
||||
|
||||
this.queueStop = workerTaskQueue.receiveTransaction(consumerGroup, queueType, (dslContext, eithers) -> {
|
||||
WorkerInstance workerInstance = jdbcHeartbeat.get();
|
||||
|
||||
Worker worker = (Worker) serviceRegistry.waitForServiceAndGet(Service.ServiceType.WORKER);
|
||||
|
||||
final WorkerInstance workerInstance = WorkerInstance
|
||||
.builder()
|
||||
.workerUuid(worker.getId())
|
||||
.workerGroup(worker.getWorkerGroup())
|
||||
.build();
|
||||
|
||||
eithers.forEach(either -> {
|
||||
if (either.isRight()) {
|
||||
@@ -92,14 +99,6 @@ public class JdbcWorkerJobQueueService {
|
||||
}
|
||||
}
|
||||
|
||||
public void cleanup() {
|
||||
if (jdbcHeartbeat.get() != null) {
|
||||
synchronized (this) {
|
||||
workerInstanceRepository.delete(jdbcHeartbeat.get());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void close() {
|
||||
this.stopQueue();
|
||||
}
|
||||
|
||||
@@ -0,0 +1,264 @@
|
||||
package io.kestra.jdbc.repository;
|
||||
|
||||
import io.kestra.core.repositories.ServiceInstanceRepositoryInterface;
|
||||
import io.kestra.core.server.Service;
|
||||
import io.kestra.core.server.ServiceInstance;
|
||||
import io.kestra.core.server.ServiceStateTransition;
|
||||
import jakarta.inject.Singleton;
|
||||
import lombok.Getter;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.commons.lang3.tuple.ImmutablePair;
|
||||
import org.jooq.Configuration;
|
||||
import org.jooq.DSLContext;
|
||||
import org.jooq.DeleteResultStep;
|
||||
import org.jooq.Field;
|
||||
import org.jooq.Record;
|
||||
import org.jooq.Record1;
|
||||
import org.jooq.SelectConditionStep;
|
||||
import org.jooq.Table;
|
||||
import org.jooq.TransactionalCallable;
|
||||
import org.jooq.TransactionalRunnable;
|
||||
|
||||
import java.time.Instant;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
||||
import static org.jooq.impl.DSL.using;
|
||||
|
||||
@Singleton
|
||||
@Getter
|
||||
@Slf4j
|
||||
public abstract class AbstractJdbcServiceInstanceRepository extends AbstractJdbcRepository implements ServiceInstanceRepositoryInterface {
|
||||
|
||||
private static final Field<Object> STATE = field("state");
|
||||
private static final Field<Object> VALUE = field("value");
|
||||
private static final Field<Instant> UPDATED_AT = field("updated_at", Instant.class);
|
||||
private static final Field<Instant> CREATED_AT = field("created_at", Instant.class);
|
||||
private static final Field<Object> SERVICE_ID = field("service_id");
|
||||
|
||||
protected io.kestra.jdbc.AbstractJdbcRepository<ServiceInstance> jdbcRepository;
|
||||
|
||||
public AbstractJdbcServiceInstanceRepository(final io.kestra.jdbc.AbstractJdbcRepository<ServiceInstance> jdbcRepository) {
|
||||
this.jdbcRepository = jdbcRepository;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
**/
|
||||
@Override
|
||||
public Optional<ServiceInstance> findById(final String id) {
|
||||
return jdbcRepository.getDslContextWrapper().transactionResult(
|
||||
configuration -> findById(id, configuration, false)
|
||||
);
|
||||
}
|
||||
|
||||
public Optional<ServiceInstance> findById(final String id,
|
||||
final Configuration configuration,
|
||||
final boolean isForUpdate) {
|
||||
|
||||
SelectConditionStep<Record1<Object>> query = using(configuration)
|
||||
.select(VALUE)
|
||||
.from(table())
|
||||
.where(SERVICE_ID.eq(id));
|
||||
|
||||
return isForUpdate ?
|
||||
this.jdbcRepository.fetchOne(query.forUpdate()) :
|
||||
this.jdbcRepository.fetchOne(query);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
**/
|
||||
@Override
|
||||
public List<ServiceInstance> findAllInstancesInState(final Service.ServiceState state) {
|
||||
return this.jdbcRepository.getDslContextWrapper()
|
||||
.transactionResult(configuration -> {
|
||||
SelectConditionStep<Record1<Object>> query = using(configuration)
|
||||
.select(VALUE)
|
||||
.from(table())
|
||||
.where(STATE.eq(state.name()));
|
||||
return this.jdbcRepository.fetch(query);
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
**/
|
||||
@Override
|
||||
public List<ServiceInstance> findAllInstancesInStates(final List<Service.ServiceState> states) {
|
||||
return this.jdbcRepository.getDslContextWrapper()
|
||||
.transactionResult(configuration -> {
|
||||
SelectConditionStep<Record1<Object>> query = using(configuration)
|
||||
.select(VALUE)
|
||||
.from(table())
|
||||
.where(STATE.in(states.stream().map(Enum::name).toList()));
|
||||
return this.jdbcRepository.fetch(query);
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Finds all service instances which are NOT {@link Service.ServiceState#RUNNING}.
|
||||
*
|
||||
* @return the list of {@link ServiceInstance}.
|
||||
*/
|
||||
public List<ServiceInstance> findAllNonRunningInstances() {
|
||||
return jdbcRepository.getDslContextWrapper().transactionResult(
|
||||
configuration -> findAllNonRunningInstances(configuration, false)
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Finds all service instances which are NOT {@link Service.ServiceState#RUNNING}.
|
||||
*
|
||||
* @return the list of {@link ServiceInstance}.
|
||||
*/
|
||||
public List<ServiceInstance> findAllNonRunningInstances(final Configuration configuration,
|
||||
final boolean isForUpdate) {
|
||||
SelectConditionStep<Record1<Object>> query = using(configuration)
|
||||
.select(VALUE)
|
||||
.from(table())
|
||||
.where(STATE.notIn(Service.ServiceState.CREATED.name(), Service.ServiceState.RUNNING.name()));
|
||||
|
||||
return isForUpdate ?
|
||||
this.jdbcRepository.fetch(query.forUpdate()) :
|
||||
this.jdbcRepository.fetch(query);
|
||||
}
|
||||
|
||||
/**
|
||||
* Finds all service instances which are {@link Service.ServiceState#NOT_RUNNING}.
|
||||
*
|
||||
* @return the list of {@link ServiceInstance}.
|
||||
*/
|
||||
public List<ServiceInstance> findAllInstancesInNotRunningState() {
|
||||
return jdbcRepository.getDslContextWrapper().transactionResult(
|
||||
configuration -> findAllInstancesInNotRunningState(configuration, false)
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Finds all service instances which are {@link Service.ServiceState#NOT_RUNNING}.
|
||||
*
|
||||
* @return the list of {@link ServiceInstance}.
|
||||
*/
|
||||
public List<ServiceInstance> findAllInstancesInNotRunningState(final Configuration configuration,
|
||||
final boolean isForUpdate) {
|
||||
SelectConditionStep<Record1<Object>> query = using(configuration)
|
||||
.select(VALUE)
|
||||
.from(table())
|
||||
.where(STATE.eq(Service.ServiceState.NOT_RUNNING.name()));
|
||||
|
||||
return isForUpdate ?
|
||||
this.jdbcRepository.fetch(query.forUpdate()) :
|
||||
this.jdbcRepository.fetch(query);
|
||||
}
|
||||
|
||||
public void transaction(final TransactionalRunnable runnable) {
|
||||
this.jdbcRepository
|
||||
.getDslContextWrapper()
|
||||
.transaction(runnable);
|
||||
}
|
||||
|
||||
public <T> T transactionResult(final TransactionalCallable<T> runnable) {
|
||||
return this.jdbcRepository
|
||||
.getDslContextWrapper()
|
||||
.transactionResult(runnable);
|
||||
}
|
||||
|
||||
public void delete(DSLContext context, ServiceInstance instance) {
|
||||
this.jdbcRepository.delete(context, instance);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
**/
|
||||
@Override
|
||||
public void delete(final ServiceInstance instance) {
|
||||
this.jdbcRepository.delete(instance);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
**/
|
||||
@Override
|
||||
public ServiceInstance save(final ServiceInstance instance) {
|
||||
this.jdbcRepository.persist(instance, this.jdbcRepository.persistFields(instance));
|
||||
return instance;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
**/
|
||||
@Override
|
||||
public List<ServiceInstance> findAll() {
|
||||
return this.jdbcRepository
|
||||
.getDslContextWrapper()
|
||||
.transactionResult(configuration -> this.jdbcRepository.fetch(
|
||||
using(configuration).select(VALUE).from(table()))
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
**/
|
||||
@Override
|
||||
public ServiceStateTransition.Response mayTransitionServiceTo(final ServiceInstance instance,
|
||||
final Service.ServiceState newState,
|
||||
final String reason) {
|
||||
return transactionResult(configuration -> mayTransitServiceTo(configuration, instance, newState, reason));
|
||||
}
|
||||
|
||||
/**
|
||||
* Attempt to transit the status of a given service to given new status.
|
||||
* This method may not update the service if the transition is not valid.
|
||||
*
|
||||
* @param instance the service instance.
|
||||
* @param newState the new state of the service.
|
||||
* @return an optional of the {@link ServiceInstance} or {@link Optional#empty()} if the service is not running.
|
||||
*/
|
||||
public ServiceStateTransition.Response mayTransitServiceTo(final Configuration configuration,
|
||||
final ServiceInstance instance,
|
||||
final Service.ServiceState newState,
|
||||
final String reason) {
|
||||
ImmutablePair<ServiceInstance, ServiceInstance> result = mayUpdateStatusById(
|
||||
configuration,
|
||||
instance.id(),
|
||||
newState,
|
||||
reason
|
||||
);
|
||||
return ServiceStateTransition.logTransitionAndGetResponse(instance, newState, result);
|
||||
}
|
||||
|
||||
/**
|
||||
* Attempt to transit the status of a given service to given new status.
|
||||
* This method may not update the service if the transition is not valid.
|
||||
*
|
||||
* @param id the service's uid.
|
||||
* @param newState the new state of the service.
|
||||
* @return an {@link Optional} of {@link ImmutablePair} holding the old (left), and new {@link ServiceInstance} or {@code null} if transition failed (right).
|
||||
* Otherwise, an {@link Optional#empty()} if the no service can be found.
|
||||
*/
|
||||
private ImmutablePair<ServiceInstance, ServiceInstance> mayUpdateStatusById(final Configuration configuration,
|
||||
final String id,
|
||||
final Service.ServiceState newState,
|
||||
final String reason) {
|
||||
// Find the ServiceInstance to be updated
|
||||
Optional<ServiceInstance> optional = findById(id, configuration, true);
|
||||
|
||||
// Check whether service was found.
|
||||
if (optional.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
// Check whether the status transition is valid before saving.
|
||||
ServiceInstance serviceInstance = optional.get();
|
||||
if (serviceInstance.state().isValidTransition(newState)) {
|
||||
ServiceInstance updated = serviceInstance.updateState(newState, Instant.now(), reason);
|
||||
return new ImmutablePair<>(serviceInstance, save(updated));
|
||||
}
|
||||
return new ImmutablePair<>(serviceInstance, null);
|
||||
}
|
||||
|
||||
private Table<Record> table() {
|
||||
return this.jdbcRepository.getTable();
|
||||
}
|
||||
}
|
||||
@@ -1,169 +0,0 @@
|
||||
package io.kestra.jdbc.repository;
|
||||
|
||||
import io.kestra.core.repositories.WorkerInstanceRepositoryInterface;
|
||||
import io.kestra.core.runners.ServerInstance;
|
||||
import io.kestra.core.runners.WorkerInstance;
|
||||
import io.micronaut.context.annotation.Value;
|
||||
import jakarta.inject.Singleton;
|
||||
import lombok.Getter;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.jooq.*;
|
||||
import org.jooq.impl.DSL;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.function.Function;
|
||||
|
||||
@Singleton
|
||||
@Getter
|
||||
@Slf4j
|
||||
public abstract class AbstractJdbcWorkerInstanceRepository extends AbstractJdbcRepository implements WorkerInstanceRepositoryInterface {
|
||||
protected io.kestra.jdbc.AbstractJdbcRepository<WorkerInstance> jdbcRepository;
|
||||
|
||||
public AbstractJdbcWorkerInstanceRepository(io.kestra.jdbc.AbstractJdbcRepository<WorkerInstance> jdbcRepository) {
|
||||
this.jdbcRepository = jdbcRepository;
|
||||
}
|
||||
|
||||
@Value("${kestra.heartbeat.frequency}")
|
||||
private Duration frequency;
|
||||
|
||||
@Value("${kestra.heartbeat.heartbeat-missed}")
|
||||
private Integer nbMissed;
|
||||
|
||||
@Override
|
||||
public Optional<WorkerInstance> findByWorkerUuid(String workerUuid) {
|
||||
return this.jdbcRepository
|
||||
.getDslContextWrapper()
|
||||
.transactionResult(configuration -> {
|
||||
SelectConditionStep<Record1<Object>> select = this.heartbeatSelectQuery(DSL.using(configuration), workerUuid);
|
||||
|
||||
return this.jdbcRepository.fetchOne(select);
|
||||
});
|
||||
}
|
||||
|
||||
public Optional<WorkerInstance> heartbeatCheckUp(String workerUuid) {
|
||||
return this.jdbcRepository
|
||||
.getDslContextWrapper()
|
||||
.transactionResult(configuration -> {
|
||||
SelectForUpdateOfStep<Record1<Object>> select =
|
||||
this.heartbeatSelectQuery(DSL.using(configuration), workerUuid)
|
||||
.forUpdate();
|
||||
|
||||
Optional<WorkerInstance> workerInstance = this.jdbcRepository.fetchOne(select);
|
||||
|
||||
if (workerInstance.isPresent()) {
|
||||
WorkerInstance updatedWorker = workerInstance.get().toBuilder().status(WorkerInstance.Status.UP).heartbeatDate(Instant.now()).build();
|
||||
this.save(updatedWorker);
|
||||
return Optional.of(updatedWorker);
|
||||
}
|
||||
return Optional.empty();
|
||||
});
|
||||
}
|
||||
|
||||
public void heartbeatStatusUpdate(String workerUuid, DSLContext context) {
|
||||
SelectForUpdateOfStep<Record1<Object>> select =
|
||||
this.heartbeatSelectQuery(context, workerUuid)
|
||||
.and(field("status").eq(WorkerInstance.Status.UP.toString()))
|
||||
// We consider a heartbeat dead if it's older than heartbeat missed times the frequency
|
||||
.and(field("heartbeat_date").lessThan(Instant.now().minusSeconds(getNbMissed() * getFrequency().getSeconds())))
|
||||
.forUpdate();
|
||||
|
||||
Optional<WorkerInstance> workerInstance = this.jdbcRepository.fetchOne(select)
|
||||
// exclude any worker running on the same server as the executor, to prevent the latter from shutting down.
|
||||
.filter(instance -> !instance.getServer().equals(ServerInstance.getInstance()));
|
||||
|
||||
workerInstance.ifPresent(heartbeat -> {
|
||||
heartbeat.setStatus(WorkerInstance.Status.DEAD);
|
||||
|
||||
log.warn("Detected non-responding worker, stated to DEAD: {}", heartbeat);
|
||||
|
||||
this.jdbcRepository.persist(heartbeat, context, this.jdbcRepository.persistFields(heartbeat));
|
||||
});
|
||||
}
|
||||
|
||||
public void heartbeatsStatusUpdate(DSLContext context) {
|
||||
this.findAllAlive(context).forEach(heartbeat -> {
|
||||
this.heartbeatStatusUpdate(heartbeat.getWorkerUuid().toString(), context);
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
public void lockedWorkersUpdate(Function<DSLContext, Void> function) {
|
||||
this.jdbcRepository
|
||||
.getDslContextWrapper()
|
||||
.transactionResult(configuration -> {
|
||||
DSLContext context = DSL.using(configuration);
|
||||
|
||||
// Update all workers status
|
||||
heartbeatsStatusUpdate(context);
|
||||
|
||||
function.apply(context);
|
||||
|
||||
return null;
|
||||
});
|
||||
}
|
||||
|
||||
public List<WorkerInstance> findAll(DSLContext context) {
|
||||
return this.jdbcRepository.fetch(this.heartbeatSelectAllQuery(context));
|
||||
}
|
||||
|
||||
public List<WorkerInstance> findAll() {
|
||||
return this.jdbcRepository
|
||||
.getDslContextWrapper()
|
||||
.transactionResult(configuration -> {
|
||||
DSLContext context = DSL.using(configuration);
|
||||
return this.jdbcRepository.fetch(this.heartbeatSelectAllQuery(context));
|
||||
});
|
||||
}
|
||||
|
||||
public List<WorkerInstance> findAllAlive(DSLContext context) {
|
||||
return this.jdbcRepository
|
||||
.getDslContextWrapper()
|
||||
.transactionResult(configuration -> this.jdbcRepository.fetch(
|
||||
this.heartbeatSelectAllQuery(context)
|
||||
.where(field("status").eq(WorkerInstance.Status.UP.toString()))
|
||||
));
|
||||
}
|
||||
|
||||
public List<WorkerInstance> findAllToDelete(DSLContext context) {
|
||||
return this.jdbcRepository
|
||||
.getDslContextWrapper()
|
||||
.transactionResult(configuration -> this.jdbcRepository.fetch(
|
||||
this.heartbeatSelectAllQuery(context)
|
||||
.where(field("status").eq(WorkerInstance.Status.DEAD.toString()))
|
||||
.and(field("heartbeat_date").lessThan(Instant.now().minusSeconds(2 * getNbMissed() * getFrequency().getSeconds())))
|
||||
));
|
||||
}
|
||||
|
||||
public void delete(DSLContext context, WorkerInstance workerInstance) {
|
||||
this.jdbcRepository.delete(context, workerInstance);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void delete(WorkerInstance workerInstance) {
|
||||
this.jdbcRepository.delete(workerInstance);
|
||||
}
|
||||
|
||||
@Override
|
||||
public WorkerInstance save(WorkerInstance workerInstance) {
|
||||
this.jdbcRepository.persist(workerInstance, this.jdbcRepository.persistFields(workerInstance));
|
||||
return workerInstance;
|
||||
}
|
||||
|
||||
private SelectConditionStep<Record1<Object>> heartbeatSelectQuery(DSLContext context, String workerUuid) {
|
||||
return context
|
||||
.select(field("value"))
|
||||
.from(this.jdbcRepository.getTable())
|
||||
.where(
|
||||
field("worker_uuid").eq(workerUuid)
|
||||
);
|
||||
}
|
||||
|
||||
private SelectJoinStep<Record1<Object>> heartbeatSelectAllQuery(DSLContext dsl) {
|
||||
return dsl.select(field("value"))
|
||||
.from(this.jdbcRepository.getTable());
|
||||
}
|
||||
|
||||
}
|
||||
@@ -4,7 +4,11 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import io.kestra.core.exceptions.DeserializationException;
|
||||
import io.kestra.core.exceptions.InternalException;
|
||||
import io.kestra.core.metrics.MetricRegistry;
|
||||
import io.kestra.core.models.executions.*;
|
||||
import io.kestra.core.models.executions.Execution;
|
||||
import io.kestra.core.models.executions.ExecutionKilled;
|
||||
import io.kestra.core.models.executions.LogEntry;
|
||||
import io.kestra.core.models.executions.TaskRun;
|
||||
import io.kestra.core.models.executions.TaskRunAttempt;
|
||||
import io.kestra.core.models.executions.statistics.ExecutionCount;
|
||||
import io.kestra.core.models.flows.Concurrency;
|
||||
import io.kestra.core.models.flows.Flow;
|
||||
@@ -17,22 +21,39 @@ import io.kestra.core.models.triggers.multipleflows.MultipleConditionStorageInte
|
||||
import io.kestra.core.queues.QueueFactoryInterface;
|
||||
import io.kestra.core.queues.QueueInterface;
|
||||
import io.kestra.core.repositories.FlowRepositoryInterface;
|
||||
import io.kestra.core.runners.DefaultFlowExecutor;
|
||||
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.*;
|
||||
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.server.Service;
|
||||
import io.kestra.core.server.ServiceStateChangeEvent;
|
||||
import io.kestra.core.services.*;
|
||||
import io.kestra.core.tasks.flows.ForEachItem;
|
||||
import io.kestra.core.tasks.flows.Template;
|
||||
import io.kestra.core.topologies.FlowTopologyService;
|
||||
import io.kestra.core.utils.Await;
|
||||
import io.kestra.core.utils.Either;
|
||||
import io.kestra.core.utils.IdUtils;
|
||||
import io.kestra.jdbc.JdbcMapper;
|
||||
import io.kestra.jdbc.repository.AbstractJdbcExecutionRepository;
|
||||
import io.kestra.jdbc.repository.AbstractJdbcFlowTopologyRepository;
|
||||
import io.kestra.jdbc.repository.AbstractJdbcWorkerInstanceRepository;
|
||||
import io.kestra.jdbc.repository.AbstractJdbcWorkerJobRunningRepository;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import io.micronaut.context.annotation.Value;
|
||||
import io.micronaut.context.event.ApplicationEventPublisher;
|
||||
import io.micronaut.transaction.exceptions.CannotCreateTransactionException;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Named;
|
||||
@@ -40,26 +61,30 @@ import jakarta.inject.Singleton;
|
||||
import lombok.SneakyThrows;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.jooq.Configuration;
|
||||
import org.slf4j.event.Level;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.time.Duration;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.*;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
@Singleton
|
||||
@JdbcRunnerEnabled
|
||||
@Slf4j
|
||||
public class JdbcExecutor implements ExecutorInterface {
|
||||
public class JdbcExecutor implements ExecutorInterface, Service {
|
||||
private static final ObjectMapper MAPPER = JdbcMapper.of();
|
||||
|
||||
private final ScheduledExecutorService scheduledDelay = Executors.newSingleThreadScheduledExecutor();
|
||||
|
||||
private final ScheduledExecutorService scheduledHeartbeat = Executors.newSingleThreadScheduledExecutor();
|
||||
|
||||
private volatile boolean isShutdown = false;
|
||||
|
||||
@Inject
|
||||
@@ -136,9 +161,6 @@ public class JdbcExecutor implements ExecutorInterface {
|
||||
@Inject
|
||||
private AbstractJdbcFlowTopologyRepository flowTopologyRepository;
|
||||
|
||||
@Inject
|
||||
private AbstractJdbcWorkerInstanceRepository workerInstanceRepository;
|
||||
|
||||
protected List<Flow> allFlows;
|
||||
|
||||
@Inject
|
||||
@@ -154,9 +176,6 @@ public class JdbcExecutor implements ExecutorInterface {
|
||||
@Inject
|
||||
private AbstractJdbcWorkerJobRunningRepository workerJobRunningRepository;
|
||||
|
||||
@Value("${kestra.heartbeat.frequency}")
|
||||
private Duration frequency;
|
||||
|
||||
@Inject
|
||||
@Named(QueueFactoryInterface.KILL_NAMED)
|
||||
protected QueueInterface<ExecutionKilled> killQueue;
|
||||
@@ -168,9 +187,20 @@ public class JdbcExecutor implements ExecutorInterface {
|
||||
@Inject
|
||||
private LogService logService;
|
||||
|
||||
@Inject
|
||||
private JdbcServiceLivenessCoordinator serviceLivenessCoordinator;
|
||||
|
||||
@Inject
|
||||
private ApplicationEventPublisher<ServiceStateChangeEvent> eventPublisher;
|
||||
|
||||
private final String id = IdUtils.create();
|
||||
private final AtomicReference<ServiceState> state = new AtomicReference<>();
|
||||
|
||||
@SneakyThrows
|
||||
@Override
|
||||
public void run() {
|
||||
setState(ServiceState.CREATED);
|
||||
serviceLivenessCoordinator.setExecutor(this);
|
||||
flowListeners.run();
|
||||
flowListeners.listen(flows -> this.allFlows = flows);
|
||||
|
||||
@@ -190,13 +220,6 @@ public class JdbcExecutor implements ExecutorInterface {
|
||||
TimeUnit.SECONDS
|
||||
);
|
||||
|
||||
ScheduledFuture<?> scheduledHeartbeatFuture = scheduledHeartbeat.scheduleAtFixedRate(
|
||||
this::workersUpdate,
|
||||
frequency.toSeconds(),
|
||||
frequency.toSeconds(),
|
||||
TimeUnit.SECONDS
|
||||
);
|
||||
|
||||
// look at exceptions on the scheduledDelay thread
|
||||
Thread scheduledDelayExceptionThread = new Thread(
|
||||
() -> {
|
||||
@@ -221,30 +244,6 @@ public class JdbcExecutor implements ExecutorInterface {
|
||||
);
|
||||
scheduledDelayExceptionThread.start();
|
||||
|
||||
// look at exceptions on the scheduledHeartbeat thread
|
||||
Thread scheduledHeartbeatExceptionThread = new Thread(
|
||||
() -> {
|
||||
Await.until(scheduledHeartbeatFuture::isDone);
|
||||
|
||||
try {
|
||||
scheduledHeartbeatFuture.get();
|
||||
} catch (ExecutionException | InterruptedException e) {
|
||||
if (e.getCause().getClass() != CannotCreateTransactionException.class) {
|
||||
log.error("Executor fatal exception in the scheduledHeartbeat thread", e);
|
||||
|
||||
try {
|
||||
close();
|
||||
applicationContext.stop();
|
||||
} catch (IOException ioe) {
|
||||
log.error("Unable to properly close the executor", ioe);
|
||||
}
|
||||
}
|
||||
}
|
||||
},
|
||||
"jdbc-heartbeat-exception-watcher"
|
||||
);
|
||||
scheduledHeartbeatExceptionThread.start();
|
||||
|
||||
flowQueue.receive(
|
||||
FlowTopology.class,
|
||||
either -> {
|
||||
@@ -278,64 +277,51 @@ public class JdbcExecutor implements ExecutorInterface {
|
||||
);
|
||||
}
|
||||
);
|
||||
|
||||
setState(ServiceState.RUNNING);
|
||||
}
|
||||
|
||||
protected void workersUpdate() {
|
||||
workerInstanceRepository.lockedWorkersUpdate(context -> {
|
||||
List<WorkerInstance> workersToDelete = workerInstanceRepository
|
||||
.findAllToDelete(context);
|
||||
List<String> workersToDeleteUuids = workersToDelete.stream().map(worker -> worker.getWorkerUuid().toString()).collect(Collectors.toList());
|
||||
|
||||
// Before deleting a worker, we resubmit all his tasks
|
||||
workerJobRunningRepository.getWorkerJobWithWorkerDead(context, workersToDeleteUuids)
|
||||
.forEach(workerJobRunning -> {
|
||||
if (workerJobRunning instanceof WorkerTaskRunning workerTaskRunning) {
|
||||
if (skipExecutionService.skipExecution(workerTaskRunning.getTaskRun().getExecutionId())) {
|
||||
// if the execution is skipped, we remove the workerTaskRunning and skip its resubmission
|
||||
log.warn("Skipping execution {}", workerTaskRunning.getTaskRun().getId());
|
||||
workerJobRunningRepository.deleteByKey(workerTaskRunning.uid());
|
||||
} else {
|
||||
workerTaskQueue.emit(WorkerTask.builder()
|
||||
.taskRun(workerTaskRunning.getTaskRun())
|
||||
.task(workerTaskRunning.getTask())
|
||||
.runContext(workerTaskRunning.getRunContext())
|
||||
.build()
|
||||
);
|
||||
|
||||
logService.logTaskRun(
|
||||
workerTaskRunning.getTaskRun(),
|
||||
log,
|
||||
Level.WARN,
|
||||
"WorkerTask is being resend"
|
||||
);
|
||||
}
|
||||
|
||||
} else if (workerJobRunning instanceof WorkerTriggerRunning workerTriggerRunning) {
|
||||
workerTaskQueue.emit(WorkerTrigger.builder()
|
||||
.trigger(workerTriggerRunning.getTrigger())
|
||||
.conditionContext(workerTriggerRunning.getConditionContext())
|
||||
.triggerContext(workerTriggerRunning.getTriggerContext())
|
||||
.build());
|
||||
|
||||
logService.logTrigger(
|
||||
workerTriggerRunning.getTriggerContext(),
|
||||
void reEmitWorkerJobsForWorkers(final Configuration configuration,
|
||||
final List<String> ids) {
|
||||
workerJobRunningRepository.getWorkerJobWithWorkerDead(configuration.dsl(), ids)
|
||||
.forEach(workerJobRunning -> {
|
||||
// WorkerTaskRunning
|
||||
if (workerJobRunning instanceof WorkerTaskRunning workerTaskRunning) {
|
||||
if (skipExecutionService.skipExecution(workerTaskRunning.getTaskRun().getExecutionId())) {
|
||||
// if the execution is skipped, we remove the workerTaskRunning and skip its resubmission
|
||||
log.warn("Skipping execution {}", workerTaskRunning.getTaskRun().getId());
|
||||
workerJobRunningRepository.deleteByKey(workerTaskRunning.uid());
|
||||
} else {
|
||||
workerTaskQueue.emit(WorkerTask.builder()
|
||||
.taskRun(workerTaskRunning.getTaskRun())
|
||||
.task(workerTaskRunning.getTask())
|
||||
.runContext(workerTaskRunning.getRunContext())
|
||||
.build()
|
||||
);
|
||||
logService.logTaskRun(
|
||||
workerTaskRunning.getTaskRun(),
|
||||
log,
|
||||
Level.WARN,
|
||||
"WorkerTrigger is being resend"
|
||||
"Re-emitting WorkerTask."
|
||||
);
|
||||
} else {
|
||||
throw new IllegalArgumentException("Object is of type " + workerJobRunning.getClass() + " which should never occurs");
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
workersToDelete.forEach(worker -> {
|
||||
log.warn("Deleted dead worker: {}", worker);
|
||||
workerInstanceRepository.delete(context, worker);
|
||||
// WorkerTriggerRunning
|
||||
if (workerJobRunning instanceof WorkerTriggerRunning workerTriggerRunning) {
|
||||
workerTaskQueue.emit(WorkerTrigger.builder()
|
||||
.trigger(workerTriggerRunning.getTrigger())
|
||||
.conditionContext(workerTriggerRunning.getConditionContext())
|
||||
.triggerContext(workerTriggerRunning.getTriggerContext())
|
||||
.build());
|
||||
|
||||
logService.logTrigger(
|
||||
workerTriggerRunning.getTriggerContext(),
|
||||
log,
|
||||
Level.WARN,
|
||||
"Re-emitting WorkerTrigger."
|
||||
);
|
||||
}
|
||||
});
|
||||
|
||||
return null;
|
||||
});
|
||||
}
|
||||
|
||||
private void executionQueue(Either<Execution, DeserializationException> either) {
|
||||
@@ -900,7 +886,7 @@ public class JdbcExecutor implements ExecutorInterface {
|
||||
private boolean deduplicateSubflowExecution(Execution execution, ExecutorState executorState, TaskRun taskRun) {
|
||||
// There can be multiple executions for the same task, so we need to deduplicated with the worker task execution iteration
|
||||
String deduplicationKey = taskRun.getId() + (taskRun.getIteration() == null ? "" : "-" + taskRun.getIteration());
|
||||
State.Type current = executorState.getSubflowExecutionDeduplication().get(deduplicationKey);
|
||||
State.Type current = executorState.getSubflowExecutionDeduplication().get(deduplicationKey);
|
||||
|
||||
if (current == taskRun.getState().getCurrent()) {
|
||||
log.trace("Duplicate SubflowExecution on execution '{}' for taskRun '{}', value '{}, taskId '{}'", execution.getId(), taskRun.getId(), taskRun.getValue(), taskRun.getTaskId());
|
||||
@@ -923,10 +909,35 @@ public class JdbcExecutor implements ExecutorInterface {
|
||||
return executor.withExecution(failedExecutionWithLog.getExecution(), "exception");
|
||||
}
|
||||
|
||||
/** {@inheritDoc} **/
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
setState(ServiceState.TERMINATING);
|
||||
isShutdown = true;
|
||||
scheduledDelay.shutdown();
|
||||
scheduledHeartbeat.shutdown();
|
||||
setState(ServiceState.TERMINATED_GRACEFULLY);
|
||||
}
|
||||
|
||||
private void setState(final ServiceState state) {
|
||||
this.state.set(state);
|
||||
eventPublisher.publishEvent(new ServiceStateChangeEvent(this));
|
||||
}
|
||||
|
||||
/** {@inheritDoc} **/
|
||||
@Override
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} **/
|
||||
@Override
|
||||
public ServiceType getType() {
|
||||
return ServiceType.EXECUTOR;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} **/
|
||||
@Override
|
||||
public ServiceState getState() {
|
||||
return state.get();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,108 +0,0 @@
|
||||
package io.kestra.jdbc.runner;
|
||||
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import io.kestra.core.models.ServerType;
|
||||
import io.kestra.core.runners.ServerInstance;
|
||||
import io.kestra.core.runners.Worker;
|
||||
import io.kestra.core.runners.WorkerInstance;
|
||||
import io.kestra.jdbc.repository.AbstractJdbcWorkerInstanceRepository;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import io.micronaut.context.annotation.Requires;
|
||||
import io.micronaut.context.annotation.Value;
|
||||
import io.micronaut.context.env.Environment;
|
||||
import io.micronaut.core.annotation.Nullable;
|
||||
import io.micronaut.scheduling.annotation.Scheduled;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
|
||||
import java.net.InetAddress;
|
||||
import java.net.UnknownHostException;
|
||||
import java.util.UUID;
|
||||
|
||||
@Singleton
|
||||
@JdbcRunnerEnabled
|
||||
@Requires(property = "kestra.server-type", pattern = "(WORKER|STANDALONE)")
|
||||
@Slf4j
|
||||
public class JdbcHeartbeat {
|
||||
private static final String HOSTNAME;
|
||||
|
||||
@Nullable
|
||||
@Value("${kestra.server-type}")
|
||||
protected ServerType serverType;
|
||||
|
||||
static {
|
||||
try {
|
||||
HOSTNAME = InetAddress.getLocalHost().getHostName();
|
||||
} catch (UnknownHostException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Inject
|
||||
AbstractJdbcWorkerInstanceRepository workerInstanceRepository;
|
||||
|
||||
private volatile WorkerInstance workerInstance;
|
||||
|
||||
private final ApplicationContext applicationContext;
|
||||
|
||||
private ServerInstance serverInstance;
|
||||
|
||||
public JdbcHeartbeat(ApplicationContext applicationContext) {
|
||||
this.applicationContext = applicationContext;
|
||||
this.serverInstance = ServerInstance.getInstance();
|
||||
}
|
||||
|
||||
private void registerWorkerInstance(Worker worker) {
|
||||
synchronized (this) {
|
||||
if (workerInstance == null) {
|
||||
this.workerInstance = WorkerInstance.builder()
|
||||
.workerUuid(UUID.randomUUID())
|
||||
.hostname(HOSTNAME)
|
||||
.port(applicationContext.getEnvironment().getProperty("micronaut.server.port", Integer.class).orElse(8080))
|
||||
.managementPort(applicationContext.getEnvironment().getProperty("endpoints.all.port", Integer.class).orElse(8081))
|
||||
.workerGroup(worker.getWorkerGroup())
|
||||
.server(serverInstance)
|
||||
.build();
|
||||
|
||||
if (log.isDebugEnabled()) {
|
||||
log.debug("Registered WorkerInstance of: {}", workerInstance.getWorkerUuid());
|
||||
}
|
||||
|
||||
this.workerInstanceRepository.save(workerInstance);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Scheduled(fixedDelay = "${kestra.heartbeat.frequency}")
|
||||
public void updateHeartbeat() {
|
||||
if (applicationContext.containsBean(Worker.class) && !applicationContext.getEnvironment().getActiveNames().contains(Environment.TEST)) {
|
||||
if (workerInstance == null) {
|
||||
registerWorkerInstance(applicationContext.getBean(Worker.class));
|
||||
}
|
||||
|
||||
if (log.isTraceEnabled()) {
|
||||
log.error("Heartbeat of: {}", workerInstance.getWorkerUuid());
|
||||
}
|
||||
|
||||
if (workerInstanceRepository.heartbeatCheckUp(workerInstance.getWorkerUuid().toString()).isEmpty()) {
|
||||
log.error("heartbeatCheckUp failed, unable to find current instance '{}', Shutting down now!", workerInstance.getWorkerUuid());
|
||||
Runtime.getRuntime().exit(1);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
void setServerInstance(final ServerInstance serverInstance) {
|
||||
this.serverInstance = serverInstance;
|
||||
}
|
||||
|
||||
public WorkerInstance get() {
|
||||
if (workerInstance == null) {
|
||||
registerWorkerInstance(applicationContext.getBean(Worker.class));
|
||||
}
|
||||
|
||||
return workerInstance;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,135 @@
|
||||
package io.kestra.jdbc.runner;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import io.kestra.core.server.AbstractServiceLivenessCoordinator;
|
||||
import io.kestra.core.server.ServerConfig;
|
||||
import io.kestra.core.server.Service;
|
||||
import io.kestra.core.server.Service.ServiceState;
|
||||
import io.kestra.core.server.ServiceInstance;
|
||||
import io.kestra.jdbc.repository.AbstractJdbcServiceInstanceRepository;
|
||||
import io.micronaut.context.annotation.Requires;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static io.kestra.core.server.Service.ServiceState.*;
|
||||
|
||||
/**
|
||||
* Responsible for coordinating the state of all service instances.
|
||||
*
|
||||
* @see ServiceInstance
|
||||
*/
|
||||
@Singleton
|
||||
@JdbcRunnerEnabled
|
||||
@Requires(property = "kestra.server-type", pattern = "(EXECUTOR|STANDALONE)")
|
||||
public final class JdbcServiceLivenessCoordinator extends AbstractServiceLivenessCoordinator {
|
||||
|
||||
private final static Logger log = LoggerFactory.getLogger(JdbcServiceLivenessCoordinator.class);
|
||||
|
||||
private final AtomicReference<JdbcExecutor> executor = new AtomicReference<>();
|
||||
private final AbstractJdbcServiceInstanceRepository serviceInstanceRepository;
|
||||
|
||||
/**
|
||||
* Creates a new {@link JdbcServiceLivenessCoordinator} instance.
|
||||
*
|
||||
* @param serviceInstanceRepository The {@link AbstractJdbcServiceInstanceRepository}.
|
||||
* @param serverConfig The server liveness configuration.
|
||||
*/
|
||||
@Inject
|
||||
public JdbcServiceLivenessCoordinator(final AbstractJdbcServiceInstanceRepository serviceInstanceRepository,
|
||||
final ServerConfig serverConfig) {
|
||||
super(serviceInstanceRepository, serverConfig);
|
||||
this.serviceInstanceRepository = serviceInstanceRepository;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
**/
|
||||
@Override
|
||||
protected void onSchedule(final Instant now) {
|
||||
if (executor.get() == null) return; // only True during startup
|
||||
|
||||
// Transition all RUNNING but non-responding services to DISCONNECTED.
|
||||
transitionAllNonRespondingService(now);
|
||||
|
||||
// Finds all workers which are not in a RUNNING state.
|
||||
serviceInstanceRepository.transaction(configuration -> {
|
||||
List<ServiceInstance> nonRunningWorkers = serviceInstanceRepository
|
||||
.findAllNonRunningInstances(configuration, true)
|
||||
.stream()
|
||||
.filter(instance -> instance.is(Service.ServiceType.WORKER))
|
||||
.toList();
|
||||
|
||||
// List of workers for which we don't know the actual state of tasks executions.
|
||||
final List<ServiceInstance> uncleanShutdownWorkers = new ArrayList<>();
|
||||
|
||||
// ...all workers that have transitioned to DISCONNECTED or TERMINATING for more than terminationGracePeriod).
|
||||
uncleanShutdownWorkers.addAll(nonRunningWorkers.stream()
|
||||
.filter(nonRunning -> nonRunning.state().isDisconnectedOrTerminating())
|
||||
.filter(disconnectedOrTerminating -> disconnectedOrTerminating.isTerminationGracePeriodElapsed(now))
|
||||
.peek(instance -> {
|
||||
log.warn("Detected non-responding service [id={}, type={}, hostname={}] after termination grace period ({}ms).",
|
||||
instance.id(),
|
||||
instance.type(),
|
||||
instance.server().hostname(),
|
||||
now.toEpochMilli() - instance.updatedAt().toEpochMilli()
|
||||
);
|
||||
})
|
||||
.toList()
|
||||
);
|
||||
// ...all workers that have transitioned to TERMINATED_FORCED.
|
||||
uncleanShutdownWorkers.addAll(nonRunningWorkers.stream()
|
||||
.filter(nonRunning -> nonRunning.is(ServiceState.TERMINATED_FORCED))
|
||||
.toList()
|
||||
);
|
||||
|
||||
// Re-emit all WorkerJobs for unclean workers
|
||||
if (!uncleanShutdownWorkers.isEmpty()) {
|
||||
List<String> ids = uncleanShutdownWorkers.stream().map(ServiceInstance::id).toList();
|
||||
executor.get().reEmitWorkerJobsForWorkers(configuration, ids);
|
||||
}
|
||||
|
||||
// Transit all GRACEFUL AND UNCLEAN SHUTDOWN workers to NOT_RUNNING.
|
||||
Stream<ServiceInstance> cleanShutdownWorkers = nonRunningWorkers.stream()
|
||||
.filter(nonRunning -> nonRunning.is(ServiceState.TERMINATED_GRACEFULLY));
|
||||
Stream.concat(cleanShutdownWorkers, uncleanShutdownWorkers.stream()).forEach(
|
||||
instance -> serviceInstanceRepository.mayTransitServiceTo(configuration,
|
||||
instance,
|
||||
ServiceState.NOT_RUNNING,
|
||||
"The worker was detected as non-responsive after termination grace period. Service was transitioned to the 'NOT_RUNNING' state."
|
||||
)
|
||||
);
|
||||
});
|
||||
|
||||
// Transition all TERMINATED services to NOT_RUNNING.
|
||||
serviceInstanceRepository
|
||||
.findAllInstancesInStates(List.of(DISCONNECTED, TERMINATING, TERMINATED_GRACEFULLY, TERMINATED_FORCED)).stream()
|
||||
.filter(instance -> !instance.is(Service.ServiceType.WORKER)) // WORKERS are handle above.
|
||||
.filter(instance ->instance.isTerminationGracePeriodElapsed(now))
|
||||
.forEach(instance -> safelyTransitionServiceTo(instance, NOT_RUNNING, null));
|
||||
|
||||
|
||||
// Soft delete all services which are NOT_RUNNING anymore.
|
||||
serviceInstanceRepository.findAllInstancesInState(ServiceState.NOT_RUNNING)
|
||||
.forEach(instance -> safelyTransitionServiceTo(instance, ServiceState.EMPTY, null));
|
||||
|
||||
mayDetectAndLogNewConnectedServices();
|
||||
}
|
||||
|
||||
synchronized void setExecutor(final JdbcExecutor executor) {
|
||||
this.executor.set(executor);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
void setServerInstance(final String serverId) {
|
||||
this.serverId = serverId;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,98 @@
|
||||
package io.kestra.jdbc.server;
|
||||
|
||||
import io.kestra.core.contexts.KestraContext;
|
||||
import io.kestra.core.models.ServerType;
|
||||
import io.kestra.core.repositories.ServiceInstanceRepositoryInterface;
|
||||
import io.kestra.core.server.ServerConfig;
|
||||
import io.kestra.core.server.Service;
|
||||
import io.kestra.core.server.ServiceInstance;
|
||||
import io.kestra.core.server.ServiceInstanceFactory;
|
||||
import io.kestra.core.server.ServiceLivenessManager;
|
||||
import io.kestra.core.server.ServiceRegistry;
|
||||
import io.kestra.core.server.ServiceStateChangeEvent;
|
||||
import io.kestra.jdbc.runner.JdbcRunnerEnabled;
|
||||
import io.micronaut.context.annotation.Context;
|
||||
import io.micronaut.runtime.event.annotation.EventListener;
|
||||
import jakarta.inject.Inject;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.time.Instant;
|
||||
|
||||
@Context
|
||||
@JdbcRunnerEnabled
|
||||
public final class JdbcServiceLivenessManager extends ServiceLivenessManager {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(JdbcServiceLivenessManager.class);
|
||||
|
||||
@Inject
|
||||
public JdbcServiceLivenessManager(final ServerConfig configuration,
|
||||
final ServiceRegistry registry,
|
||||
final ServiceInstanceFactory serviceInstanceFactory,
|
||||
final ServiceInstanceRepositoryInterface repository) {
|
||||
super(
|
||||
configuration,
|
||||
registry,
|
||||
serviceInstanceFactory,
|
||||
repository,
|
||||
new DefaultStateTransitionFailureCallback()
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
**/
|
||||
@Override
|
||||
@EventListener
|
||||
public void onServiceStateChangeEvent(final ServiceStateChangeEvent event) {
|
||||
super.onServiceStateChangeEvent(event);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
**/
|
||||
@Override
|
||||
protected boolean beforeScheduledStateUpdate(final Instant now,
|
||||
final Service service,
|
||||
final ServiceInstance instance) {
|
||||
|
||||
// Proactively disconnect a WORKER server when it fails to update its current state
|
||||
// for more than the configured liveness timeout (this is to prevent zombie server).
|
||||
if (isLivenessEnabled() && isWorkerServer() && isServerDisconnected(now)) {
|
||||
log.error("[Service id={}, type='{}', hostname='{}'] Failed to update state before reaching timeout ({}ms). Disconnecting.",
|
||||
instance.id(),
|
||||
instance.type(),
|
||||
instance.server().hostname(),
|
||||
getElapsedMilliSinceLastStateUpdate(now)
|
||||
);
|
||||
// Force the WORKER to transition to DISCONNECTED.
|
||||
ServiceInstance updated = updateServiceInstanceState(now, service, Service.ServiceState.DISCONNECTED, OnStateTransitionFailureCallback.NOOP);
|
||||
|
||||
// Trigger state transition failure callback.
|
||||
onStateTransitionFailureCallback.execute(now, service, updated, true);
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks whether the current server is running in WORKER mode.
|
||||
*/
|
||||
private boolean isWorkerServer() {
|
||||
return KestraContext.getContext().getServerType().equals(ServerType.WORKER);
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks whether the server is DISCONNECTED.
|
||||
*/
|
||||
private boolean isServerDisconnected(final Instant now) {
|
||||
long timeoutMilli = serverConfig.liveness().timeout().toMillis();
|
||||
// Check thread starvation or clock leap (i.e., JVM was frozen)
|
||||
return getElapsedMilliSinceLastSchedule(now) < timeoutMilli && getElapsedMilliSinceLastStateUpdate(now) > timeoutMilli;
|
||||
}
|
||||
|
||||
private long getElapsedMilliSinceLastStateUpdate(final Instant now) {
|
||||
return now.toEpochMilli() - (lastSucceedStateUpdated() != null ? lastSucceedStateUpdated().toEpochMilli() : now.toEpochMilli());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,273 @@
|
||||
package io.kestra.jdbc.repository;
|
||||
|
||||
import io.kestra.core.server.ServerConfig;
|
||||
import io.kestra.core.server.ServerInstance;
|
||||
import io.kestra.core.server.Service;
|
||||
import io.kestra.core.server.ServiceInstance;
|
||||
import io.kestra.core.server.ServiceStateTransition;
|
||||
import io.kestra.core.utils.IdUtils;
|
||||
import io.kestra.core.utils.Network;
|
||||
import io.kestra.jdbc.JdbcTestUtils;
|
||||
import io.micronaut.test.extensions.junit5.annotation.MicronautTest;
|
||||
import jakarta.inject.Inject;
|
||||
import org.hamcrest.Matchers;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
import java.time.temporal.ChronoUnit;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
|
||||
import static io.kestra.core.server.ServiceStateTransition.Result.FAILED;
|
||||
import static io.kestra.core.server.ServiceStateTransition.Result.SUCCEED;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
@MicronautTest(transactional = false)
|
||||
public abstract class AbstractJdbcServiceInstanceRepositoryTest {
|
||||
|
||||
@Inject
|
||||
protected AbstractJdbcServiceInstanceRepository repository;
|
||||
|
||||
@Inject
|
||||
JdbcTestUtils jdbcTestUtils;
|
||||
|
||||
@BeforeEach
|
||||
protected void init() {
|
||||
jdbcTestUtils.drop();
|
||||
jdbcTestUtils.migrate();
|
||||
}
|
||||
|
||||
@Test
|
||||
protected void shouldSaveServiceInstance() {
|
||||
// Given
|
||||
ServiceInstance instance = AbstractJdbcServiceInstanceRepositoryTest.Fixtures.RunningServiceInstance;
|
||||
|
||||
// When
|
||||
repository.save(instance);
|
||||
|
||||
// Then
|
||||
Optional<ServiceInstance> result = repository.findById(instance.id());
|
||||
Assertions.assertEquals(Optional.of(instance), result);
|
||||
}
|
||||
|
||||
@Test
|
||||
protected void shouldDeleteGivenServiceInstance() {
|
||||
// Given
|
||||
AbstractJdbcServiceInstanceRepositoryTest.Fixtures.all().forEach(repository::save);
|
||||
final ServiceInstance instance = AbstractJdbcServiceInstanceRepositoryTest.Fixtures.EmptyServiceInstance;
|
||||
|
||||
// When
|
||||
repository.delete(instance);
|
||||
|
||||
// Then
|
||||
Optional<ServiceInstance> result = repository.findById(instance.id());
|
||||
Assertions.assertEquals(Optional.empty(), result);
|
||||
}
|
||||
|
||||
@Test
|
||||
protected void shouldFindByServiceId() {
|
||||
// Given
|
||||
AbstractJdbcServiceInstanceRepositoryTest.Fixtures.all().forEach(repository::save);
|
||||
String uuid = AbstractJdbcServiceInstanceRepositoryTest.Fixtures.EmptyServiceInstance.id();
|
||||
|
||||
// When
|
||||
Optional<ServiceInstance> result = repository.findById(uuid);
|
||||
|
||||
// Then
|
||||
Assertions.assertEquals(Optional.of(AbstractJdbcServiceInstanceRepositoryTest.Fixtures.EmptyServiceInstance), result);
|
||||
}
|
||||
|
||||
@Test
|
||||
protected void shouldFindAllServiceInstances() {
|
||||
// Given
|
||||
AbstractJdbcServiceInstanceRepositoryTest.Fixtures.all().forEach(repository::save);
|
||||
|
||||
// When
|
||||
List<ServiceInstance> results = repository.findAll();
|
||||
|
||||
// Then
|
||||
assertEquals(results.size(), AbstractJdbcServiceInstanceRepositoryTest.Fixtures.all().size());
|
||||
assertThat(results, Matchers.containsInAnyOrder(AbstractJdbcServiceInstanceRepositoryTest.Fixtures.all().toArray()));
|
||||
}
|
||||
|
||||
@Test
|
||||
protected void shouldFindAllNonRunningInstances() {
|
||||
// Given
|
||||
AbstractJdbcServiceInstanceRepositoryTest.Fixtures.all().forEach(repository::save);
|
||||
|
||||
// When
|
||||
List<ServiceInstance> results = repository.findAllNonRunningInstances();
|
||||
|
||||
// Then
|
||||
assertEquals(AbstractJdbcServiceInstanceRepositoryTest.Fixtures.allNonRunning().size(), results.size());
|
||||
assertThat(results, Matchers.containsInAnyOrder(AbstractJdbcServiceInstanceRepositoryTest.Fixtures.allNonRunning().toArray()));
|
||||
}
|
||||
|
||||
@Test
|
||||
protected void shouldFindAllInstancesInNotRunningState() {
|
||||
// Given
|
||||
AbstractJdbcServiceInstanceRepositoryTest.Fixtures.all().forEach(repository::save);
|
||||
|
||||
// When
|
||||
List<ServiceInstance> results = repository.findAllInstancesInNotRunningState();
|
||||
|
||||
// Then
|
||||
assertEquals(AbstractJdbcServiceInstanceRepositoryTest.Fixtures.allInNotRunningState().size(), results.size());
|
||||
assertThat(results, Matchers.containsInAnyOrder(AbstractJdbcServiceInstanceRepositoryTest.Fixtures.allInNotRunningState().toArray()));
|
||||
}
|
||||
|
||||
@Test
|
||||
protected void shouldFindTimeoutRunningInstancesGivenTimeoutInstance() {
|
||||
// Given
|
||||
final Instant now = Instant.now();
|
||||
ServiceInstance instance = AbstractJdbcServiceInstanceRepositoryTest.Fixtures.RunningServiceInstance
|
||||
.updateState(Service.ServiceState.RUNNING, now.minus(Duration.ofSeconds(30)).truncatedTo(ChronoUnit.MILLIS));
|
||||
|
||||
repository.save(instance);
|
||||
|
||||
// When
|
||||
List<ServiceInstance> results = repository.findAllTimeoutRunningInstances(now);
|
||||
|
||||
// Then
|
||||
assertEquals(1, results.size());
|
||||
assertThat(results, Matchers.containsInAnyOrder(instance));
|
||||
}
|
||||
|
||||
@Test
|
||||
protected void shouldNotFindTimeoutRunningInstanceGivenHealthyInstance() {
|
||||
// Given
|
||||
final Instant now = Instant.now();
|
||||
ServiceInstance instance = AbstractJdbcServiceInstanceRepositoryTest.Fixtures.RunningServiceInstance
|
||||
.updateState(Service.ServiceState.RUNNING, now.minus(Duration.ofSeconds(5)).truncatedTo(ChronoUnit.MILLIS));
|
||||
|
||||
repository.save(instance);
|
||||
|
||||
// When
|
||||
List<ServiceInstance> results = repository.findAllTimeoutRunningInstances(now);
|
||||
|
||||
// Then
|
||||
assertTrue(results.isEmpty());
|
||||
}
|
||||
|
||||
@Test
|
||||
void shouldReturnEmptyForTransitionWorkerStateGivenInvalidWorker() {
|
||||
// Given
|
||||
ServiceInstance instance = Fixtures.RunningServiceInstance;
|
||||
|
||||
// When
|
||||
ServiceStateTransition.Response result = repository
|
||||
.mayTransitionServiceTo(instance, Service.ServiceState.TERMINATING);
|
||||
|
||||
// Then
|
||||
Assertions.assertEquals(new ServiceStateTransition.Response(ServiceStateTransition.Result.ABORTED), result);
|
||||
}
|
||||
|
||||
@Test
|
||||
void shouldReturnSucceedTransitionResponseForValidTransition() {
|
||||
// Given
|
||||
ServiceInstance instance = Fixtures.RunningServiceInstance;
|
||||
repository.save(instance);
|
||||
|
||||
// When
|
||||
ServiceStateTransition.Response response = repository
|
||||
.mayTransitionServiceTo(instance, Service.ServiceState.TERMINATING); // RUNNING -> TERMINATING: valid transition
|
||||
|
||||
// Then
|
||||
Assertions.assertEquals(SUCCEED, response.result());
|
||||
Assertions.assertEquals(Service.ServiceState.TERMINATING, response.instance().state());
|
||||
Assertions.assertTrue(response.instance().updatedAt().isAfter(instance.updatedAt()));
|
||||
}
|
||||
|
||||
@Test
|
||||
void shouldReturnInvalidTransitionResponseForInvalidTransition() {
|
||||
// Given
|
||||
ServiceInstance instance = Fixtures.EmptyServiceInstance;
|
||||
repository.save(instance);
|
||||
|
||||
// When
|
||||
ServiceStateTransition.Response response = repository
|
||||
.mayTransitionServiceTo(instance, Service.ServiceState.RUNNING); // EMPTY -> RUNNING: INVALID transition
|
||||
|
||||
// Then
|
||||
Assertions.assertEquals(new ServiceStateTransition.Response(FAILED, instance), response);
|
||||
}
|
||||
|
||||
public static final class Fixtures {
|
||||
|
||||
public static List<ServiceInstance> all() {
|
||||
return List.of(
|
||||
RunningServiceInstance,
|
||||
PendingShutdownServiceInstance,
|
||||
GracefulShutdownServiceInstance,
|
||||
ForcedShutdownServiceInstance,
|
||||
NotRunningServiceInstance,
|
||||
EmptyServiceInstance
|
||||
);
|
||||
}
|
||||
|
||||
public static List<ServiceInstance> allNonRunning() {
|
||||
return List.of(
|
||||
PendingShutdownServiceInstance,
|
||||
GracefulShutdownServiceInstance,
|
||||
ForcedShutdownServiceInstance,
|
||||
NotRunningServiceInstance,
|
||||
EmptyServiceInstance
|
||||
);
|
||||
}
|
||||
|
||||
public static List<ServiceInstance> allInNotRunningState() {
|
||||
return List.of(NotRunningServiceInstance);
|
||||
}
|
||||
|
||||
public static final ServiceInstance RunningServiceInstance =
|
||||
serviceInstanceFor(Service.ServiceState.RUNNING);
|
||||
|
||||
public static final ServiceInstance PendingShutdownServiceInstance =
|
||||
serviceInstanceFor(Service.ServiceState.TERMINATING);
|
||||
|
||||
public static final ServiceInstance GracefulShutdownServiceInstance =
|
||||
serviceInstanceFor(Service.ServiceState.TERMINATED_GRACEFULLY);
|
||||
|
||||
public static final ServiceInstance ForcedShutdownServiceInstance =
|
||||
serviceInstanceFor(Service.ServiceState.TERMINATED_FORCED);
|
||||
|
||||
public static final ServiceInstance NotRunningServiceInstance =
|
||||
serviceInstanceFor(Service.ServiceState.NOT_RUNNING);
|
||||
|
||||
public static final ServiceInstance EmptyServiceInstance =
|
||||
serviceInstanceFor(Service.ServiceState.EMPTY);
|
||||
|
||||
public static ServiceInstance serviceInstanceFor(final Service.ServiceState state) {
|
||||
ServerConfig config = new ServerConfig(Duration.ZERO,
|
||||
new ServerConfig.Liveness(
|
||||
true,
|
||||
Duration.ZERO,
|
||||
Duration.ofSeconds(10), // timeout
|
||||
Duration.ZERO,
|
||||
Duration.ZERO
|
||||
)
|
||||
);
|
||||
return new ServiceInstance(
|
||||
IdUtils.create(),
|
||||
Service.ServiceType.WORKER,
|
||||
state,
|
||||
new ServerInstance(
|
||||
ServerInstance.Type.STANDALONE,
|
||||
"N/A",
|
||||
Network.localHostname(), Map.of()
|
||||
),
|
||||
Instant.now().truncatedTo(ChronoUnit.MILLIS),
|
||||
Instant.now().truncatedTo(ChronoUnit.MILLIS),
|
||||
List.of(),
|
||||
config,
|
||||
Map.of()
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,149 +0,0 @@
|
||||
package io.kestra.jdbc.repository;
|
||||
|
||||
import io.kestra.core.runners.ServerInstance;
|
||||
import io.kestra.core.runners.WorkerInstance;
|
||||
import io.kestra.jdbc.JdbcTestUtils;
|
||||
import io.kestra.jdbc.JooqDSLContextWrapper;
|
||||
import io.micronaut.test.extensions.junit5.annotation.MicronautTest;
|
||||
import jakarta.inject.Inject;
|
||||
import org.jooq.DSLContext;
|
||||
import org.jooq.impl.DSL;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.time.Instant;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.hamcrest.Matchers.greaterThan;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
|
||||
@MicronautTest(transactional = false)
|
||||
public abstract class AbstractJdbcWorkerInstanceRepositoryTest {
|
||||
@Inject
|
||||
protected AbstractJdbcWorkerInstanceRepository workerInstanceRepository;
|
||||
|
||||
@Inject
|
||||
JdbcTestUtils jdbcTestUtils;
|
||||
|
||||
@Inject
|
||||
protected JooqDSLContextWrapper dslContextWrapper;
|
||||
|
||||
@BeforeEach
|
||||
protected void init() {
|
||||
jdbcTestUtils.drop();
|
||||
jdbcTestUtils.migrate();
|
||||
}
|
||||
|
||||
@Test
|
||||
protected void save() {
|
||||
WorkerInstance workerInstance = createWorkerInstance(UUID.randomUUID().toString());
|
||||
workerInstanceRepository.save(workerInstance);
|
||||
|
||||
Optional<WorkerInstance> find = workerInstanceRepository.findByWorkerUuid(workerInstance.getWorkerUuid().toString());
|
||||
assertThat(find.isPresent(), is(true));
|
||||
assertThat(find.get().getWorkerUuid(), is(workerInstance.getWorkerUuid()));
|
||||
}
|
||||
|
||||
@Test
|
||||
protected void delete() {
|
||||
WorkerInstance workerInstance = createWorkerInstance(UUID.randomUUID().toString());
|
||||
workerInstanceRepository.save(workerInstance);
|
||||
|
||||
dslContextWrapper.transaction(configuration -> {
|
||||
DSLContext context = DSL.using(configuration);
|
||||
|
||||
Optional<WorkerInstance> find = workerInstanceRepository.findByWorkerUuid(workerInstance.getWorkerUuid().toString());
|
||||
assertThat(find.isPresent(), is(true));
|
||||
assertThat(find.get().getWorkerUuid(), is(workerInstance.getWorkerUuid()));
|
||||
|
||||
workerInstanceRepository.delete(context, workerInstance);
|
||||
find = workerInstanceRepository.findByWorkerUuid(workerInstance.getWorkerUuid().toString());
|
||||
assertThat(find.isPresent(), is(false));
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
protected void findAll() {
|
||||
WorkerInstance workerInstance = createWorkerInstance(UUID.randomUUID().toString());
|
||||
WorkerInstance workerInstanceAlive = createWorkerInstance(UUID.randomUUID().toString());
|
||||
WorkerInstance workerInstanceDead = createWorkerInstance(UUID.randomUUID().toString(), false);
|
||||
|
||||
workerInstanceRepository.save(workerInstance);
|
||||
workerInstanceRepository.save(workerInstanceAlive);
|
||||
workerInstanceRepository.save(workerInstanceDead);
|
||||
|
||||
dslContextWrapper.transaction(configuration -> {
|
||||
DSLContext context = DSL.using(configuration);
|
||||
|
||||
List<WorkerInstance> finds = workerInstanceRepository.findAll(context);
|
||||
assertThat(finds.size(), is(3));
|
||||
|
||||
finds = workerInstanceRepository.findAllToDelete(context);
|
||||
assertThat(finds.size(), is(1));
|
||||
|
||||
finds = workerInstanceRepository.findAllAlive(context);
|
||||
assertThat(finds.size(), is(2));
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
protected void find() {
|
||||
WorkerInstance workerInstance = createWorkerInstance(UUID.randomUUID().toString());
|
||||
workerInstanceRepository.save(workerInstance);
|
||||
|
||||
Optional<WorkerInstance> find = workerInstanceRepository.findByWorkerUuid(workerInstance.getWorkerUuid().toString());
|
||||
assertThat(find.isPresent(), is(true));
|
||||
assertThat(find.get().getWorkerUuid(), is(workerInstance.getWorkerUuid()));
|
||||
}
|
||||
|
||||
@Test
|
||||
protected void heartbeatCheckup() throws InterruptedException {
|
||||
WorkerInstance workerInstance = createWorkerInstance(UUID.randomUUID().toString());
|
||||
workerInstanceRepository.save(workerInstance);
|
||||
CountDownLatch queueCount = new CountDownLatch(1);
|
||||
|
||||
queueCount.await(15, TimeUnit.SECONDS);
|
||||
Optional<WorkerInstance> updatedWorkerInstance = workerInstanceRepository.heartbeatCheckUp(workerInstance.getWorkerUuid().toString());
|
||||
assertThat(updatedWorkerInstance.isPresent(), is(true));
|
||||
assertThat(updatedWorkerInstance.get().getHeartbeatDate(), greaterThan(workerInstance.getHeartbeatDate()));
|
||||
}
|
||||
|
||||
@Test
|
||||
protected void heartbeatsStatusUpdate() {
|
||||
WorkerInstance workerInstance = createWorkerInstance(UUID.randomUUID().toString());
|
||||
workerInstance.setHeartbeatDate(Instant.now().minusSeconds(3600));
|
||||
workerInstanceRepository.save(workerInstance);
|
||||
|
||||
dslContextWrapper.transaction(configuration -> {
|
||||
DSLContext context = DSL.using(configuration);
|
||||
|
||||
workerInstanceRepository.heartbeatsStatusUpdate(context);
|
||||
Optional<WorkerInstance> find = workerInstanceRepository.findByWorkerUuid(workerInstance.getWorkerUuid().toString());
|
||||
assertThat(find.isPresent(), is(true));
|
||||
assertThat(find.get().getStatus(), is(WorkerInstance.Status.DEAD));
|
||||
});
|
||||
}
|
||||
|
||||
private WorkerInstance createWorkerInstance(String workerUuid, Boolean alive) {
|
||||
return WorkerInstance.builder()
|
||||
.workerUuid(UUID.fromString(workerUuid))
|
||||
.workerGroup(null)
|
||||
.managementPort(0)
|
||||
.hostname("kestra.io")
|
||||
.partitions(null)
|
||||
.port(0)
|
||||
.status(alive ? WorkerInstance.Status.UP : WorkerInstance.Status.DEAD)
|
||||
.heartbeatDate(alive ? Instant.now() : Instant.now().minusSeconds(3600))
|
||||
.server(new ServerInstance(UUID.randomUUID())) // simulate worker is running on different server
|
||||
.build();
|
||||
}
|
||||
|
||||
private WorkerInstance createWorkerInstance(String workerUuid) {
|
||||
return createWorkerInstance(workerUuid, true);
|
||||
}
|
||||
}
|
||||
@@ -11,7 +11,14 @@ import io.kestra.core.models.triggers.TriggerContext;
|
||||
import io.kestra.core.queues.QueueFactoryInterface;
|
||||
import io.kestra.core.queues.QueueInterface;
|
||||
import io.kestra.core.repositories.LocalFlowRepositoryLoader;
|
||||
import io.kestra.core.runners.*;
|
||||
import io.kestra.core.runners.RunContextFactory;
|
||||
import io.kestra.core.runners.StandAloneRunner;
|
||||
import io.kestra.core.runners.Worker;
|
||||
import io.kestra.core.runners.WorkerJob;
|
||||
import io.kestra.core.runners.WorkerTask;
|
||||
import io.kestra.core.runners.WorkerTaskResult;
|
||||
import io.kestra.core.runners.WorkerTrigger;
|
||||
import io.kestra.core.runners.WorkerTriggerResult;
|
||||
import io.kestra.core.services.SkipExecutionService;
|
||||
import io.kestra.core.tasks.test.Sleep;
|
||||
import io.kestra.core.tasks.test.SleepTrigger;
|
||||
@@ -25,6 +32,7 @@ import io.micronaut.test.extensions.junit5.annotation.MicronautTest;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Named;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.Disabled;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.TestInstance;
|
||||
|
||||
@@ -34,7 +42,6 @@ import java.time.Duration;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
@@ -44,10 +51,10 @@ import static org.hamcrest.Matchers.is;
|
||||
import static org.hamcrest.Matchers.not;
|
||||
import static org.junit.jupiter.api.Assertions.fail;
|
||||
|
||||
@MicronautTest(transactional = false, environments = "heartbeat")
|
||||
@MicronautTest(transactional = false, environments = {"test", "liveness"})
|
||||
@TestInstance(TestInstance.Lifecycle.PER_CLASS) // must be per-class to allow calling once init() which took a lot of time
|
||||
@Property(name = "kestra.server-type", value = "EXECUTOR")
|
||||
public abstract class JdbcHeartbeatTest {
|
||||
public abstract class JdbcServiceLivenessCoordinatorTest {
|
||||
@Inject
|
||||
private StandAloneRunner runner;
|
||||
|
||||
@@ -76,31 +83,32 @@ public abstract class JdbcHeartbeatTest {
|
||||
QueueInterface<WorkerTriggerResult> workerTriggerResultQueue;
|
||||
|
||||
@Inject
|
||||
SkipExecutionService skipExecutionService;
|
||||
JdbcServiceLivenessCoordinator jdbcServiceLivenessHandler;
|
||||
|
||||
@Inject JdbcHeartbeat heartbeat;
|
||||
@Inject
|
||||
SkipExecutionService skipExecutionService;
|
||||
|
||||
@BeforeAll
|
||||
void init() throws IOException, URISyntaxException {
|
||||
jdbcTestUtils.drop();
|
||||
jdbcTestUtils.migrate();
|
||||
|
||||
TestsUtils.loads(repositoryLoader);
|
||||
heartbeat.setServerInstance(new ServerInstance(UUID.randomUUID())); // simulate worker is running on different server
|
||||
// Simulate that executor and workers are not running on the same JVM.
|
||||
jdbcServiceLivenessHandler.setServerInstance(IdUtils.create());
|
||||
}
|
||||
|
||||
@Test
|
||||
void taskResubmit() throws Exception {
|
||||
void shouldReEmitTasksWhenWorkerIsDetectedAsNonResponding() throws Exception {
|
||||
CountDownLatch runningLatch = new CountDownLatch(1);
|
||||
CountDownLatch resubmitLatch = new CountDownLatch(1);
|
||||
|
||||
Worker worker = new Worker(applicationContext, 8, null);
|
||||
applicationContext.registerSingleton(worker);
|
||||
// create first worker
|
||||
Worker worker = new Worker(applicationContext, 1, null, IdUtils.create());
|
||||
worker.run();
|
||||
|
||||
runner.setSchedulerEnabled(false);
|
||||
runner.setWorkerEnabled(false);
|
||||
runner.run();
|
||||
|
||||
AtomicReference<WorkerTaskResult> workerTaskResult = new AtomicReference<>(null);
|
||||
workerTaskResultQueue.receive(either -> {
|
||||
workerTaskResult.set(either.getLeft());
|
||||
@@ -114,20 +122,20 @@ public abstract class JdbcHeartbeatTest {
|
||||
}
|
||||
});
|
||||
|
||||
workerJobQueue.emit(workerTask(1500));
|
||||
runningLatch.await(2, TimeUnit.SECONDS);
|
||||
worker.shutdown();
|
||||
workerJobQueue.emit(workerTask(Duration.ofSeconds(10)));
|
||||
runningLatch.await(5, TimeUnit.SECONDS);
|
||||
worker.shutdown(); // stop processing task
|
||||
|
||||
Worker newWorker = new Worker(applicationContext, 8, null);
|
||||
applicationContext.registerSingleton(newWorker);
|
||||
// create second worker (this will revoke previously one).
|
||||
Worker newWorker = new Worker(applicationContext, 1, null, IdUtils.create());
|
||||
newWorker.run();
|
||||
resubmitLatch.await(15, TimeUnit.SECONDS);
|
||||
|
||||
resubmitLatch.await(30, TimeUnit.SECONDS);
|
||||
newWorker.shutdown();
|
||||
assertThat(workerTaskResult.get().getTaskRun().getState().getCurrent(), is(Type.SUCCESS));
|
||||
}
|
||||
|
||||
@Test
|
||||
@Disabled
|
||||
void taskResubmitSkipExecution() throws Exception {
|
||||
CountDownLatch runningLatch = new CountDownLatch(1);
|
||||
|
||||
@@ -137,7 +145,7 @@ public abstract class JdbcHeartbeatTest {
|
||||
runner.setSchedulerEnabled(false);
|
||||
runner.setWorkerEnabled(false);
|
||||
runner.run();
|
||||
WorkerTask workerTask = workerTask(1500);
|
||||
WorkerTask workerTask = workerTask(Duration.ofSeconds(10));
|
||||
skipExecutionService.setSkipExecutions(List.of(workerTask.getTaskRun().getExecutionId()));
|
||||
|
||||
AtomicReference<WorkerTaskResult> workerTaskResult = new AtomicReference<>(null);
|
||||
@@ -170,10 +178,10 @@ public abstract class JdbcHeartbeatTest {
|
||||
}
|
||||
|
||||
@Test
|
||||
void triggerResubmit() throws Exception {
|
||||
void shouldReEmitTriggerWhenWorkerIsDetectedAsNonResponding() throws Exception {
|
||||
CountDownLatch countDownLatch = new CountDownLatch(1);
|
||||
|
||||
Worker worker = new Worker(applicationContext, 8, null);
|
||||
Worker worker = new Worker(applicationContext, 1, null);
|
||||
applicationContext.registerSingleton(worker);
|
||||
worker.run();
|
||||
runner.setSchedulerEnabled(false);
|
||||
@@ -186,32 +194,33 @@ public abstract class JdbcHeartbeatTest {
|
||||
countDownLatch.countDown();
|
||||
});
|
||||
|
||||
WorkerTrigger workerTrigger = workerTrigger(7000);
|
||||
WorkerTrigger workerTrigger = workerTrigger(Duration.ofSeconds(10));
|
||||
workerJobQueue.emit(workerTrigger);
|
||||
Await.until(() -> worker.getEvaluateTriggerRunningCount()
|
||||
.get("io.kestra.jdbc.runner.jdbcheartbeattest_workertrigger_unit-test") != null,
|
||||
.get(workerTrigger.getTriggerContext().uid()) != null,
|
||||
Duration.ofMillis(100),
|
||||
Duration.ofSeconds(5)
|
||||
);
|
||||
worker.shutdown();
|
||||
|
||||
Worker newWorker = new Worker(applicationContext, 8, null);
|
||||
Worker newWorker = new Worker(applicationContext, 1, null);
|
||||
applicationContext.registerSingleton(newWorker);
|
||||
newWorker.run();
|
||||
boolean lastAwait = countDownLatch.await(12, TimeUnit.SECONDS);
|
||||
|
||||
boolean lastAwait = countDownLatch.await(10, TimeUnit.SECONDS);
|
||||
|
||||
newWorker.shutdown();
|
||||
assertThat("Last await result was " + lastAwait, workerTriggerResult.get().getSuccess(), is(true));
|
||||
}
|
||||
|
||||
private WorkerTask workerTask(long sleepDuration) {
|
||||
private WorkerTask workerTask(Duration sleep) {
|
||||
Sleep bash = Sleep.builder()
|
||||
.type(Sleep.class.getName())
|
||||
.id("unit-test")
|
||||
.duration(sleepDuration)
|
||||
.duration(sleep.toMillis())
|
||||
.build();
|
||||
|
||||
Execution execution = TestsUtils.mockExecution(flowBuilder(sleepDuration), ImmutableMap.of());
|
||||
Execution execution = TestsUtils.mockExecution(flowBuilder(sleep), ImmutableMap.of());
|
||||
|
||||
ResolvedTask resolvedTask = ResolvedTask.of(bash);
|
||||
|
||||
@@ -222,11 +231,11 @@ public abstract class JdbcHeartbeatTest {
|
||||
.build();
|
||||
}
|
||||
|
||||
private WorkerTrigger workerTrigger(long sleepDuration) {
|
||||
private WorkerTrigger workerTrigger(Duration sleep) {
|
||||
SleepTrigger trigger = SleepTrigger.builder()
|
||||
.type(SleepTrigger.class.getName())
|
||||
.id("unit-test")
|
||||
.duration(sleepDuration)
|
||||
.duration(sleep.toMillis())
|
||||
.build();
|
||||
|
||||
Map.Entry<ConditionContext, TriggerContext> mockedTrigger = TestsUtils.mockTrigger(runContextFactory, trigger);
|
||||
@@ -238,17 +247,17 @@ public abstract class JdbcHeartbeatTest {
|
||||
.build();
|
||||
}
|
||||
|
||||
private Flow flowBuilder(long sleepDuration) {
|
||||
private Flow flowBuilder(final Duration sleep) {
|
||||
Sleep bash = Sleep.builder()
|
||||
.type(Sleep.class.getName())
|
||||
.id("unit-test")
|
||||
.duration(sleepDuration)
|
||||
.duration(sleep.toMillis())
|
||||
.build();
|
||||
|
||||
SleepTrigger trigger = SleepTrigger.builder()
|
||||
.type(SleepTrigger.class.getName())
|
||||
.id("unit-test")
|
||||
.duration(sleepDuration)
|
||||
.duration(sleep.toMillis())
|
||||
.build();
|
||||
|
||||
return Flow.builder()
|
||||
@@ -0,0 +1,94 @@
|
||||
package io.kestra.jdbc.server;
|
||||
|
||||
import io.kestra.core.contexts.KestraContext;
|
||||
import io.kestra.core.models.ServerType;
|
||||
import io.kestra.core.repositories.ServiceInstanceRepositoryInterface;
|
||||
import io.kestra.core.server.ServerConfig;
|
||||
import io.kestra.core.server.Service;
|
||||
import io.kestra.core.server.ServiceInstance;
|
||||
import io.kestra.core.server.ServiceInstanceFactory;
|
||||
import io.kestra.core.server.ServiceRegistry;
|
||||
import io.kestra.core.server.ServiceStateTransition;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.junit.jupiter.MockitoExtension;
|
||||
import org.mockito.junit.jupiter.MockitoSettings;
|
||||
import org.mockito.quality.Strictness;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
|
||||
import static io.kestra.core.server.ServiceLivenessManagerTest.newServiceForState;
|
||||
import static io.kestra.core.server.ServiceLivenessManagerTest.serviceInstanceFor;
|
||||
import static io.kestra.core.server.ServiceStateTransition.Result.SUCCEED;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
|
||||
@ExtendWith({MockitoExtension.class})
|
||||
@MockitoSettings(strictness = Strictness.LENIENT)
|
||||
class JdbcServiceLivenessManagerTest {
|
||||
|
||||
private static final Duration DEFAULT_DURATION = Duration.ofSeconds(5);
|
||||
|
||||
@Mock
|
||||
public ServiceInstanceRepositoryInterface repository;
|
||||
|
||||
private JdbcServiceLivenessManager serviceLivenessManager;
|
||||
|
||||
@Mock
|
||||
private KestraContext context;
|
||||
|
||||
@BeforeEach
|
||||
void beforeEach() {
|
||||
Mockito.when(context.getServerType()).thenReturn(ServerType.WORKER);
|
||||
KestraContext.setContext(context);
|
||||
ServerConfig config = new ServerConfig(Duration.ZERO,
|
||||
new ServerConfig.Liveness(
|
||||
true,
|
||||
Duration.ZERO,
|
||||
DEFAULT_DURATION, // timeout
|
||||
DEFAULT_DURATION,
|
||||
DEFAULT_DURATION
|
||||
)
|
||||
);
|
||||
|
||||
this.serviceLivenessManager = new JdbcServiceLivenessManager(
|
||||
config,
|
||||
new ServiceRegistry(),
|
||||
new ServiceInstanceFactory(config, null, null),
|
||||
repository
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
void shouldRunOnStateTransitionFailureWhenTimeoutForWorker() {
|
||||
// Given
|
||||
Service running = newServiceForState(Service.ServiceState.RUNNING);
|
||||
serviceLivenessManager.updateServiceInstance(running, serviceInstanceFor(running));
|
||||
|
||||
// When
|
||||
Instant now = Instant.now();
|
||||
final ServiceStateTransition.Response response = new ServiceStateTransition.Response(
|
||||
SUCCEED,
|
||||
serviceInstanceFor(running)
|
||||
);
|
||||
|
||||
Mockito.when(repository.mayTransitionServiceTo(any(ServiceInstance.class), any(Service.ServiceState.class))).thenReturn(response);
|
||||
serviceLivenessManager.run(now); // SUCCEED
|
||||
|
||||
// Simulate exception on each transition
|
||||
Mockito.when(repository.mayTransitionServiceTo(any(ServiceInstance.class), any(Service.ServiceState.class))).thenThrow(new RuntimeException());
|
||||
|
||||
serviceLivenessManager.run(now.plus(Duration.ofSeconds(2))); // FAIL
|
||||
Mockito.verify(context, Mockito.never()).exit(Mockito.eq(1));
|
||||
|
||||
serviceLivenessManager.run(now.plus(Duration.ofSeconds(4))); // FAIL
|
||||
Mockito.verify(context, Mockito.never()).exit(Mockito.eq(1));
|
||||
|
||||
// Then
|
||||
serviceLivenessManager.run(now.plus(Duration.ofSeconds(6))); // TIMEOUT
|
||||
Mockito.verify(context, Mockito.times(1)).exit(Mockito.eq(1));
|
||||
}
|
||||
}
|
||||
@@ -1,4 +0,0 @@
|
||||
kestra:
|
||||
heartbeat:
|
||||
frequency: 3s
|
||||
heartbeat-missed: 1
|
||||
@@ -0,0 +1,68 @@
|
||||
package io.kestra.repository.memory;
|
||||
|
||||
import io.kestra.core.repositories.ServiceInstanceRepositoryInterface;
|
||||
import io.kestra.core.server.Service;
|
||||
import io.kestra.core.server.ServiceInstance;
|
||||
import jakarta.inject.Singleton;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
|
||||
@Singleton
|
||||
@MemoryRepositoryEnabled
|
||||
public class MemoryServiceInstanceRepository implements ServiceInstanceRepositoryInterface {
|
||||
|
||||
private final Map<String, ServiceInstance> data = new HashMap<>();
|
||||
|
||||
/** {@inheritDoc} **/
|
||||
@Override
|
||||
public Optional<ServiceInstance> findById(String id) {
|
||||
return Optional.ofNullable(data.get(id));
|
||||
}
|
||||
|
||||
/** {@inheritDoc} **/
|
||||
@Override
|
||||
public List<ServiceInstance> findAll() {
|
||||
return new ArrayList<>(data.values());
|
||||
}
|
||||
|
||||
/** {@inheritDoc} **/
|
||||
@Override
|
||||
public void delete(ServiceInstance service) {
|
||||
data.remove(service.id());
|
||||
}
|
||||
|
||||
/** {@inheritDoc} **/
|
||||
@Override
|
||||
public ServiceInstance save(ServiceInstance service) {
|
||||
data.put(service.id(), service);
|
||||
return service;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} **/
|
||||
@Override
|
||||
public List<ServiceInstance> findAllInstancesInState(Service.ServiceState state) {
|
||||
List<ServiceInstance> instancesInState = new ArrayList<>();
|
||||
for (ServiceInstance instance : data.values()) {
|
||||
if (instance.state() == state) {
|
||||
instancesInState.add(instance);
|
||||
}
|
||||
}
|
||||
return instancesInState;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} **/
|
||||
@Override
|
||||
public List<ServiceInstance> findAllInstancesInStates(List<Service.ServiceState> states) {
|
||||
List<ServiceInstance> instancesInStates = new ArrayList<>();
|
||||
for (ServiceInstance instance : data.values()) {
|
||||
if (states.contains(instance.state())) {
|
||||
instancesInStates.add(instance);
|
||||
}
|
||||
}
|
||||
return instancesInStates;
|
||||
}
|
||||
}
|
||||
@@ -3,3 +3,6 @@ kestra:
|
||||
type: memory
|
||||
repository:
|
||||
type: memory
|
||||
worker:
|
||||
liveness:
|
||||
enabled: false
|
||||
@@ -32,11 +32,6 @@ public class MemoryWorkerJobQueue implements WorkerJobQueueInterface {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cleanup() {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
|
||||
|
||||
@@ -7,3 +7,6 @@ kestra:
|
||||
type: local
|
||||
local:
|
||||
base-path: /tmp/unittest
|
||||
worker:
|
||||
liveness:
|
||||
enabled: false
|
||||
|
||||
76
ui/src/components/admin/Services.vue
Normal file
76
ui/src/components/admin/Services.vue
Normal file
@@ -0,0 +1,76 @@
|
||||
<template>
|
||||
<top-nav-bar :title="routeInfo.title" />
|
||||
<section class="container">
|
||||
<nav>
|
||||
<collapse>
|
||||
<el-form-item>
|
||||
<refresh-button class="float-right" @refresh="loadData" />
|
||||
</el-form-item>
|
||||
</collapse>
|
||||
</nav>
|
||||
<el-table
|
||||
:data="services"
|
||||
ref="table"
|
||||
:default-sort="{prop: 'hostname', order: 'ascending'}"
|
||||
stripe
|
||||
table-layout="auto"
|
||||
fixed
|
||||
>
|
||||
<el-table-column prop="id" sortable :sort-orders="['ascending', 'descending']" :label="$t('id')">
|
||||
<template #default="scope">
|
||||
<id :value="scope.row.id" :shrink="true" />
|
||||
</template>
|
||||
</el-table-column>
|
||||
<el-table-column prop="type" sortable :sort-orders="['ascending', 'descending']" :label="$t('type')" />
|
||||
<el-table-column prop="state" sortable :sort-orders="['ascending', 'descending']" :label="$t('state')" />
|
||||
<el-table-column prop="server.hostname" sortable :sort-orders="['ascending', 'descending']" :label="$t('hostname')" />
|
||||
<el-table-column prop="server.type" sortable :sort-orders="['ascending', 'descending']" :label="$t('server type')" />
|
||||
<el-table-column prop="server.version" sortable :sort-orders="['ascending', 'descending']" :label="$t('version')" />
|
||||
<el-table-column prop="createdAt" sortable :sort-orders="['ascending', 'descending']" :label="$t('started date')">
|
||||
<template #default="scope">
|
||||
<date-ago class-name="text-muted small" :inverted="true" :date="scope.row.createdAt" />
|
||||
</template>
|
||||
</el-table-column>
|
||||
<el-table-column prop="updatedAt" sortable :sort-orders="['ascending', 'descending']" :label="$t('healthcheck date')">
|
||||
<template #default="scope">
|
||||
<date-ago class-name="text-muted small" :inverted="true" :date="scope.row.updatedAt" />
|
||||
</template>
|
||||
</el-table-column>
|
||||
</el-table>
|
||||
</section>
|
||||
</template>
|
||||
<script>
|
||||
import RouteContext from "../../mixins/routeContext";
|
||||
import TopNavBar from "../../components/layout/TopNavBar.vue";
|
||||
import RefreshButton from "../../components/layout/RefreshButton.vue";
|
||||
import Collapse from "../../components/layout/Collapse.vue";
|
||||
import DateAgo from "../layout/DateAgo.vue";
|
||||
import Id from "../Id.vue";
|
||||
|
||||
export default {
|
||||
mixins: [RouteContext],
|
||||
components: {DateAgo, RefreshButton, Collapse, TopNavBar, Id},
|
||||
data() {
|
||||
return {
|
||||
services: undefined,
|
||||
};
|
||||
},
|
||||
created() {
|
||||
this.loadData();
|
||||
},
|
||||
methods: {
|
||||
loadData() {
|
||||
this.$store.dispatch("service/findAll").then(services => {
|
||||
this.services = services;
|
||||
});
|
||||
}
|
||||
},
|
||||
computed: {
|
||||
routeInfo() {
|
||||
return {
|
||||
title: this.$t("services")
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
</script>
|
||||
@@ -1,64 +0,0 @@
|
||||
<template>
|
||||
<top-nav-bar :title="routeInfo.title" />
|
||||
<section class="container">
|
||||
<nav>
|
||||
<collapse>
|
||||
<el-form-item>
|
||||
<refresh-button class="float-right" @refresh="loadData" />
|
||||
</el-form-item>
|
||||
</collapse>
|
||||
</nav>
|
||||
<el-table
|
||||
:data="workers"
|
||||
ref="table"
|
||||
:default-sort="{prop: 'hostname', order: 'ascending'}"
|
||||
stripe
|
||||
table-layout="auto"
|
||||
fixed
|
||||
>
|
||||
<el-table-column prop="workerUuid" sortable :sort-orders="['ascending', 'descending']" :label="$t('id')" />
|
||||
<el-table-column prop="hostname" sortable :sort-orders="['ascending', 'descending']" :label="$t('hostname')" />
|
||||
<el-table-column prop="workerGroup" sortable :sort-orders="['ascending', 'descending']" :label="$t('worker group')" />
|
||||
<el-table-column prop="status" sortable :sort-orders="['ascending', 'descending']" :label="$t('state')" />
|
||||
<el-table-column prop="heartbeatDate" sortable :sort-orders="['ascending', 'descending']" :label="$t('date')">
|
||||
<template #default="scope">
|
||||
<date-ago class-name="text-muted small" :inverted="true" :date="scope.row.heartbeatDate" />
|
||||
</template>
|
||||
</el-table-column>
|
||||
</el-table>
|
||||
</section>
|
||||
</template>
|
||||
<script>
|
||||
import RouteContext from "../../mixins/routeContext";
|
||||
import TopNavBar from "../../components/layout/TopNavBar.vue";
|
||||
import RefreshButton from "../../components/layout/RefreshButton.vue";
|
||||
import Collapse from "../../components/layout/Collapse.vue";
|
||||
import DateAgo from "../layout/DateAgo.vue";
|
||||
|
||||
export default {
|
||||
mixins: [RouteContext],
|
||||
components: {DateAgo, RefreshButton, Collapse, TopNavBar},
|
||||
data() {
|
||||
return {
|
||||
workers: undefined,
|
||||
};
|
||||
},
|
||||
created() {
|
||||
this.loadData();
|
||||
},
|
||||
methods: {
|
||||
loadData() {
|
||||
this.$store.dispatch("worker/findAll").then(workers => {
|
||||
this.workers = workers;
|
||||
});
|
||||
}
|
||||
},
|
||||
computed: {
|
||||
routeInfo() {
|
||||
return {
|
||||
title: this.$t("workers")
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
</script>
|
||||
@@ -188,9 +188,9 @@
|
||||
}
|
||||
},
|
||||
{
|
||||
href: {name: "admin/workers"},
|
||||
routes: this.routeStartWith("admin/workers"),
|
||||
title: this.$t("workers"),
|
||||
href: {name: "admin/cluster/services"},
|
||||
routes: this.routeStartWith("admin/cluster/services"),
|
||||
title: this.$t("cluster"),
|
||||
icon: {
|
||||
element: shallowRef(ServerOutline),
|
||||
class: "menu-icon"
|
||||
|
||||
@@ -16,7 +16,7 @@ import FlowCreate from "../components/flows/FlowCreate.vue";
|
||||
import Blueprints from "override/components/flows/blueprints/Blueprints.vue";
|
||||
import BlueprintDetail from "../components/flows/blueprints/BlueprintDetail.vue";
|
||||
import Triggers from "../components/admin/Triggers.vue";
|
||||
import Workers from "../components/admin/Workers.vue";
|
||||
import Services from "../components/admin/Services.vue";
|
||||
import Editor from "../components/namespace/Editor.vue";
|
||||
import Stats from "override/components/admin/stats/Stats.vue";
|
||||
|
||||
@@ -65,7 +65,7 @@ export default [
|
||||
|
||||
//Admin
|
||||
{name: "admin/triggers", path: "/:tenant?/admin/triggers", component: Triggers},
|
||||
{name: "admin/workers", path: "/:tenant?/admin/workers", component: Workers},
|
||||
{name: "admin/cluster/services", path: "/:tenant?/admin/services", component: Services},
|
||||
{name: "admin/stats", path: "/:tenant?/admin/stats", component: Stats},
|
||||
|
||||
//Errors
|
||||
|
||||
@@ -5,7 +5,7 @@ export default {
|
||||
|
||||
actions: {
|
||||
findAll(_, __) {
|
||||
return this.$http.get(`${apiUrl(this)}/workers`).then(response => {
|
||||
return this.$http.get(`${apiUrl(this)}/cluster/services`).then(response => {
|
||||
return response.data;
|
||||
})
|
||||
}
|
||||
@@ -14,7 +14,7 @@ import stat from "./stat"
|
||||
import template from "./template"
|
||||
import taskrun from "./taskruns"
|
||||
import trigger from "./trigger";
|
||||
import worker from "./worker";
|
||||
import service from "./service";
|
||||
|
||||
export default {
|
||||
modules: {
|
||||
@@ -34,6 +34,6 @@ export default {
|
||||
plugin,
|
||||
taskrun,
|
||||
trigger,
|
||||
worker
|
||||
service
|
||||
}
|
||||
}
|
||||
|
||||
@@ -569,6 +569,10 @@
|
||||
"flow deleted, you can restore it": "The flow has been deleted and this is a read-only view. You can still restore it.",
|
||||
"workers": "Workers",
|
||||
"worker group": "Worker Group",
|
||||
"version": "Version",
|
||||
"services": "Services",
|
||||
"cluster": "Cluster",
|
||||
"server type": "Server type",
|
||||
"hostname": "Hostname",
|
||||
"port": "Port",
|
||||
"management port": "Management port",
|
||||
|
||||
@@ -87,6 +87,8 @@
|
||||
},
|
||||
"created date": "Date de création",
|
||||
"updated date": "Date de mise à jour",
|
||||
"started date": "Date de démarrage",
|
||||
"healthcheck date": "Dernier Healthcheck",
|
||||
"next execution date": "Prochaine date d'exécution",
|
||||
"jump to...": "Aller à...",
|
||||
"source": "Source",
|
||||
|
||||
@@ -0,0 +1,76 @@
|
||||
package io.kestra.webserver.controllers;
|
||||
|
||||
import io.kestra.core.repositories.ServiceInstanceRepositoryInterface;
|
||||
import io.kestra.core.server.ServerInstance;
|
||||
import io.kestra.core.server.Service;
|
||||
import io.kestra.core.server.ServiceInstance;
|
||||
import io.micronaut.context.annotation.Requires;
|
||||
import io.micronaut.http.HttpResponse;
|
||||
import io.micronaut.http.annotation.Controller;
|
||||
import io.micronaut.http.annotation.Get;
|
||||
import io.micronaut.http.exceptions.HttpStatusException;
|
||||
import io.micronaut.scheduling.TaskExecutors;
|
||||
import io.micronaut.scheduling.annotation.ExecuteOn;
|
||||
import io.swagger.v3.oas.annotations.Operation;
|
||||
import jakarta.inject.Inject;
|
||||
|
||||
import java.time.Instant;
|
||||
import java.util.List;
|
||||
|
||||
import static io.kestra.core.server.Service.ServiceState.EMPTY;
|
||||
import static io.kestra.core.server.Service.ServiceState.NOT_RUNNING;
|
||||
|
||||
@Controller("/api/v1/cluster")
|
||||
@Requires(bean = ServiceInstanceRepositoryInterface.class)
|
||||
public class ClusterController {
|
||||
|
||||
private final ServiceInstanceRepositoryInterface repository;
|
||||
|
||||
@Inject
|
||||
public ClusterController(final ServiceInstanceRepositoryInterface repository) {
|
||||
this.repository = repository;
|
||||
}
|
||||
|
||||
@ExecuteOn(TaskExecutors.IO)
|
||||
@Get("services")
|
||||
@Operation(tags = {"Services"}, summary = "Get all the services for teh Kestra cluster")
|
||||
public HttpResponse<List<ApiServiceInstance>> findAll() throws HttpStatusException {
|
||||
List<ServiceInstance> instances = repository.findAll();
|
||||
List<ApiServiceInstance> entities = instances.stream()
|
||||
.filter(it -> {
|
||||
Service.ServiceState state = it.state();
|
||||
return !(state.equals(EMPTY) || state.equals(NOT_RUNNING));
|
||||
})
|
||||
.map(it -> new ApiServiceInstance(
|
||||
it.id(),
|
||||
it.type(),
|
||||
it.state(),
|
||||
new ApiServerInstance(
|
||||
it.server().id(),
|
||||
it.server().type(),
|
||||
it.server().version(),
|
||||
it.server().hostname()
|
||||
),
|
||||
it.createdAt(),
|
||||
it.updatedAt()
|
||||
)
|
||||
).toList();
|
||||
return HttpResponse.ok(entities);
|
||||
}
|
||||
|
||||
public record ApiServerInstance(
|
||||
String id,
|
||||
ServerInstance.Type type,
|
||||
String version,
|
||||
String hostname
|
||||
){}
|
||||
|
||||
public record ApiServiceInstance(
|
||||
String id,
|
||||
Service.ServiceType type,
|
||||
Service.ServiceState state,
|
||||
ApiServerInstance server,
|
||||
Instant createdAt,
|
||||
Instant updatedAt
|
||||
){}
|
||||
}
|
||||
@@ -1,29 +0,0 @@
|
||||
package io.kestra.webserver.controllers;
|
||||
|
||||
import io.kestra.core.repositories.WorkerInstanceRepositoryInterface;
|
||||
import io.kestra.core.runners.WorkerInstance;
|
||||
import io.micronaut.context.annotation.Requires;
|
||||
import io.micronaut.http.MediaType;
|
||||
import io.micronaut.http.annotation.Controller;
|
||||
import io.micronaut.http.annotation.Get;
|
||||
import io.micronaut.http.exceptions.HttpStatusException;
|
||||
import io.micronaut.scheduling.TaskExecutors;
|
||||
import io.micronaut.scheduling.annotation.ExecuteOn;
|
||||
import io.swagger.v3.oas.annotations.Operation;
|
||||
import jakarta.inject.Inject;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
@Controller("/api/v1/workers")
|
||||
@Requires(bean = WorkerInstanceRepositoryInterface.class)
|
||||
public class WorkerInstanceController {
|
||||
@Inject
|
||||
private WorkerInstanceRepositoryInterface workerInstanceRepositoryInterface;
|
||||
|
||||
@ExecuteOn(TaskExecutors.IO)
|
||||
@Get
|
||||
@Operation(tags = {"Workers"}, summary = "Get all workers")
|
||||
public List<WorkerInstance> findAll() throws HttpStatusException {
|
||||
return workerInstanceRepositoryInterface.findAll();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,74 @@
|
||||
package io.kestra.webserver.services;
|
||||
|
||||
import io.kestra.core.server.Service;
|
||||
import io.kestra.core.server.ServiceStateChangeEvent;
|
||||
import io.kestra.core.utils.IdUtils;
|
||||
import io.micronaut.context.annotation.Context;
|
||||
import io.micronaut.context.annotation.Requires;
|
||||
import io.micronaut.context.event.ApplicationEventPublisher;
|
||||
import io.micronaut.runtime.event.annotation.EventListener;
|
||||
import io.micronaut.runtime.server.event.ServerShutdownEvent;
|
||||
import io.micronaut.runtime.server.event.ServerStartupEvent;
|
||||
import jakarta.annotation.PostConstruct;
|
||||
import jakarta.annotation.PreDestroy;
|
||||
import jakarta.inject.Inject;
|
||||
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
/**
|
||||
* This class is used for service registration and liveness.
|
||||
*/
|
||||
@Context
|
||||
@Requires(property = "kestra.server-type", pattern = "(WEBSERVER|STANDALONE)")
|
||||
public final class WebserverService implements Service {
|
||||
|
||||
private final String id = IdUtils.create();
|
||||
|
||||
@Inject
|
||||
private ApplicationEventPublisher<ServiceStateChangeEvent> eventPublisher;
|
||||
|
||||
private final AtomicReference<ServiceState> state = new AtomicReference<>();
|
||||
|
||||
private void setState(final ServiceState state) {
|
||||
this.state.set(state);
|
||||
eventPublisher.publishEvent(new ServiceStateChangeEvent(this));
|
||||
}
|
||||
|
||||
/** {@inheritDoc} **/
|
||||
@Override
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} **/
|
||||
@Override
|
||||
public ServiceType getType() {
|
||||
return ServiceType.WEBSERVER;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} **/
|
||||
@Override
|
||||
public ServiceState getState() {
|
||||
return state.get();
|
||||
}
|
||||
|
||||
@PostConstruct
|
||||
public void postConstruct() {
|
||||
setState(ServiceState.CREATED);
|
||||
}
|
||||
|
||||
@EventListener
|
||||
public void onServerStartup(ServerStartupEvent event) {
|
||||
setState(ServiceState.RUNNING);
|
||||
}
|
||||
|
||||
@EventListener
|
||||
public void onServeShutdown(ServerShutdownEvent event) {
|
||||
setState(ServiceState.TERMINATING);
|
||||
}
|
||||
|
||||
@PreDestroy
|
||||
public void preDestroy() {
|
||||
setState(ServiceState.TERMINATED_GRACEFULLY);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,73 @@
|
||||
package io.kestra.webserver.controllers;
|
||||
|
||||
import io.kestra.core.repositories.ServiceInstanceRepositoryInterface;
|
||||
import io.kestra.core.server.Service;
|
||||
import io.kestra.core.server.ServiceInstance;
|
||||
import io.kestra.core.server.ServiceInstanceFactory;
|
||||
import io.kestra.core.utils.IdUtils;
|
||||
import io.kestra.webserver.controllers.h2.JdbcH2ControllerTest;
|
||||
import io.micronaut.core.type.Argument;
|
||||
import io.micronaut.http.HttpRequest;
|
||||
import io.micronaut.http.client.annotation.Client;
|
||||
import io.micronaut.reactor.http.client.ReactorHttpClient;
|
||||
import jakarta.inject.Inject;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.hamcrest.Matchers.in;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
|
||||
class ClusterControllerTest extends JdbcH2ControllerTest {
|
||||
|
||||
@Inject
|
||||
@Client("/")
|
||||
ReactorHttpClient client;
|
||||
|
||||
@Inject
|
||||
ServiceInstanceRepositoryInterface jdbcWorkerInstanceRepository;
|
||||
|
||||
@Inject
|
||||
ServiceInstanceFactory serviceInstanceFactory;
|
||||
|
||||
@BeforeEach
|
||||
protected void init() {
|
||||
super.setup();
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Test
|
||||
void list() {
|
||||
Service service = new Service() {
|
||||
@Override
|
||||
public String getId() {
|
||||
return IdUtils.create();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServiceType getType() {
|
||||
return ServiceType.WORKER;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServiceState getState() {
|
||||
return ServiceState.CREATED;
|
||||
}
|
||||
};
|
||||
ServiceInstance instance = serviceInstanceFactory.newServiceInstance(service, Map.of());
|
||||
|
||||
jdbcWorkerInstanceRepository.save(instance);
|
||||
|
||||
List<ServiceInstance> find = client.toBlocking().retrieve(
|
||||
HttpRequest.GET("/api/v1/cluster/services"),
|
||||
Argument.of(List.class, ServiceInstance.class)
|
||||
);
|
||||
|
||||
Assertions.assertFalse(find.isEmpty());
|
||||
Assertions.assertTrue(find.stream().anyMatch(it -> it.id().equalsIgnoreCase(instance.id())));
|
||||
}
|
||||
}
|
||||
@@ -1,54 +0,0 @@
|
||||
package io.kestra.webserver.controllers;
|
||||
|
||||
import io.kestra.core.runners.WorkerInstance;
|
||||
import io.kestra.jdbc.repository.AbstractJdbcWorkerInstanceRepository;
|
||||
import io.kestra.webserver.controllers.h2.JdbcH2ControllerTest;
|
||||
import io.micronaut.core.type.Argument;
|
||||
import io.micronaut.http.HttpRequest;
|
||||
import io.micronaut.http.client.annotation.Client;
|
||||
import io.micronaut.reactor.http.client.ReactorHttpClient;
|
||||
import jakarta.inject.Inject;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
|
||||
class WorkerInstanceControllerTest extends JdbcH2ControllerTest {
|
||||
@Inject
|
||||
@Client("/")
|
||||
ReactorHttpClient client;
|
||||
|
||||
@Inject
|
||||
AbstractJdbcWorkerInstanceRepository jdbcWorkerInstanceRepository;
|
||||
|
||||
@BeforeEach
|
||||
protected void init() {
|
||||
super.setup();
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Test
|
||||
void list() {
|
||||
WorkerInstance workerInstance = WorkerInstance.builder()
|
||||
.workerUuid(UUID.randomUUID())
|
||||
.workerGroup(null)
|
||||
.managementPort(0)
|
||||
.hostname("kestra.io")
|
||||
.partitions(null)
|
||||
.port(0)
|
||||
.status(WorkerInstance.Status.UP)
|
||||
.build();
|
||||
|
||||
|
||||
jdbcWorkerInstanceRepository.save(workerInstance);
|
||||
|
||||
List<WorkerInstance> find = client.toBlocking().retrieve(HttpRequest.GET("/api/v1/workers"), Argument.of(List.class, WorkerInstance.class));
|
||||
assertThat(find.size(), is(1));
|
||||
assertThat(find.get(0).getWorkerUuid(), is(workerInstance.getWorkerUuid()));
|
||||
}
|
||||
|
||||
}
|
||||
@@ -32,9 +32,6 @@ kestra:
|
||||
type: local
|
||||
local:
|
||||
base-path: /tmp/unittest
|
||||
heartbeat:
|
||||
frequency: 10s
|
||||
heartbeat-missed: 3
|
||||
anonymous-usage-report:
|
||||
enabled: true
|
||||
uri: https://api.kestra.io/v1/reports/usages
|
||||
@@ -50,6 +47,8 @@ kestra:
|
||||
open-urls:
|
||||
- "/ping"
|
||||
- "/api/v1/executions/webhook/"
|
||||
liveness:
|
||||
enabled: false
|
||||
queue:
|
||||
type: h2
|
||||
repository:
|
||||
@@ -94,16 +93,15 @@ kestra:
|
||||
flowtopologies:
|
||||
table: "flow_topologies"
|
||||
cls: io.kestra.core.models.topologies.FlowTopology
|
||||
workerinstance:
|
||||
cls: io.kestra.core.runners.WorkerInstance
|
||||
table: "worker_instance"
|
||||
serviceinstance:
|
||||
cls: io.kestra.core.server.ServiceInstance
|
||||
table: "service_instance"
|
||||
workerjobrunning:
|
||||
cls: io.kestra.core.runners.WorkerJobRunning
|
||||
table: "worker_job_running"
|
||||
executionqueued:
|
||||
table: "execution_queued"
|
||||
cls: io.kestra.core.runners.ExecutionQueued
|
||||
|
||||
datasources:
|
||||
h2:
|
||||
url: jdbc:h2:mem:public;DB_CLOSE_DELAY=-1;DB_CLOSE_ON_EXIT=FALSE
|
||||
|
||||
Reference in New Issue
Block a user