diff --git a/.env b/.env index 23e2c1b1402e6..d8ab738b59fd4 100644 --- a/.env +++ b/.env @@ -81,6 +81,8 @@ LOG_LEVEL=INFO ### APPLICATIONS ### # Worker # WORKERS_MICRONAUT_ENVIRONMENTS=control-plane +# Cron # +CRON_MICRONAUT_ENVIRONMENTS=control-plane # Relevant to scaling. MAX_SYNC_WORKERS=5 MAX_SPEC_WORKERS=5 @@ -102,4 +104,4 @@ METRIC_CLIENT= # Useful only when metric client is set to be otel. Must start with http:// or https://. OTEL_COLLECTOR_ENDPOINT="http://host.docker.internal:4317" -USE_STREAM_CAPABLE_STATE=true \ No newline at end of file +USE_STREAM_CAPABLE_STATE=true diff --git a/.env.dev b/.env.dev index 687fbe07b5dc3..3fa990232cb80 100644 --- a/.env.dev +++ b/.env.dev @@ -26,6 +26,7 @@ INTERNAL_API_HOST=airbyte-server:8001 SYNC_JOB_MAX_ATTEMPTS=3 SYNC_JOB_MAX_TIMEOUT_DAYS=3 WORKERS_MICRONAUT_ENVIRONMENTS=control-plane +CRON_MICRONAUT_ENVIRONMENTS=control-plane # Sentry SENTRY_DSN="" diff --git a/airbyte-commons-temporal/build.gradle b/airbyte-commons-temporal/build.gradle index 04101720844c6..3140b4272ba59 100644 --- a/airbyte-commons-temporal/build.gradle +++ b/airbyte-commons-temporal/build.gradle @@ -1,8 +1,5 @@ -import org.jsonschema2pojo.SourceType - plugins { id "java-library" - id 'com.github.eirnym.js2p' version '1.0' } dependencies { @@ -18,27 +15,16 @@ dependencies { testAnnotationProcessor platform(libs.micronaut.bom) testAnnotationProcessor libs.bundles.micronaut.test.annotation.processor - implementation project(':airbyte-config:config-models') + implementation project(':airbyte-config:config-persistence') implementation project(':airbyte-metrics:metrics-lib') + implementation project(':airbyte-persistence:job-persistence') + implementation project(':airbyte-protocol:protocol-models') + implementation project(':airbyte-worker-models') testImplementation 'io.temporal:temporal-testing:1.8.1' // Needed to be able to mock final class testImplementation 'org.mockito:mockito-inline:4.7.0' } -jsonSchema2Pojo { - sourceType = SourceType.YAMLSCHEMA - source = files("${sourceSets.main.output.resourcesDir}/workers_models") - targetDirectory = new File(project.buildDir, 'generated/src/gen/java/') - removeOldOutput = true - - targetPackage = 'io.airbyte.persistence.job.models' - - useLongIntegers = true - generateBuilders = true - includeConstructors = false - includeSetters = true -} - Task publishArtifactsTask = getPublishArtifactsTask("$rootProject.ext.version", project) diff --git a/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/ConnectionManagerUtils.java b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/ConnectionManagerUtils.java index 178c17151f159..47ccaa36ab1ff 100644 --- a/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/ConnectionManagerUtils.java +++ b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/ConnectionManagerUtils.java @@ -4,11 +4,24 @@ package io.airbyte.commons.temporal; +import io.airbyte.commons.temporal.exception.DeletedWorkflowException; +import io.airbyte.commons.temporal.exception.UnreachableWorkflowException; import io.airbyte.commons.temporal.scheduling.ConnectionManagerWorkflow; import io.airbyte.commons.temporal.scheduling.ConnectionUpdaterInput; +import io.airbyte.commons.temporal.scheduling.state.WorkflowState; +import io.temporal.api.common.v1.WorkflowExecution; +import io.temporal.api.enums.v1.WorkflowExecutionStatus; +import io.temporal.api.workflowservice.v1.DescribeWorkflowExecutionRequest; +import io.temporal.api.workflowservice.v1.DescribeWorkflowExecutionResponse; +import io.temporal.client.BatchRequest; import io.temporal.client.WorkflowClient; +import io.temporal.workflow.Functions.Proc; +import io.temporal.workflow.Functions.Proc1; +import io.temporal.workflow.Functions.TemporalFunctionalInterfaceMarker; import jakarta.inject.Singleton; +import java.util.Optional; import java.util.UUID; +import java.util.function.Function; import lombok.NoArgsConstructor; import lombok.extern.slf4j.Slf4j; @@ -17,6 +30,105 @@ @Slf4j public class ConnectionManagerUtils { + /** + * Attempts to send a signal to the existing ConnectionManagerWorkflow for the provided connection. + * + * If the workflow is unreachable, this will restart the workflow and send the signal in a single + * batched request. Batching is used to avoid race conditions between starting the workflow and + * executing the signal. + * + * @param client the WorkflowClient for interacting with temporal + * @param connectionId the connection ID to execute this operation for + * @param signalMethod a function that takes in a connection manager workflow and executes a signal + * method on it, with no arguments + * @return the healthy connection manager workflow that was signaled + * @throws DeletedWorkflowException if the connection manager workflow was deleted + */ + public ConnectionManagerWorkflow signalWorkflowAndRepairIfNecessary(final WorkflowClient client, + final UUID connectionId, + final Function signalMethod) + throws DeletedWorkflowException { + return signalWorkflowAndRepairIfNecessary(client, connectionId, signalMethod, Optional.empty()); + } + + /** + * Attempts to send a signal to the existing ConnectionManagerWorkflow for the provided connection. + * + * If the workflow is unreachable, this will restart the workflow and send the signal in a single + * batched request. Batching is used to avoid race conditions between starting the workflow and + * executing the signal. + * + * @param client the WorkflowClient for interacting with temporal + * @param connectionId the connection ID to execute this operation for + * @param signalMethod a function that takes in a connection manager workflow and executes a signal + * method on it, with 1 argument + * @param signalArgument the single argument to be input to the signal + * @return the healthy connection manager workflow that was signaled + * @throws DeletedWorkflowException if the connection manager workflow was deleted + */ + public ConnectionManagerWorkflow signalWorkflowAndRepairIfNecessary(final WorkflowClient client, + final UUID connectionId, + final Function> signalMethod, + final T signalArgument) + throws DeletedWorkflowException { + return signalWorkflowAndRepairIfNecessary(client, connectionId, signalMethod, Optional.of(signalArgument)); + } + + // This method unifies the logic of the above two, by using the optional signalArgument parameter to + // indicate if an argument is being provided to the signal or not. + // Keeping this private and only exposing the above methods outside this class provides a strict + // type enforcement for external calls, and means this method can assume consistent type + // implementations for both cases. + private ConnectionManagerWorkflow signalWorkflowAndRepairIfNecessary(final WorkflowClient client, + final UUID connectionId, + final Function signalMethod, + final Optional signalArgument) + throws DeletedWorkflowException { + try { + final ConnectionManagerWorkflow connectionManagerWorkflow = getConnectionManagerWorkflow(client, connectionId); + log.info("Retrieved existing connection manager workflow for connection {}. Executing signal.", connectionId); + // retrieve the signal from the lambda + final TemporalFunctionalInterfaceMarker signal = signalMethod.apply(connectionManagerWorkflow); + // execute the signal + if (signalArgument.isPresent()) { + ((Proc1) signal).apply(signalArgument.get()); + } else { + ((Proc) signal).apply(); + } + return connectionManagerWorkflow; + } catch (final UnreachableWorkflowException e) { + log.error( + String.format( + "Failed to retrieve ConnectionManagerWorkflow for connection %s. Repairing state by creating new workflow and starting with the signal.", + connectionId), + e); + + // in case there is an existing workflow in a bad state, attempt to terminate it first before + // starting a new workflow + safeTerminateWorkflow(client, connectionId, "Terminating workflow in unreachable state before starting a new workflow for this connection"); + + final ConnectionManagerWorkflow connectionManagerWorkflow = newConnectionManagerWorkflowStub(client, connectionId); + final ConnectionUpdaterInput startWorkflowInput = TemporalWorkflowUtils.buildStartWorkflowInput(connectionId); + + final BatchRequest batchRequest = client.newSignalWithStartRequest(); + batchRequest.add(connectionManagerWorkflow::run, startWorkflowInput); + + // retrieve the signal from the lambda + final TemporalFunctionalInterfaceMarker signal = signalMethod.apply(connectionManagerWorkflow); + // add signal to batch request + if (signalArgument.isPresent()) { + batchRequest.add((Proc1) signal, signalArgument.get()); + } else { + batchRequest.add((Proc) signal); + } + + client.signalWithStart(batchRequest); + log.info("Connection manager workflow for connection {} has been started and signaled.", connectionId); + + return connectionManagerWorkflow; + } + } + void safeTerminateWorkflow(final WorkflowClient client, final String workflowId, final String reason) { log.info("Attempting to terminate existing workflow for workflowId {}.", workflowId); try { @@ -33,10 +145,6 @@ public void safeTerminateWorkflow(final WorkflowClient client, final UUID connec safeTerminateWorkflow(client, getConnectionManagerName(connectionId), reason); } - public String getConnectionManagerName(final UUID connectionId) { - return "connection_manager_" + connectionId; - } - public ConnectionManagerWorkflow startConnectionManagerNoSignal(final WorkflowClient client, final UUID connectionId) { final ConnectionManagerWorkflow connectionManagerWorkflow = newConnectionManagerWorkflowStub(client, connectionId); final ConnectionUpdaterInput input = TemporalWorkflowUtils.buildStartWorkflowInput(connectionId); @@ -45,9 +153,89 @@ public ConnectionManagerWorkflow startConnectionManagerNoSignal(final WorkflowCl return connectionManagerWorkflow; } + /** + * Attempts to retrieve the connection manager workflow for the provided connection. + * + * @param connectionId the ID of the connection whose workflow should be retrieved + * @return the healthy ConnectionManagerWorkflow + * @throws DeletedWorkflowException if the workflow was deleted, according to the workflow state + * @throws UnreachableWorkflowException if the workflow is in an unreachable state + */ + public ConnectionManagerWorkflow getConnectionManagerWorkflow(final WorkflowClient client, final UUID connectionId) + throws DeletedWorkflowException, UnreachableWorkflowException { + + final ConnectionManagerWorkflow connectionManagerWorkflow; + final WorkflowState workflowState; + final WorkflowExecutionStatus workflowExecutionStatus; + try { + connectionManagerWorkflow = client.newWorkflowStub(ConnectionManagerWorkflow.class, getConnectionManagerName(connectionId)); + workflowState = connectionManagerWorkflow.getState(); + workflowExecutionStatus = getConnectionManagerWorkflowStatus(client, connectionId); + } catch (final Exception e) { + throw new UnreachableWorkflowException( + String.format("Failed to retrieve ConnectionManagerWorkflow for connection %s due to the following error:", connectionId), + e); + } + + if (WorkflowExecutionStatus.WORKFLOW_EXECUTION_STATUS_COMPLETED.equals(workflowExecutionStatus)) { + if (workflowState.isDeleted()) { + throw new DeletedWorkflowException(String.format( + "The connection manager workflow for connection %s is deleted, so no further operations cannot be performed on it.", + connectionId)); + } + + // A non-deleted workflow being in a COMPLETED state is unexpected, and should be corrected + throw new UnreachableWorkflowException( + String.format("ConnectionManagerWorkflow for connection %s is unreachable due to having COMPLETED status.", connectionId)); + } + + if (workflowState.isQuarantined()) { + throw new UnreachableWorkflowException( + String.format("ConnectionManagerWorkflow for connection %s is unreachable due to being in a quarantined state.", connectionId)); + } + + return connectionManagerWorkflow; + } + + boolean isWorkflowStateRunning(final WorkflowClient client, final UUID connectionId) { + try { + final ConnectionManagerWorkflow connectionManagerWorkflow = client.newWorkflowStub(ConnectionManagerWorkflow.class, + getConnectionManagerName(connectionId)); + return connectionManagerWorkflow.getState().isRunning(); + } catch (final Exception e) { + return false; + } + } + + public WorkflowExecutionStatus getConnectionManagerWorkflowStatus(final WorkflowClient workflowClient, final UUID connectionId) { + final DescribeWorkflowExecutionRequest describeWorkflowExecutionRequest = DescribeWorkflowExecutionRequest.newBuilder() + .setExecution(WorkflowExecution.newBuilder() + .setWorkflowId(getConnectionManagerName(connectionId)) + .build()) + .setNamespace(workflowClient.getOptions().getNamespace()).build(); + + final DescribeWorkflowExecutionResponse describeWorkflowExecutionResponse = workflowClient.getWorkflowServiceStubs().blockingStub() + .describeWorkflowExecution(describeWorkflowExecutionRequest); + + return describeWorkflowExecutionResponse.getWorkflowExecutionInfo().getStatus(); + } + + public long getCurrentJobId(final WorkflowClient client, final UUID connectionId) { + try { + final ConnectionManagerWorkflow connectionManagerWorkflow = getConnectionManagerWorkflow(client, connectionId); + return connectionManagerWorkflow.getJobInformation().getJobId(); + } catch (final Exception e) { + return ConnectionManagerWorkflow.NON_RUNNING_JOB_ID; + } + } + public ConnectionManagerWorkflow newConnectionManagerWorkflowStub(final WorkflowClient client, final UUID connectionId) { return client.newWorkflowStub(ConnectionManagerWorkflow.class, TemporalWorkflowUtils.buildWorkflowOptions(TemporalJobType.CONNECTION_UPDATER, getConnectionManagerName(connectionId))); } + public String getConnectionManagerName(final UUID connectionId) { + return "connection_manager_" + connectionId; + } + } diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/ErrorCode.java b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/ErrorCode.java similarity index 79% rename from airbyte-workers/src/main/java/io/airbyte/workers/temporal/ErrorCode.java rename to airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/ErrorCode.java index cd0a14b0c4c9c..2d276320919c2 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/ErrorCode.java +++ b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/ErrorCode.java @@ -2,7 +2,7 @@ * Copyright (c) 2022 Airbyte, Inc., all rights reserved. */ -package io.airbyte.workers.temporal; +package io.airbyte.commons.temporal; public enum ErrorCode { UNKNOWN, diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/JobMetadata.java b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/JobMetadata.java similarity index 96% rename from airbyte-workers/src/main/java/io/airbyte/workers/temporal/JobMetadata.java rename to airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/JobMetadata.java index 612efc0eb1cc7..9ee1328079c49 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/JobMetadata.java +++ b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/JobMetadata.java @@ -2,7 +2,7 @@ * Copyright (c) 2022 Airbyte, Inc., all rights reserved. */ -package io.airbyte.workers.temporal; +package io.airbyte.commons.temporal; import java.nio.file.Path; import java.util.Objects; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/StreamResetRecordsHelper.java b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/StreamResetRecordsHelper.java similarity index 95% rename from airbyte-workers/src/main/java/io/airbyte/workers/temporal/StreamResetRecordsHelper.java rename to airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/StreamResetRecordsHelper.java index e12149e806007..77c3e7d6ba540 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/StreamResetRecordsHelper.java +++ b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/StreamResetRecordsHelper.java @@ -2,14 +2,14 @@ * Copyright (c) 2022 Airbyte, Inc., all rights reserved. */ -package io.airbyte.workers.temporal; +package io.airbyte.commons.temporal; +import io.airbyte.commons.temporal.exception.RetryableException; import io.airbyte.config.JobConfig.ConfigType; import io.airbyte.config.persistence.StreamResetPersistence; import io.airbyte.persistence.job.JobPersistence; import io.airbyte.persistence.job.models.Job; import io.airbyte.protocol.models.StreamDescriptor; -import io.airbyte.workers.temporal.exception.RetryableException; import jakarta.inject.Singleton; import java.io.IOException; import java.util.List; diff --git a/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/TemporalClient.java b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/TemporalClient.java index 8f97e20c6308f..433a22bdb902b 100644 --- a/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/TemporalClient.java +++ b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/TemporalClient.java @@ -4,9 +4,31 @@ package io.airbyte.commons.temporal; +import static io.airbyte.commons.temporal.scheduling.ConnectionManagerWorkflow.NON_RUNNING_JOB_ID; + import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.ByteString; +import io.airbyte.commons.temporal.config.WorkerMode; +import io.airbyte.commons.temporal.exception.DeletedWorkflowException; +import io.airbyte.commons.temporal.exception.UnreachableWorkflowException; +import io.airbyte.commons.temporal.scheduling.CheckConnectionWorkflow; import io.airbyte.commons.temporal.scheduling.ConnectionManagerWorkflow; +import io.airbyte.commons.temporal.scheduling.DiscoverCatalogWorkflow; +import io.airbyte.commons.temporal.scheduling.SpecWorkflow; +import io.airbyte.commons.temporal.scheduling.SyncWorkflow; +import io.airbyte.config.ConnectorJobOutput; +import io.airbyte.config.JobCheckConnectionConfig; +import io.airbyte.config.JobDiscoverCatalogConfig; +import io.airbyte.config.JobGetSpecConfig; +import io.airbyte.config.JobSyncConfig; +import io.airbyte.config.StandardCheckConnectionInput; +import io.airbyte.config.StandardDiscoverCatalogInput; +import io.airbyte.config.StandardSyncInput; +import io.airbyte.config.StandardSyncOutput; +import io.airbyte.config.persistence.StreamResetPersistence; +import io.airbyte.persistence.job.models.IntegrationLauncherConfig; +import io.airbyte.persistence.job.models.JobRunConfig; +import io.airbyte.protocol.models.StreamDescriptor; import io.micronaut.context.annotation.Requires; import io.temporal.api.common.v1.WorkflowType; import io.temporal.api.enums.v1.WorkflowExecutionStatus; @@ -16,32 +38,59 @@ import io.temporal.api.workflowservice.v1.ListOpenWorkflowExecutionsResponse; import io.temporal.client.WorkflowClient; import io.temporal.serviceclient.WorkflowServiceStubs; -import jakarta.inject.Inject; +import jakarta.inject.Named; import jakarta.inject.Singleton; +import java.io.IOException; +import java.nio.file.Path; import java.util.HashSet; +import java.util.List; import java.util.Optional; import java.util.Set; import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.function.Supplier; import java.util.stream.Collectors; -import lombok.AllArgsConstructor; -import lombok.NoArgsConstructor; +import lombok.Builder; +import lombok.Value; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.time.StopWatch; -@AllArgsConstructor -@NoArgsConstructor @Slf4j @Singleton -@Requires(property = "airbyte.worker.plane", - notEquals = "DATA_PLANE") +@Requires(env = WorkerMode.CONTROL_PLANE) public class TemporalClient { - @Inject - private WorkflowClient client; - @Inject - private WorkflowServiceStubs service; - @Inject - private ConnectionManagerUtils connectionManagerUtils; + /** + * This is used to sleep between 2 temporal queries. The query is needed to ensure that the cancel + * and start manual sync methods wait before returning. Since temporal signals are async, we need to + * use the queries to make sure that we are in a state in which we want to continue with. + */ + private static final int DELAY_BETWEEN_QUERY_MS = 10; + + private final Path workspaceRoot; + private final WorkflowClient client; + private final WorkflowServiceStubs service; + private final StreamResetPersistence streamResetPersistence; + private final ConnectionManagerUtils connectionManagerUtils; + private final StreamResetRecordsHelper streamResetRecordsHelper; + + public TemporalClient(@Named("workspaceRootTemporal") final Path workspaceRoot, + final WorkflowClient client, + final WorkflowServiceStubs service, + final StreamResetPersistence streamResetPersistence, + final ConnectionManagerUtils connectionManagerUtils, + final StreamResetRecordsHelper streamResetRecordsHelper) { + this.workspaceRoot = workspaceRoot; + this.client = client; + this.service = service; + this.streamResetPersistence = streamResetPersistence; + this.connectionManagerUtils = connectionManagerUtils; + this.streamResetRecordsHelper = streamResetRecordsHelper; + } private final Set workflowNames = new HashSet<>(); @@ -132,4 +181,347 @@ Optional extractConnectionIdFromWorkflowId(final String workflowId) { stringUUID -> UUID.fromString(stringUUID)); } + @Value + @Builder + public static class ManualOperationResult { + + final Optional failingReason; + final Optional jobId; + final Optional errorCode; + + } + + public ManualOperationResult startNewManualSync(final UUID connectionId) { + log.info("Manual sync request"); + + if (connectionManagerUtils.isWorkflowStateRunning(client, connectionId)) { + // TODO Bmoric: Error is running + return new ManualOperationResult( + Optional.of("A sync is already running for: " + connectionId), + Optional.empty(), Optional.of(ErrorCode.WORKFLOW_RUNNING)); + } + + try { + connectionManagerUtils.signalWorkflowAndRepairIfNecessary(client, connectionId, workflow -> workflow::submitManualSync); + } catch (final DeletedWorkflowException e) { + log.error("Can't sync a deleted connection.", e); + return new ManualOperationResult( + Optional.of(e.getMessage()), + Optional.empty(), Optional.of(ErrorCode.WORKFLOW_DELETED)); + } + + do { + try { + Thread.sleep(DELAY_BETWEEN_QUERY_MS); + } catch (final InterruptedException e) { + return new ManualOperationResult( + Optional.of("Didn't managed to start a sync for: " + connectionId), + Optional.empty(), Optional.of(ErrorCode.UNKNOWN)); + } + } while (!connectionManagerUtils.isWorkflowStateRunning(client, connectionId)); + + log.info("end of manual schedule"); + + final long jobId = connectionManagerUtils.getCurrentJobId(client, connectionId); + + return new ManualOperationResult( + Optional.empty(), + Optional.of(jobId), Optional.empty()); + } + + public ManualOperationResult startNewCancellation(final UUID connectionId) { + log.info("Manual cancellation request"); + + final long jobId = connectionManagerUtils.getCurrentJobId(client, connectionId); + + try { + connectionManagerUtils.signalWorkflowAndRepairIfNecessary(client, connectionId, workflow -> workflow::cancelJob); + } catch (final DeletedWorkflowException e) { + log.error("Can't cancel a deleted workflow", e); + return new ManualOperationResult( + Optional.of(e.getMessage()), + Optional.empty(), Optional.of(ErrorCode.WORKFLOW_DELETED)); + } + + do { + try { + Thread.sleep(DELAY_BETWEEN_QUERY_MS); + } catch (final InterruptedException e) { + return new ManualOperationResult( + Optional.of("Didn't manage to cancel a sync for: " + connectionId), + Optional.empty(), Optional.of(ErrorCode.UNKNOWN)); + } + } while (connectionManagerUtils.isWorkflowStateRunning(client, connectionId)); + + streamResetRecordsHelper.deleteStreamResetRecordsForJob(jobId, connectionId); + + log.info("end of manual cancellation"); + + return new ManualOperationResult( + Optional.empty(), + Optional.of(jobId), Optional.empty()); + } + + public ManualOperationResult resetConnection(final UUID connectionId, + final List streamsToReset, + final boolean syncImmediatelyAfter) { + log.info("reset sync request"); + + try { + streamResetPersistence.createStreamResets(connectionId, streamsToReset); + } catch (final IOException e) { + log.error("Could not persist streams to reset.", e); + return new ManualOperationResult( + Optional.of(e.getMessage()), + Optional.empty(), Optional.of(ErrorCode.UNKNOWN)); + } + + // get the job ID before the reset, defaulting to NON_RUNNING_JOB_ID if workflow is unreachable + final long oldJobId = connectionManagerUtils.getCurrentJobId(client, connectionId); + + try { + if (syncImmediatelyAfter) { + connectionManagerUtils.signalWorkflowAndRepairIfNecessary(client, connectionId, workflow -> workflow::resetConnectionAndSkipNextScheduling); + } else { + connectionManagerUtils.signalWorkflowAndRepairIfNecessary(client, connectionId, workflow -> workflow::resetConnection); + } + } catch (final DeletedWorkflowException e) { + log.error("Can't reset a deleted workflow", e); + return new ManualOperationResult( + Optional.of(e.getMessage()), + Optional.empty(), Optional.of(ErrorCode.UNKNOWN)); + } + + Optional newJobId; + + do { + try { + Thread.sleep(DELAY_BETWEEN_QUERY_MS); + } catch (final InterruptedException e) { + return new ManualOperationResult( + Optional.of("Didn't manage to reset a sync for: " + connectionId), + Optional.empty(), Optional.of(ErrorCode.UNKNOWN)); + } + newJobId = getNewJobId(connectionId, oldJobId); + } while (newJobId.isEmpty()); + + log.info("end of reset submission"); + + return new ManualOperationResult( + Optional.empty(), + newJobId, Optional.empty()); + } + + private Optional getNewJobId(final UUID connectionId, final long oldJobId) { + final long currentJobId = connectionManagerUtils.getCurrentJobId(client, connectionId); + if (currentJobId == NON_RUNNING_JOB_ID || currentJobId == oldJobId) { + return Optional.empty(); + } else { + return Optional.of(currentJobId); + } + } + + public TemporalResponse submitGetSpec(final UUID jobId, final int attempt, final JobGetSpecConfig config) { + final JobRunConfig jobRunConfig = TemporalWorkflowUtils.createJobRunConfig(jobId, attempt); + + final IntegrationLauncherConfig launcherConfig = new IntegrationLauncherConfig() + .withJobId(jobId.toString()) + .withAttemptId((long) attempt) + .withDockerImage(config.getDockerImage()); + return execute(jobRunConfig, + () -> getWorkflowStub(SpecWorkflow.class, TemporalJobType.GET_SPEC).run(jobRunConfig, launcherConfig)); + + } + + public TemporalResponse submitCheckConnection(final UUID jobId, + final int attempt, + final JobCheckConnectionConfig config) { + final JobRunConfig jobRunConfig = TemporalWorkflowUtils.createJobRunConfig(jobId, attempt); + final IntegrationLauncherConfig launcherConfig = new IntegrationLauncherConfig() + .withJobId(jobId.toString()) + .withAttemptId((long) attempt) + .withDockerImage(config.getDockerImage()) + .withProtocolVersion(config.getProtocolVersion()); + final StandardCheckConnectionInput input = new StandardCheckConnectionInput().withConnectionConfiguration(config.getConnectionConfiguration()); + + return execute(jobRunConfig, + () -> getWorkflowStub(CheckConnectionWorkflow.class, TemporalJobType.CHECK_CONNECTION).run(jobRunConfig, launcherConfig, input)); + } + + public TemporalResponse submitDiscoverSchema(final UUID jobId, + final int attempt, + final JobDiscoverCatalogConfig config) { + final JobRunConfig jobRunConfig = TemporalWorkflowUtils.createJobRunConfig(jobId, attempt); + final IntegrationLauncherConfig launcherConfig = new IntegrationLauncherConfig() + .withJobId(jobId.toString()) + .withAttemptId((long) attempt) + .withDockerImage(config.getDockerImage()); + final StandardDiscoverCatalogInput input = new StandardDiscoverCatalogInput().withConnectionConfiguration(config.getConnectionConfiguration()) + .withSourceId(config.getSourceId()).withConnectorVersion(config.getConnectorVersion()).withConfigHash(config.getConfigHash()); + + return execute(jobRunConfig, + () -> getWorkflowStub(DiscoverCatalogWorkflow.class, TemporalJobType.DISCOVER_SCHEMA).run(jobRunConfig, launcherConfig, input)); + } + + public TemporalResponse submitSync(final long jobId, final int attempt, final JobSyncConfig config, final UUID connectionId) { + final JobRunConfig jobRunConfig = TemporalWorkflowUtils.createJobRunConfig(jobId, attempt); + + final IntegrationLauncherConfig sourceLauncherConfig = new IntegrationLauncherConfig() + .withJobId(String.valueOf(jobId)) + .withAttemptId((long) attempt) + .withDockerImage(config.getSourceDockerImage()); + + final IntegrationLauncherConfig destinationLauncherConfig = new IntegrationLauncherConfig() + .withJobId(String.valueOf(jobId)) + .withAttemptId((long) attempt) + .withDockerImage(config.getDestinationDockerImage()); + + final StandardSyncInput input = new StandardSyncInput() + .withNamespaceDefinition(config.getNamespaceDefinition()) + .withNamespaceFormat(config.getNamespaceFormat()) + .withPrefix(config.getPrefix()) + .withSourceConfiguration(config.getSourceConfiguration()) + .withDestinationConfiguration(config.getDestinationConfiguration()) + .withOperationSequence(config.getOperationSequence()) + .withCatalog(config.getConfiguredAirbyteCatalog()) + .withState(config.getState()) + .withResourceRequirements(config.getResourceRequirements()) + .withSourceResourceRequirements(config.getSourceResourceRequirements()) + .withDestinationResourceRequirements(config.getDestinationResourceRequirements()); + + return execute(jobRunConfig, + () -> getWorkflowStub(SyncWorkflow.class, TemporalJobType.SYNC).run( + jobRunConfig, + sourceLauncherConfig, + destinationLauncherConfig, + input, + connectionId)); + } + + public void migrateSyncIfNeeded(final Set connectionIds) { + final StopWatch globalMigrationWatch = new StopWatch(); + globalMigrationWatch.start(); + refreshRunningWorkflow(); + + connectionIds.forEach((connectionId) -> { + final StopWatch singleSyncMigrationWatch = new StopWatch(); + singleSyncMigrationWatch.start(); + if (!isInRunningWorkflowCache(connectionManagerUtils.getConnectionManagerName(connectionId))) { + log.info("Migrating: " + connectionId); + try { + submitConnectionUpdaterAsync(connectionId); + } catch (final Exception e) { + log.error("New workflow submission failed, retrying", e); + refreshRunningWorkflow(); + submitConnectionUpdaterAsync(connectionId); + } + } + singleSyncMigrationWatch.stop(); + log.info("Sync migration took: " + singleSyncMigrationWatch.formatTime()); + }); + globalMigrationWatch.stop(); + + log.info("The migration to the new scheduler took: " + globalMigrationWatch.formatTime()); + } + + @VisibleForTesting + TemporalResponse execute(final JobRunConfig jobRunConfig, final Supplier executor) { + final Path jobRoot = TemporalUtils.getJobRoot(workspaceRoot, jobRunConfig); + final Path logPath = TemporalUtils.getLogPath(jobRoot); + + T operationOutput = null; + RuntimeException exception = null; + + try { + operationOutput = executor.get(); + } catch (final RuntimeException e) { + exception = e; + } + + boolean succeeded = exception == null; + if (succeeded && operationOutput instanceof ConnectorJobOutput) { + succeeded = getConnectorJobSucceeded((ConnectorJobOutput) operationOutput); + } + + final JobMetadata metadata = new JobMetadata(succeeded, logPath); + return new TemporalResponse<>(operationOutput, metadata); + } + + private T getWorkflowStub(final Class workflowClass, final TemporalJobType jobType) { + return client.newWorkflowStub(workflowClass, TemporalWorkflowUtils.buildWorkflowOptions(jobType)); + } + + public ConnectionManagerWorkflow submitConnectionUpdaterAsync(final UUID connectionId) { + log.info("Starting the scheduler temporal wf"); + final ConnectionManagerWorkflow connectionManagerWorkflow = + connectionManagerUtils.startConnectionManagerNoSignal(client, connectionId); + try { + CompletableFuture.supplyAsync(() -> { + try { + do { + Thread.sleep(DELAY_BETWEEN_QUERY_MS); + } while (!isWorkflowReachable(connectionId)); + } catch (final InterruptedException e) {} + return null; + }).get(60, TimeUnit.SECONDS); + } catch (final InterruptedException | ExecutionException e) { + log.error("Failed to create a new connection manager workflow", e); + } catch (final TimeoutException e) { + log.error("Can't create a new connection manager workflow due to timeout", e); + } + + return connectionManagerWorkflow; + } + + public void deleteConnection(final UUID connectionId) { + try { + connectionManagerUtils.signalWorkflowAndRepairIfNecessary(client, connectionId, + connectionManagerWorkflow -> connectionManagerWorkflow::deleteConnection); + } catch (final DeletedWorkflowException e) { + log.info("Connection {} has already been deleted.", connectionId); + } + } + + public void update(final UUID connectionId) { + final ConnectionManagerWorkflow connectionManagerWorkflow; + try { + connectionManagerWorkflow = connectionManagerUtils.getConnectionManagerWorkflow(client, connectionId); + } catch (final DeletedWorkflowException e) { + log.info("Connection {} is deleted, and therefore cannot be updated.", connectionId); + return; + } catch (final UnreachableWorkflowException e) { + log.error( + String.format("Failed to retrieve ConnectionManagerWorkflow for connection %s. Repairing state by creating new workflow.", connectionId), + e); + connectionManagerUtils.safeTerminateWorkflow(client, connectionId, + "Terminating workflow in unreachable state before starting a new workflow for this connection"); + submitConnectionUpdaterAsync(connectionId); + return; + } + + connectionManagerWorkflow.connectionUpdated(); + } + + private boolean getConnectorJobSucceeded(final ConnectorJobOutput output) { + return output.getFailureReason() == null; + } + + /** + * Check if a workflow is reachable for signal calls by attempting to query for current state. If + * the query succeeds, and the workflow is not marked as deleted, the workflow is reachable. + */ + @VisibleForTesting + boolean isWorkflowReachable(final UUID connectionId) { + try { + connectionManagerUtils.getConnectionManagerWorkflow(client, connectionId); + return true; + } catch (final Exception e) { + return false; + } + } + + boolean isInRunningWorkflowCache(final String workflowName) { + return workflowNames.contains(workflowName); + } + } diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/TemporalResponse.java b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/TemporalResponse.java similarity index 97% rename from airbyte-workers/src/main/java/io/airbyte/workers/temporal/TemporalResponse.java rename to airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/TemporalResponse.java index 238f027e85c4d..2f659507cae2e 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/TemporalResponse.java +++ b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/TemporalResponse.java @@ -2,7 +2,7 @@ * Copyright (c) 2022 Airbyte, Inc., all rights reserved. */ -package io.airbyte.workers.temporal; +package io.airbyte.commons.temporal; import java.util.Objects; import java.util.Optional; diff --git a/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/TemporalUtils.java b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/TemporalUtils.java index e3824e1fdbb49..8c5700c748cac 100644 --- a/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/TemporalUtils.java +++ b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/TemporalUtils.java @@ -8,7 +8,9 @@ import com.uber.m3.tally.Scope; import com.uber.m3.tally.StatsReporter; import io.airbyte.commons.lang.Exceptions; +import io.airbyte.config.helpers.LogClientSingleton; import io.airbyte.metrics.lib.MetricClientFactory; +import io.airbyte.persistence.job.models.JobRunConfig; import io.micrometer.core.instrument.MeterRegistry; import io.micronaut.context.annotation.Property; import io.micronaut.context.annotation.Value; @@ -32,6 +34,7 @@ import java.io.InputStream; import java.io.Serializable; import java.nio.charset.StandardCharsets; +import java.nio.file.Path; import java.time.Duration; import java.util.UUID; import java.util.concurrent.Callable; @@ -315,4 +318,24 @@ public T withBackgroundHeartbeat(final AtomicReference afterCancel } } + // todo (cgardens) - there are 2 sources of truth for job path. we need to reduce this down to one, + // once we are fully on temporal. + public static Path getJobRoot(final Path workspaceRoot, final JobRunConfig jobRunConfig) { + return getJobRoot(workspaceRoot, jobRunConfig.getJobId(), jobRunConfig.getAttemptId()); + } + + public static Path getLogPath(final Path jobRoot) { + return jobRoot.resolve(LogClientSingleton.LOG_FILENAME); + } + + public static Path getJobRoot(final Path workspaceRoot, final String jobId, final long attemptId) { + return getJobRoot(workspaceRoot, jobId, Math.toIntExact(attemptId)); + } + + public static Path getJobRoot(final Path workspaceRoot, final String jobId, final int attemptId) { + return workspaceRoot + .resolve(String.valueOf(jobId)) + .resolve(String.valueOf(attemptId)); + } + } diff --git a/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/config/TemporalBeanFactory.java b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/config/TemporalBeanFactory.java index 2a86687ef532a..24eede27ffbad 100644 --- a/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/config/TemporalBeanFactory.java +++ b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/config/TemporalBeanFactory.java @@ -7,9 +7,12 @@ import io.airbyte.commons.temporal.TemporalUtils; import io.airbyte.commons.temporal.TemporalWorkflowUtils; import io.micronaut.context.annotation.Factory; +import io.micronaut.context.annotation.Value; import io.temporal.client.WorkflowClient; import io.temporal.serviceclient.WorkflowServiceStubs; +import jakarta.inject.Named; import jakarta.inject.Singleton; +import java.nio.file.Path; /** * Micronaut bean factory for Temporal-related singletons. @@ -29,4 +32,10 @@ public WorkflowClient workflowClient( return TemporalWorkflowUtils.createWorkflowClient(temporalService, temporalUtils.getNamespace()); } + @Singleton + @Named("workspaceRootTemporal") + public Path workspaceRoot(@Value("${airbyte.workspace.root}") final String workspaceRoot) { + return Path.of(workspaceRoot); + } + } diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/config/WorkerMode.java b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/config/WorkerMode.java similarity index 90% rename from airbyte-commons-worker/src/main/java/io/airbyte/workers/config/WorkerMode.java rename to airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/config/WorkerMode.java index 66e164b2fbca6..16960c51bb474 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/config/WorkerMode.java +++ b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/config/WorkerMode.java @@ -2,7 +2,7 @@ * Copyright (c) 2022 Airbyte, Inc., all rights reserved. */ -package io.airbyte.workers.config; +package io.airbyte.commons.temporal.config; /** * Defines the different execution modes for the workers application. diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/exception/DeletedWorkflowException.java b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/exception/DeletedWorkflowException.java similarity index 81% rename from airbyte-workers/src/main/java/io/airbyte/workers/temporal/exception/DeletedWorkflowException.java rename to airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/exception/DeletedWorkflowException.java index 8488051066a0a..4be081eb4d29d 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/exception/DeletedWorkflowException.java +++ b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/exception/DeletedWorkflowException.java @@ -2,7 +2,7 @@ * Copyright (c) 2022 Airbyte, Inc., all rights reserved. */ -package io.airbyte.workers.temporal.exception; +package io.airbyte.commons.temporal.exception; public class DeletedWorkflowException extends Exception { diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/exception/RetryableException.java b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/exception/RetryableException.java similarity index 80% rename from airbyte-workers/src/main/java/io/airbyte/workers/temporal/exception/RetryableException.java rename to airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/exception/RetryableException.java index 7ef8b178ec858..870f9b4613c44 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/exception/RetryableException.java +++ b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/exception/RetryableException.java @@ -2,7 +2,7 @@ * Copyright (c) 2022 Airbyte, Inc., all rights reserved. */ -package io.airbyte.workers.temporal.exception; +package io.airbyte.commons.temporal.exception; public class RetryableException extends RuntimeException { diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/exception/UnreachableWorkflowException.java b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/exception/UnreachableWorkflowException.java similarity index 87% rename from airbyte-workers/src/main/java/io/airbyte/workers/temporal/exception/UnreachableWorkflowException.java rename to airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/exception/UnreachableWorkflowException.java index 91e8ed2bd5cb4..e7daccf9260ea 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/exception/UnreachableWorkflowException.java +++ b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/exception/UnreachableWorkflowException.java @@ -2,7 +2,7 @@ * Copyright (c) 2022 Airbyte, Inc., all rights reserved. */ -package io.airbyte.workers.temporal.exception; +package io.airbyte.commons.temporal.exception; public class UnreachableWorkflowException extends Exception { diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/check/connection/CheckConnectionWorkflow.java b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/scheduling/CheckConnectionWorkflow.java similarity index 92% rename from airbyte-workers/src/main/java/io/airbyte/workers/temporal/check/connection/CheckConnectionWorkflow.java rename to airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/scheduling/CheckConnectionWorkflow.java index 13528fdfeaa61..cff77025e9951 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/check/connection/CheckConnectionWorkflow.java +++ b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/scheduling/CheckConnectionWorkflow.java @@ -2,7 +2,7 @@ * Copyright (c) 2022 Airbyte, Inc., all rights reserved. */ -package io.airbyte.workers.temporal.check.connection; +package io.airbyte.commons.temporal.scheduling; import io.airbyte.config.ConnectorJobOutput; import io.airbyte.config.StandardCheckConnectionInput; diff --git a/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/scheduling/ConnectionManagerWorkflow.java b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/scheduling/ConnectionManagerWorkflow.java index 43597afd7c1cc..dff392109b070 100644 --- a/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/scheduling/ConnectionManagerWorkflow.java +++ b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/scheduling/ConnectionManagerWorkflow.java @@ -17,6 +17,9 @@ @WorkflowInterface public interface ConnectionManagerWorkflow { + long NON_RUNNING_JOB_ID = -1; + int NON_RUNNING_ATTEMPT_ID = -1; + /** * Workflow method to launch a {@link ConnectionManagerWorkflow}. Launches a workflow responsible * for scheduling syncs. This workflow will run and then continue running until deleted. diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/discover/catalog/DiscoverCatalogWorkflow.java b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/scheduling/DiscoverCatalogWorkflow.java similarity index 92% rename from airbyte-workers/src/main/java/io/airbyte/workers/temporal/discover/catalog/DiscoverCatalogWorkflow.java rename to airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/scheduling/DiscoverCatalogWorkflow.java index 65a45e7602eec..27e99d0bf0bb9 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/discover/catalog/DiscoverCatalogWorkflow.java +++ b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/scheduling/DiscoverCatalogWorkflow.java @@ -2,7 +2,7 @@ * Copyright (c) 2022 Airbyte, Inc., all rights reserved. */ -package io.airbyte.workers.temporal.discover.catalog; +package io.airbyte.commons.temporal.scheduling; import io.airbyte.config.ConnectorJobOutput; import io.airbyte.config.StandardDiscoverCatalogInput; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/spec/SpecWorkflow.java b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/scheduling/SpecWorkflow.java similarity index 91% rename from airbyte-workers/src/main/java/io/airbyte/workers/temporal/spec/SpecWorkflow.java rename to airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/scheduling/SpecWorkflow.java index 8f42cbd29b22b..025b23ba4acd9 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/spec/SpecWorkflow.java +++ b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/scheduling/SpecWorkflow.java @@ -2,7 +2,7 @@ * Copyright (c) 2022 Airbyte, Inc., all rights reserved. */ -package io.airbyte.workers.temporal.spec; +package io.airbyte.commons.temporal.scheduling; import io.airbyte.config.ConnectorJobOutput; import io.airbyte.persistence.job.models.IntegrationLauncherConfig; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/SyncWorkflow.java b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/scheduling/SyncWorkflow.java similarity index 94% rename from airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/SyncWorkflow.java rename to airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/scheduling/SyncWorkflow.java index 6fa9efe3e47ee..513974b955340 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/SyncWorkflow.java +++ b/airbyte-commons-temporal/src/main/java/io/airbyte/commons/temporal/scheduling/SyncWorkflow.java @@ -2,7 +2,7 @@ * Copyright (c) 2022 Airbyte, Inc., all rights reserved. */ -package io.airbyte.workers.temporal.sync; +package io.airbyte.commons.temporal.scheduling; import io.airbyte.config.StandardSyncInput; import io.airbyte.config.StandardSyncOutput; diff --git a/airbyte-commons-temporal/src/test/java/io/airbyte/commons/temporal/TemporalClientTest.java b/airbyte-commons-temporal/src/test/java/io/airbyte/commons/temporal/TemporalClientTest.java index 0ddff18dc49b1..82b161e5eb05f 100644 --- a/airbyte-commons-temporal/src/test/java/io/airbyte/commons/temporal/TemporalClientTest.java +++ b/airbyte-commons-temporal/src/test/java/io/airbyte/commons/temporal/TemporalClientTest.java @@ -4,46 +4,125 @@ package io.airbyte.commons.temporal; +import static io.airbyte.commons.temporal.scheduling.ConnectionManagerWorkflow.NON_RUNNING_JOB_ID; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; +import com.google.common.collect.Sets; +import io.airbyte.commons.json.Jsons; +import io.airbyte.commons.temporal.TemporalClient.ManualOperationResult; +import io.airbyte.commons.temporal.scheduling.CheckConnectionWorkflow; import io.airbyte.commons.temporal.scheduling.ConnectionManagerWorkflow; +import io.airbyte.commons.temporal.scheduling.ConnectionManagerWorkflow.JobInformation; +import io.airbyte.commons.temporal.scheduling.DiscoverCatalogWorkflow; +import io.airbyte.commons.temporal.scheduling.SpecWorkflow; +import io.airbyte.commons.temporal.scheduling.SyncWorkflow; +import io.airbyte.commons.temporal.scheduling.state.WorkflowState; +import io.airbyte.config.ConnectorJobOutput; +import io.airbyte.config.FailureReason; +import io.airbyte.config.JobCheckConnectionConfig; +import io.airbyte.config.JobDiscoverCatalogConfig; +import io.airbyte.config.JobGetSpecConfig; +import io.airbyte.config.JobSyncConfig; +import io.airbyte.config.StandardCheckConnectionInput; +import io.airbyte.config.StandardDiscoverCatalogInput; +import io.airbyte.config.StandardSyncInput; +import io.airbyte.config.helpers.LogClientSingleton; +import io.airbyte.config.persistence.StreamResetPersistence; +import io.airbyte.persistence.job.models.IntegrationLauncherConfig; +import io.airbyte.persistence.job.models.JobRunConfig; +import io.airbyte.protocol.models.ConfiguredAirbyteCatalog; +import io.airbyte.protocol.models.StreamDescriptor; import io.temporal.api.enums.v1.WorkflowExecutionStatus; import io.temporal.api.workflow.v1.WorkflowExecutionInfo; import io.temporal.api.workflowservice.v1.DescribeWorkflowExecutionResponse; -import io.temporal.api.workflowservice.v1.WorkflowServiceGrpc; +import io.temporal.api.workflowservice.v1.WorkflowServiceGrpc.WorkflowServiceBlockingStub; +import io.temporal.client.BatchRequest; import io.temporal.client.WorkflowClient; import io.temporal.client.WorkflowClientOptions; +import io.temporal.client.WorkflowOptions; +import io.temporal.client.WorkflowStub; import io.temporal.serviceclient.WorkflowServiceStubs; +import io.temporal.workflow.Functions.Proc; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.List; import java.util.Set; import java.util.UUID; +import java.util.function.Supplier; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.DisplayName; import org.junit.jupiter.api.Nested; import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +@SuppressWarnings("PMD.JUnit5TestShouldBePackagePrivate") public class TemporalClientTest { + private static final UUID CONNECTION_ID = UUID.randomUUID(); + private static final UUID JOB_UUID = UUID.randomUUID(); + private static final long JOB_ID = 11L; + private static final int ATTEMPT_ID = 21; + private static final JobRunConfig JOB_RUN_CONFIG = new JobRunConfig() + .withJobId(String.valueOf(JOB_ID)) + .withAttemptId((long) ATTEMPT_ID); + private static final String IMAGE_NAME1 = "hms invincible"; + private static final String IMAGE_NAME2 = "hms defiant"; + private static final IntegrationLauncherConfig UUID_LAUNCHER_CONFIG = new IntegrationLauncherConfig() + .withJobId(String.valueOf(JOB_UUID)) + .withAttemptId((long) ATTEMPT_ID) + .withDockerImage(IMAGE_NAME1); + private static final IntegrationLauncherConfig LAUNCHER_CONFIG = new IntegrationLauncherConfig() + .withJobId(String.valueOf(JOB_ID)) + .withAttemptId((long) ATTEMPT_ID) + .withDockerImage(IMAGE_NAME1); private static final String NAMESPACE = "namespace"; + private static final StreamDescriptor STREAM_DESCRIPTOR = new StreamDescriptor().withName("name"); + private static final String UNCHECKED = "unchecked"; + private static final String EXCEPTION_MESSAGE = "Force state exception to simulate workflow not running"; private WorkflowClient workflowClient; private TemporalClient temporalClient; + private Path logPath; private WorkflowServiceStubs workflowServiceStubs; - private WorkflowServiceGrpc.WorkflowServiceBlockingStub workflowServiceBlockingStub; + private WorkflowServiceBlockingStub workflowServiceBlockingStub; + private StreamResetPersistence streamResetPersistence; + private ConnectionManagerUtils connectionManagerUtils; + private StreamResetRecordsHelper streamResetRecordsHelper; + private Path workspaceRoot; @BeforeEach - void setup() { + void setup() throws IOException { + workspaceRoot = Files.createTempDirectory(Path.of("/tmp"), "temporal_client_test"); + logPath = workspaceRoot.resolve(String.valueOf(JOB_ID)).resolve(String.valueOf(ATTEMPT_ID)).resolve(LogClientSingleton.LOG_FILENAME); workflowClient = mock(WorkflowClient.class); when(workflowClient.getOptions()).thenReturn(WorkflowClientOptions.newBuilder().setNamespace(NAMESPACE).build()); - workflowServiceStubs = mock(WorkflowServiceStubs.class); - workflowServiceBlockingStub = mock(WorkflowServiceGrpc.WorkflowServiceBlockingStub.class); + when(workflowClient.getWorkflowServiceStubs()).thenReturn(workflowServiceStubs); + workflowServiceBlockingStub = mock(WorkflowServiceBlockingStub.class); when(workflowServiceStubs.blockingStub()).thenReturn(workflowServiceBlockingStub); + streamResetPersistence = mock(StreamResetPersistence.class); + mockWorkflowStatus(WorkflowExecutionStatus.WORKFLOW_EXECUTION_STATUS_RUNNING); + connectionManagerUtils = spy(new ConnectionManagerUtils()); + streamResetRecordsHelper = mock(StreamResetRecordsHelper.class); + temporalClient = + spy(new TemporalClient(workspaceRoot, workflowClient, workflowServiceStubs, streamResetPersistence, connectionManagerUtils, + streamResetRecordsHelper)); } @Nested @@ -52,11 +131,12 @@ class RestartPerStatus { private ConnectionManagerUtils mConnectionManagerUtils; @BeforeEach - public void init() { + void init() { mConnectionManagerUtils = mock(ConnectionManagerUtils.class); temporalClient = spy( - new TemporalClient(workflowClient, workflowServiceStubs, mConnectionManagerUtils)); + new TemporalClient(workspaceRoot, workflowClient, workflowServiceStubs, streamResetPersistence, mConnectionManagerUtils, + streamResetRecordsHelper)); } @Test @@ -80,6 +160,713 @@ void testRestartFailed() { } + @Nested + @DisplayName("Test execute method.") + class ExecuteJob { + + @SuppressWarnings(UNCHECKED) + @Test + void testExecute() { + final Supplier supplier = mock(Supplier.class); + when(supplier.get()).thenReturn("hello"); + + final TemporalResponse response = temporalClient.execute(JOB_RUN_CONFIG, supplier); + + assertNotNull(response); + assertTrue(response.getOutput().isPresent()); + assertEquals("hello", response.getOutput().get()); + assertTrue(response.getMetadata().isSucceeded()); + assertEquals(logPath, response.getMetadata().getLogPath()); + } + + @SuppressWarnings(UNCHECKED) + @Test + void testExecuteWithException() { + final Supplier supplier = mock(Supplier.class); + when(supplier.get()).thenThrow(IllegalStateException.class); + + final TemporalResponse response = temporalClient.execute(JOB_RUN_CONFIG, supplier); + + assertNotNull(response); + assertFalse(response.getOutput().isPresent()); + assertFalse(response.getMetadata().isSucceeded()); + assertEquals(logPath, response.getMetadata().getLogPath()); + } + + @Test + void testExecuteWithConnectorJobFailure() { + final Supplier supplier = mock(Supplier.class); + final FailureReason mockFailureReason = mock(FailureReason.class); + final ConnectorJobOutput connectorJobOutput = new ConnectorJobOutput() + .withFailureReason(mockFailureReason); + when(supplier.get()).thenReturn(connectorJobOutput); + + final TemporalResponse response = temporalClient.execute(JOB_RUN_CONFIG, supplier); + + assertNotNull(response); + assertTrue(response.getOutput().isPresent()); + assertEquals(connectorJobOutput, response.getOutput().get()); + assertFalse(response.getMetadata().isSucceeded()); + assertEquals(logPath, response.getMetadata().getLogPath()); + } + + } + + @Nested + @DisplayName("Test job creation for each configuration type.") + class TestJobSubmission { + + @Test + void testSubmitGetSpec() { + final SpecWorkflow specWorkflow = mock(SpecWorkflow.class); + when(workflowClient.newWorkflowStub(SpecWorkflow.class, TemporalWorkflowUtils.buildWorkflowOptions(TemporalJobType.GET_SPEC))) + .thenReturn(specWorkflow); + final JobGetSpecConfig getSpecConfig = new JobGetSpecConfig().withDockerImage(IMAGE_NAME1); + + temporalClient.submitGetSpec(JOB_UUID, ATTEMPT_ID, getSpecConfig); + specWorkflow.run(JOB_RUN_CONFIG, UUID_LAUNCHER_CONFIG); + verify(workflowClient).newWorkflowStub(SpecWorkflow.class, TemporalWorkflowUtils.buildWorkflowOptions(TemporalJobType.GET_SPEC)); + } + + @Test + void testSubmitCheckConnection() { + final CheckConnectionWorkflow checkConnectionWorkflow = mock(CheckConnectionWorkflow.class); + when( + workflowClient.newWorkflowStub(CheckConnectionWorkflow.class, TemporalWorkflowUtils.buildWorkflowOptions(TemporalJobType.CHECK_CONNECTION))) + .thenReturn(checkConnectionWorkflow); + final JobCheckConnectionConfig checkConnectionConfig = new JobCheckConnectionConfig() + .withDockerImage(IMAGE_NAME1) + .withConnectionConfiguration(Jsons.emptyObject()); + final StandardCheckConnectionInput input = new StandardCheckConnectionInput() + .withConnectionConfiguration(checkConnectionConfig.getConnectionConfiguration()); + + temporalClient.submitCheckConnection(JOB_UUID, ATTEMPT_ID, checkConnectionConfig); + checkConnectionWorkflow.run(JOB_RUN_CONFIG, UUID_LAUNCHER_CONFIG, input); + verify(workflowClient).newWorkflowStub(CheckConnectionWorkflow.class, + TemporalWorkflowUtils.buildWorkflowOptions(TemporalJobType.CHECK_CONNECTION)); + } + + @Test + void testSubmitDiscoverSchema() { + final DiscoverCatalogWorkflow discoverCatalogWorkflow = mock(DiscoverCatalogWorkflow.class); + when(workflowClient.newWorkflowStub(DiscoverCatalogWorkflow.class, TemporalWorkflowUtils.buildWorkflowOptions(TemporalJobType.DISCOVER_SCHEMA))) + .thenReturn(discoverCatalogWorkflow); + final JobDiscoverCatalogConfig checkConnectionConfig = new JobDiscoverCatalogConfig() + .withDockerImage(IMAGE_NAME1) + .withConnectionConfiguration(Jsons.emptyObject()); + final StandardDiscoverCatalogInput input = new StandardDiscoverCatalogInput() + .withConnectionConfiguration(checkConnectionConfig.getConnectionConfiguration()); + + temporalClient.submitDiscoverSchema(JOB_UUID, ATTEMPT_ID, checkConnectionConfig); + discoverCatalogWorkflow.run(JOB_RUN_CONFIG, UUID_LAUNCHER_CONFIG, input); + verify(workflowClient).newWorkflowStub(DiscoverCatalogWorkflow.class, + TemporalWorkflowUtils.buildWorkflowOptions(TemporalJobType.DISCOVER_SCHEMA)); + } + + @Test + void testSubmitSync() { + final SyncWorkflow discoverCatalogWorkflow = mock(SyncWorkflow.class); + when(workflowClient.newWorkflowStub(SyncWorkflow.class, TemporalWorkflowUtils.buildWorkflowOptions(TemporalJobType.SYNC))) + .thenReturn(discoverCatalogWorkflow); + final JobSyncConfig syncConfig = new JobSyncConfig() + .withSourceDockerImage(IMAGE_NAME1) + .withSourceDockerImage(IMAGE_NAME2) + .withSourceConfiguration(Jsons.emptyObject()) + .withDestinationConfiguration(Jsons.emptyObject()) + .withOperationSequence(List.of()) + .withConfiguredAirbyteCatalog(new ConfiguredAirbyteCatalog()); + final StandardSyncInput input = new StandardSyncInput() + .withNamespaceDefinition(syncConfig.getNamespaceDefinition()) + .withNamespaceFormat(syncConfig.getNamespaceFormat()) + .withPrefix(syncConfig.getPrefix()) + .withSourceConfiguration(syncConfig.getSourceConfiguration()) + .withDestinationConfiguration(syncConfig.getDestinationConfiguration()) + .withOperationSequence(syncConfig.getOperationSequence()) + .withCatalog(syncConfig.getConfiguredAirbyteCatalog()) + .withState(syncConfig.getState()); + + final IntegrationLauncherConfig destinationLauncherConfig = new IntegrationLauncherConfig() + .withJobId(String.valueOf(JOB_ID)) + .withAttemptId((long) ATTEMPT_ID) + .withDockerImage(IMAGE_NAME2); + + temporalClient.submitSync(JOB_ID, ATTEMPT_ID, syncConfig, CONNECTION_ID); + discoverCatalogWorkflow.run(JOB_RUN_CONFIG, LAUNCHER_CONFIG, destinationLauncherConfig, input, CONNECTION_ID); + verify(workflowClient).newWorkflowStub(SyncWorkflow.class, TemporalWorkflowUtils.buildWorkflowOptions(TemporalJobType.SYNC)); + } + + } + + @Nested + @DisplayName("Test related to the migration to the new scheduler") + class TestMigration { + + @DisplayName("Test that the migration is properly done if needed") + @Test + void migrateCalled() { + final UUID nonMigratedId = UUID.randomUUID(); + final UUID migratedId = UUID.randomUUID(); + + when(temporalClient.isInRunningWorkflowCache(connectionManagerUtils.getConnectionManagerName(nonMigratedId))).thenReturn(false); + when(temporalClient.isInRunningWorkflowCache(connectionManagerUtils.getConnectionManagerName(migratedId))).thenReturn(true); + + doNothing() + .when(temporalClient).refreshRunningWorkflow(); + final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + doReturn(mConnectionManagerWorkflow) + .when(temporalClient).submitConnectionUpdaterAsync(nonMigratedId); + + temporalClient.migrateSyncIfNeeded(Sets.newHashSet(nonMigratedId, migratedId)); + + verify(temporalClient, times(1)).submitConnectionUpdaterAsync(nonMigratedId); + verify(temporalClient, times(0)).submitConnectionUpdaterAsync(migratedId); + } + + } + + @Nested + @DisplayName("Test delete connection method.") + class DeleteConnection { + + @Test + @SuppressWarnings(UNCHECKED) + @DisplayName("Test delete connection method when workflow is in a running state.") + void testDeleteConnection() { + final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + final WorkflowState mWorkflowState = mock(WorkflowState.class); + when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); + when(mWorkflowState.isDeleted()).thenReturn(false); + + doReturn(true).when(temporalClient).isWorkflowReachable(any(UUID.class)); + when(workflowClient.newWorkflowStub(any(Class.class), anyString())).thenReturn(mConnectionManagerWorkflow); + + final JobSyncConfig syncConfig = new JobSyncConfig() + .withSourceDockerImage(IMAGE_NAME1) + .withSourceDockerImage(IMAGE_NAME2) + .withSourceConfiguration(Jsons.emptyObject()) + .withDestinationConfiguration(Jsons.emptyObject()) + .withOperationSequence(List.of()) + .withConfiguredAirbyteCatalog(new ConfiguredAirbyteCatalog()); + + temporalClient.submitSync(JOB_ID, ATTEMPT_ID, syncConfig, CONNECTION_ID); + temporalClient.deleteConnection(CONNECTION_ID); + + verify(workflowClient, Mockito.never()).newSignalWithStartRequest(); + verify(mConnectionManagerWorkflow).deleteConnection(); + } + + @Test + @SuppressWarnings(UNCHECKED) + @DisplayName("Test delete connection method when workflow is in an unexpected state") + void testDeleteConnectionInUnexpectedState() { + final ConnectionManagerWorkflow mTerminatedConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + when(mTerminatedConnectionManagerWorkflow.getState()) + .thenThrow(new IllegalStateException(EXCEPTION_MESSAGE)); + when(workflowClient.newWorkflowStub(any(Class.class), any(String.class))).thenReturn(mTerminatedConnectionManagerWorkflow); + + final ConnectionManagerWorkflow mNewConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + when(workflowClient.newWorkflowStub(any(Class.class), any(WorkflowOptions.class))).thenReturn(mNewConnectionManagerWorkflow); + final BatchRequest mBatchRequest = mock(BatchRequest.class); + when(workflowClient.newSignalWithStartRequest()).thenReturn(mBatchRequest); + + temporalClient.deleteConnection(CONNECTION_ID); + verify(workflowClient).signalWithStart(mBatchRequest); + + // Verify that the deleteConnection signal was passed to the batch request by capturing the + // argument, + // executing the signal, and verifying that the desired signal was executed + final ArgumentCaptor batchRequestAddArgCaptor = ArgumentCaptor.forClass(Proc.class); + verify(mBatchRequest).add(batchRequestAddArgCaptor.capture()); + final Proc signal = batchRequestAddArgCaptor.getValue(); + signal.apply(); + verify(mNewConnectionManagerWorkflow).deleteConnection(); + } + + @Test + @SuppressWarnings(UNCHECKED) + @DisplayName("Test delete connection method when workflow has already been deleted") + void testDeleteConnectionOnDeletedWorkflow() { + final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + final WorkflowState mWorkflowState = mock(WorkflowState.class); + when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); + when(mWorkflowState.isDeleted()).thenReturn(true); + when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow); + mockWorkflowStatus(WorkflowExecutionStatus.WORKFLOW_EXECUTION_STATUS_COMPLETED); + + temporalClient.deleteConnection(CONNECTION_ID); + + verify(temporalClient).deleteConnection(CONNECTION_ID); + verifyNoMoreInteractions(temporalClient); + } + + } + + @Nested + @DisplayName("Test update connection behavior") + class UpdateConnection { + + @Test + @SuppressWarnings(UNCHECKED) + @DisplayName("Test update connection when workflow is running") + void testUpdateConnection() { + final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + final WorkflowState mWorkflowState = mock(WorkflowState.class); + + when(mWorkflowState.isRunning()).thenReturn(true); + when(mWorkflowState.isDeleted()).thenReturn(false); + when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); + when(workflowClient.newWorkflowStub(any(Class.class), any(String.class))).thenReturn(mConnectionManagerWorkflow); + + temporalClient.update(CONNECTION_ID); + + verify(mConnectionManagerWorkflow, Mockito.times(1)).connectionUpdated(); + } + + @Test + @SuppressWarnings(UNCHECKED) + @DisplayName("Test update connection method starts a new workflow when workflow is in an unexpected state") + void testUpdateConnectionInUnexpectedState() { + final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + + when(mConnectionManagerWorkflow.getState()).thenThrow(new IllegalStateException(EXCEPTION_MESSAGE)); + when(workflowClient.newWorkflowStub(any(Class.class), any(String.class))).thenReturn(mConnectionManagerWorkflow); + doReturn(mConnectionManagerWorkflow).when(temporalClient).submitConnectionUpdaterAsync(CONNECTION_ID); + + final WorkflowStub untypedWorkflowStub = mock(WorkflowStub.class); + when(workflowClient.newUntypedWorkflowStub(anyString())).thenReturn(untypedWorkflowStub); + + temporalClient.update(CONNECTION_ID); + + // this is only called when updating an existing workflow + verify(mConnectionManagerWorkflow, Mockito.never()).connectionUpdated(); + + verify(untypedWorkflowStub, Mockito.times(1)).terminate(anyString()); + verify(temporalClient, Mockito.times(1)).submitConnectionUpdaterAsync(CONNECTION_ID); + } + + @Test + @SuppressWarnings(UNCHECKED) + @DisplayName("Test update connection method does nothing when connection is deleted") + void testUpdateConnectionDeletedWorkflow() { + final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + final WorkflowState mWorkflowState = mock(WorkflowState.class); + when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); + when(mWorkflowState.isDeleted()).thenReturn(true); + when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow); + mockWorkflowStatus(WorkflowExecutionStatus.WORKFLOW_EXECUTION_STATUS_COMPLETED); + + temporalClient.update(CONNECTION_ID); + + // this is only called when updating an existing workflow + verify(mConnectionManagerWorkflow, Mockito.never()).connectionUpdated(); + verify(temporalClient).update(CONNECTION_ID); + verifyNoMoreInteractions(temporalClient); + } + + } + + @Nested + @DisplayName("Test manual sync behavior") + class ManualSync { + + @Test + @DisplayName("Test startNewManualSync successful") + void testStartNewManualSyncSuccess() { + final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + final WorkflowState mWorkflowState = mock(WorkflowState.class); + when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); + when(mWorkflowState.isDeleted()).thenReturn(false); + when(mWorkflowState.isRunning()).thenReturn(false).thenReturn(true); + when(mConnectionManagerWorkflow.getJobInformation()).thenReturn(new JobInformation(JOB_ID, ATTEMPT_ID)); + when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow); + + final ManualOperationResult result = temporalClient.startNewManualSync(CONNECTION_ID); + + assertTrue(result.getJobId().isPresent()); + assertEquals(JOB_ID, result.getJobId().get()); + assertFalse(result.getFailingReason().isPresent()); + verify(mConnectionManagerWorkflow).submitManualSync(); + } + + @Test + @DisplayName("Test startNewManualSync fails if job is already running") + void testStartNewManualSyncAlreadyRunning() { + final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + final WorkflowState mWorkflowState = mock(WorkflowState.class); + when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); + when(mWorkflowState.isDeleted()).thenReturn(false); + when(mWorkflowState.isRunning()).thenReturn(true); + when(mConnectionManagerWorkflow.getJobInformation()).thenReturn(new JobInformation(JOB_ID, ATTEMPT_ID)); + when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow); + + final ManualOperationResult result = temporalClient.startNewManualSync(CONNECTION_ID); + + assertFalse(result.getJobId().isPresent()); + assertTrue(result.getFailingReason().isPresent()); + verify(mConnectionManagerWorkflow, times(0)).submitManualSync(); + } + + @Test + @DisplayName("Test startNewManualSync repairs the workflow if it is in a bad state") + void testStartNewManualSyncRepairsBadWorkflowState() { + final ConnectionManagerWorkflow mTerminatedConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + when(mTerminatedConnectionManagerWorkflow.getState()) + .thenThrow(new IllegalStateException(EXCEPTION_MESSAGE)); + when(mTerminatedConnectionManagerWorkflow.getJobInformation()).thenReturn(new JobInformation(JOB_ID, ATTEMPT_ID)); + + final ConnectionManagerWorkflow mNewConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + final WorkflowState mWorkflowState = mock(WorkflowState.class); + when(mNewConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); + when(mWorkflowState.isDeleted()).thenReturn(false); + when(mWorkflowState.isRunning()).thenReturn(false).thenReturn(true); + when(mNewConnectionManagerWorkflow.getJobInformation()).thenReturn(new JobInformation(JOB_ID, ATTEMPT_ID)); + when(workflowClient.newWorkflowStub(any(Class.class), any(WorkflowOptions.class))).thenReturn(mNewConnectionManagerWorkflow); + final BatchRequest mBatchRequest = mock(BatchRequest.class); + when(workflowClient.newSignalWithStartRequest()).thenReturn(mBatchRequest); + + when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mTerminatedConnectionManagerWorkflow, mTerminatedConnectionManagerWorkflow, + mNewConnectionManagerWorkflow); + + final ManualOperationResult result = temporalClient.startNewManualSync(CONNECTION_ID); + + assertTrue(result.getJobId().isPresent()); + assertEquals(JOB_ID, result.getJobId().get()); + assertFalse(result.getFailingReason().isPresent()); + verify(workflowClient).signalWithStart(mBatchRequest); + + // Verify that the submitManualSync signal was passed to the batch request by capturing the + // argument, + // executing the signal, and verifying that the desired signal was executed + final ArgumentCaptor batchRequestAddArgCaptor = ArgumentCaptor.forClass(Proc.class); + verify(mBatchRequest).add(batchRequestAddArgCaptor.capture()); + final Proc signal = batchRequestAddArgCaptor.getValue(); + signal.apply(); + verify(mNewConnectionManagerWorkflow).submitManualSync(); + } + + @Test + @SuppressWarnings(UNCHECKED) + @DisplayName("Test startNewManualSync returns a failure reason when connection is deleted") + void testStartNewManualSyncDeletedWorkflow() { + final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + final WorkflowState mWorkflowState = mock(WorkflowState.class); + when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); + when(mWorkflowState.isDeleted()).thenReturn(true); + when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow); + mockWorkflowStatus(WorkflowExecutionStatus.WORKFLOW_EXECUTION_STATUS_COMPLETED); + + final ManualOperationResult result = temporalClient.startNewManualSync(CONNECTION_ID); + + // this is only called when updating an existing workflow + assertFalse(result.getJobId().isPresent()); + assertTrue(result.getFailingReason().isPresent()); + verify(mConnectionManagerWorkflow, times(0)).submitManualSync(); + } + + } + + @Nested + @DisplayName("Test cancellation behavior") + class Cancellation { + + @Test + @DisplayName("Test startNewCancellation successful") + void testStartNewCancellationSuccess() { + final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + final WorkflowState mWorkflowState = mock(WorkflowState.class); + when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); + when(mWorkflowState.isDeleted()).thenReturn(false); + when(mWorkflowState.isRunning()).thenReturn(true).thenReturn(false); + when(mConnectionManagerWorkflow.getJobInformation()).thenReturn(new JobInformation(JOB_ID, ATTEMPT_ID)); + when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow); + + final ManualOperationResult result = temporalClient.startNewCancellation(CONNECTION_ID); + + assertTrue(result.getJobId().isPresent()); + assertEquals(JOB_ID, result.getJobId().get()); + assertFalse(result.getFailingReason().isPresent()); + verify(mConnectionManagerWorkflow).cancelJob(); + verify(streamResetRecordsHelper).deleteStreamResetRecordsForJob(JOB_ID, CONNECTION_ID); + } + + @Test + @DisplayName("Test startNewCancellation repairs the workflow if it is in a bad state") + void testStartNewCancellationRepairsBadWorkflowState() { + final ConnectionManagerWorkflow mTerminatedConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + when(mTerminatedConnectionManagerWorkflow.getState()) + .thenThrow(new IllegalStateException(EXCEPTION_MESSAGE)); + when(mTerminatedConnectionManagerWorkflow.getJobInformation()).thenReturn(new JobInformation(JOB_ID, ATTEMPT_ID)); + + final ConnectionManagerWorkflow mNewConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + final WorkflowState mWorkflowState = mock(WorkflowState.class); + when(mNewConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); + when(mWorkflowState.isDeleted()).thenReturn(false); + when(mWorkflowState.isRunning()).thenReturn(true).thenReturn(false); + when(mNewConnectionManagerWorkflow.getJobInformation()).thenReturn(new JobInformation(JOB_ID, ATTEMPT_ID)); + when(workflowClient.newWorkflowStub(any(Class.class), any(WorkflowOptions.class))).thenReturn(mNewConnectionManagerWorkflow); + final BatchRequest mBatchRequest = mock(BatchRequest.class); + when(workflowClient.newSignalWithStartRequest()).thenReturn(mBatchRequest); + + when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mTerminatedConnectionManagerWorkflow, mTerminatedConnectionManagerWorkflow, + mNewConnectionManagerWorkflow); + + final ManualOperationResult result = temporalClient.startNewCancellation(CONNECTION_ID); + + assertTrue(result.getJobId().isPresent()); + assertEquals(NON_RUNNING_JOB_ID, result.getJobId().get()); + assertFalse(result.getFailingReason().isPresent()); + verify(workflowClient).signalWithStart(mBatchRequest); + + // Verify that the cancelJob signal was passed to the batch request by capturing the argument, + // executing the signal, and verifying that the desired signal was executed + final ArgumentCaptor batchRequestAddArgCaptor = ArgumentCaptor.forClass(Proc.class); + verify(mBatchRequest).add(batchRequestAddArgCaptor.capture()); + final Proc signal = batchRequestAddArgCaptor.getValue(); + signal.apply(); + verify(mNewConnectionManagerWorkflow).cancelJob(); + } + + @Test + @SuppressWarnings(UNCHECKED) + @DisplayName("Test startNewCancellation returns a failure reason when connection is deleted") + void testStartNewCancellationDeletedWorkflow() { + final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + final WorkflowState mWorkflowState = mock(WorkflowState.class); + when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); + when(mWorkflowState.isDeleted()).thenReturn(true); + when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow); + mockWorkflowStatus(WorkflowExecutionStatus.WORKFLOW_EXECUTION_STATUS_COMPLETED); + + final ManualOperationResult result = temporalClient.startNewCancellation(CONNECTION_ID); + + // this is only called when updating an existing workflow + assertFalse(result.getJobId().isPresent()); + assertTrue(result.getFailingReason().isPresent()); + verify(mConnectionManagerWorkflow, times(0)).cancelJob(); + } + + } + + @Nested + @DisplayName("Test reset connection behavior") + class ResetConnection { + + @Test + @DisplayName("Test resetConnection successful") + void testResetConnectionSuccess() throws IOException { + final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + final WorkflowState mWorkflowState = mock(WorkflowState.class); + when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); + when(mWorkflowState.isDeleted()).thenReturn(false); + when(mWorkflowState.isRunning()).thenReturn(false); + final long jobId1 = 1; + final long jobId2 = 2; + when(mConnectionManagerWorkflow.getJobInformation()).thenReturn( + new JobInformation(jobId1, 0), + new JobInformation(jobId1, 0), + new JobInformation(NON_RUNNING_JOB_ID, 0), + new JobInformation(NON_RUNNING_JOB_ID, 0), + new JobInformation(jobId2, 0), + new JobInformation(jobId2, 0)); + when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow); + + final List streamsToReset = List.of(STREAM_DESCRIPTOR); + final ManualOperationResult result = temporalClient.resetConnection(CONNECTION_ID, streamsToReset, false); + + verify(streamResetPersistence).createStreamResets(CONNECTION_ID, streamsToReset); + + assertTrue(result.getJobId().isPresent()); + assertEquals(jobId2, result.getJobId().get()); + assertFalse(result.getFailingReason().isPresent()); + verify(mConnectionManagerWorkflow).resetConnection(); + } + + @Test + @DisplayName("Test resetConnection successful") + void testResetConnectionSuccessAndContinue() throws IOException { + final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + final WorkflowState mWorkflowState = mock(WorkflowState.class); + when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); + when(mWorkflowState.isDeleted()).thenReturn(false); + when(mWorkflowState.isRunning()).thenReturn(false); + final long jobId1 = 1; + final long jobId2 = 2; + when(mConnectionManagerWorkflow.getJobInformation()).thenReturn( + new JobInformation(jobId1, 0), + new JobInformation(jobId1, 0), + new JobInformation(NON_RUNNING_JOB_ID, 0), + new JobInformation(NON_RUNNING_JOB_ID, 0), + new JobInformation(jobId2, 0), + new JobInformation(jobId2, 0)); + when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow); + + final List streamsToReset = List.of(STREAM_DESCRIPTOR); + final ManualOperationResult result = temporalClient.resetConnection(CONNECTION_ID, streamsToReset, true); + + verify(streamResetPersistence).createStreamResets(CONNECTION_ID, streamsToReset); + + assertTrue(result.getJobId().isPresent()); + assertEquals(jobId2, result.getJobId().get()); + assertFalse(result.getFailingReason().isPresent()); + verify(mConnectionManagerWorkflow).resetConnectionAndSkipNextScheduling(); + } + + @Test + @DisplayName("Test resetConnection repairs the workflow if it is in a bad state") + void testResetConnectionRepairsBadWorkflowState() throws IOException { + final ConnectionManagerWorkflow mTerminatedConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + when(mTerminatedConnectionManagerWorkflow.getState()) + .thenThrow(new IllegalStateException(EXCEPTION_MESSAGE)); + when(mTerminatedConnectionManagerWorkflow.getJobInformation()).thenReturn(new JobInformation(JOB_ID, ATTEMPT_ID)); + + final ConnectionManagerWorkflow mNewConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + final WorkflowState mWorkflowState = mock(WorkflowState.class); + when(mNewConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); + when(mWorkflowState.isDeleted()).thenReturn(false); + when(mWorkflowState.isRunning()).thenReturn(false); + when(mNewConnectionManagerWorkflow.getJobInformation()).thenReturn( + new JobInformation(NON_RUNNING_JOB_ID, 0), + new JobInformation(NON_RUNNING_JOB_ID, 0), + new JobInformation(JOB_ID, 0), + new JobInformation(JOB_ID, 0)); + when(workflowClient.newWorkflowStub(any(Class.class), any(WorkflowOptions.class))).thenReturn(mNewConnectionManagerWorkflow); + final BatchRequest mBatchRequest = mock(BatchRequest.class); + when(workflowClient.newSignalWithStartRequest()).thenReturn(mBatchRequest); + + when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mTerminatedConnectionManagerWorkflow, mTerminatedConnectionManagerWorkflow, + mNewConnectionManagerWorkflow); + + final List streamsToReset = List.of(STREAM_DESCRIPTOR); + final ManualOperationResult result = temporalClient.resetConnection(CONNECTION_ID, streamsToReset, false); + + verify(streamResetPersistence).createStreamResets(CONNECTION_ID, streamsToReset); + + assertTrue(result.getJobId().isPresent()); + assertEquals(JOB_ID, result.getJobId().get()); + assertFalse(result.getFailingReason().isPresent()); + verify(workflowClient).signalWithStart(mBatchRequest); + + // Verify that the resetConnection signal was passed to the batch request by capturing the argument, + // executing the signal, and verifying that the desired signal was executed + final ArgumentCaptor batchRequestAddArgCaptor = ArgumentCaptor.forClass(Proc.class); + verify(mBatchRequest).add(batchRequestAddArgCaptor.capture()); + final Proc signal = batchRequestAddArgCaptor.getValue(); + signal.apply(); + verify(mNewConnectionManagerWorkflow).resetConnection(); + } + + @Test + @SuppressWarnings(UNCHECKED) + @DisplayName("Test resetConnection returns a failure reason when connection is deleted") + void testResetConnectionDeletedWorkflow() throws IOException { + final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + final WorkflowState mWorkflowState = mock(WorkflowState.class); + when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); + when(mWorkflowState.isDeleted()).thenReturn(true); + when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow); + mockWorkflowStatus(WorkflowExecutionStatus.WORKFLOW_EXECUTION_STATUS_COMPLETED); + + final List streamsToReset = List.of(STREAM_DESCRIPTOR); + final ManualOperationResult result = temporalClient.resetConnection(CONNECTION_ID, streamsToReset, false); + + verify(streamResetPersistence).createStreamResets(CONNECTION_ID, streamsToReset); + + // this is only called when updating an existing workflow + assertFalse(result.getJobId().isPresent()); + assertTrue(result.getFailingReason().isPresent()); + verify(mConnectionManagerWorkflow, times(0)).resetConnection(); + } + + } + + @Test + @DisplayName("Test manual operation on quarantined workflow causes a restart") + void testManualOperationOnQuarantinedWorkflow() { + final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + final WorkflowState mWorkflowState = mock(WorkflowState.class); + when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); + when(mWorkflowState.isQuarantined()).thenReturn(true); + + final ConnectionManagerWorkflow mNewConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + final WorkflowState mNewWorkflowState = mock(WorkflowState.class); + when(mNewConnectionManagerWorkflow.getState()).thenReturn(mNewWorkflowState); + when(mNewWorkflowState.isRunning()).thenReturn(false).thenReturn(true); + when(mNewConnectionManagerWorkflow.getJobInformation()).thenReturn(new JobInformation(JOB_ID, ATTEMPT_ID)); + when(workflowClient.newWorkflowStub(any(Class.class), any(WorkflowOptions.class))).thenReturn(mNewConnectionManagerWorkflow); + final BatchRequest mBatchRequest = mock(BatchRequest.class); + when(workflowClient.newSignalWithStartRequest()).thenReturn(mBatchRequest); + + when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow, mConnectionManagerWorkflow, + mNewConnectionManagerWorkflow); + + final WorkflowStub mWorkflowStub = mock(WorkflowStub.class); + when(workflowClient.newUntypedWorkflowStub(anyString())).thenReturn(mWorkflowStub); + + final ManualOperationResult result = temporalClient.startNewManualSync(CONNECTION_ID); + + assertTrue(result.getJobId().isPresent()); + assertEquals(JOB_ID, result.getJobId().get()); + assertFalse(result.getFailingReason().isPresent()); + verify(workflowClient).signalWithStart(mBatchRequest); + verify(mWorkflowStub).terminate(anyString()); + + // Verify that the submitManualSync signal was passed to the batch request by capturing the + // argument, + // executing the signal, and verifying that the desired signal was executed + final ArgumentCaptor batchRequestAddArgCaptor = ArgumentCaptor.forClass(Proc.class); + verify(mBatchRequest).add(batchRequestAddArgCaptor.capture()); + final Proc signal = batchRequestAddArgCaptor.getValue(); + signal.apply(); + verify(mNewConnectionManagerWorkflow).submitManualSync(); + } + + @Test + @DisplayName("Test manual operation on completed workflow causes a restart") + void testManualOperationOnCompletedWorkflow() { + final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + final WorkflowState mWorkflowState = mock(WorkflowState.class); + when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); + when(mWorkflowState.isQuarantined()).thenReturn(false); + when(mWorkflowState.isDeleted()).thenReturn(false); + when(workflowServiceBlockingStub.describeWorkflowExecution(any())) + .thenReturn(DescribeWorkflowExecutionResponse.newBuilder().setWorkflowExecutionInfo( + WorkflowExecutionInfo.newBuilder().setStatus(WorkflowExecutionStatus.WORKFLOW_EXECUTION_STATUS_COMPLETED).buildPartial()).build()) + .thenReturn(DescribeWorkflowExecutionResponse.newBuilder().setWorkflowExecutionInfo( + WorkflowExecutionInfo.newBuilder().setStatus(WorkflowExecutionStatus.WORKFLOW_EXECUTION_STATUS_RUNNING).buildPartial()).build()); + + final ConnectionManagerWorkflow mNewConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); + final WorkflowState mNewWorkflowState = mock(WorkflowState.class); + when(mNewConnectionManagerWorkflow.getState()).thenReturn(mNewWorkflowState); + when(mNewWorkflowState.isRunning()).thenReturn(false).thenReturn(true); + when(mNewConnectionManagerWorkflow.getJobInformation()).thenReturn(new JobInformation(JOB_ID, ATTEMPT_ID)); + when(workflowClient.newWorkflowStub(any(Class.class), any(WorkflowOptions.class))).thenReturn(mNewConnectionManagerWorkflow); + final BatchRequest mBatchRequest = mock(BatchRequest.class); + when(workflowClient.newSignalWithStartRequest()).thenReturn(mBatchRequest); + + when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow, mConnectionManagerWorkflow, + mNewConnectionManagerWorkflow); + + final WorkflowStub mWorkflowStub = mock(WorkflowStub.class); + when(workflowClient.newUntypedWorkflowStub(anyString())).thenReturn(mWorkflowStub); + + final ManualOperationResult result = temporalClient.startNewManualSync(CONNECTION_ID); + + assertTrue(result.getJobId().isPresent()); + assertEquals(JOB_ID, result.getJobId().get()); + assertFalse(result.getFailingReason().isPresent()); + verify(workflowClient).signalWithStart(mBatchRequest); + verify(mWorkflowStub).terminate(anyString()); + + // Verify that the submitManualSync signal was passed to the batch request by capturing the + // argument, + // executing the signal, and verifying that the desired signal was executed + final ArgumentCaptor batchRequestAddArgCaptor = ArgumentCaptor.forClass(Proc.class); + verify(mBatchRequest).add(batchRequestAddArgCaptor.capture()); + final Proc signal = batchRequestAddArgCaptor.getValue(); + signal.apply(); + verify(mNewConnectionManagerWorkflow).submitManualSync(); + } + private void mockWorkflowStatus(final WorkflowExecutionStatus status) { when(workflowServiceBlockingStub.describeWorkflowExecution(any())).thenReturn( DescribeWorkflowExecutionResponse.newBuilder().setWorkflowExecutionInfo( diff --git a/airbyte-commons-worker/build.gradle b/airbyte-commons-worker/build.gradle index b0fde55849bb0..0cf067e0e1c4e 100644 --- a/airbyte-commons-worker/build.gradle +++ b/airbyte-commons-worker/build.gradle @@ -1,8 +1,5 @@ -import org.jsonschema2pojo.SourceType - plugins { id "java-library" - id 'com.github.eirnym.js2p' version '1.0' } dependencies { @@ -17,13 +14,16 @@ dependencies { implementation 'org.apache.ant:ant:1.10.10' implementation 'org.apache.commons:commons-text:1.9' + implementation project(':airbyte-api') implementation project(':airbyte-commons-protocol') implementation project(':airbyte-commons-temporal') implementation project(':airbyte-config:config-models') + implementation project(':airbyte-config:config-persistence') implementation project(':airbyte-json-validation') implementation project(':airbyte-metrics:metrics-lib') implementation project(':airbyte-persistence:job-persistence') implementation project(':airbyte-protocol:protocol-models') + implementation project(':airbyte-worker-models') testAnnotationProcessor platform(libs.micronaut.bom) testAnnotationProcessor libs.bundles.micronaut.test.annotation.processor @@ -38,18 +38,4 @@ dependencies { testImplementation project(':airbyte-commons-docker') } -jsonSchema2Pojo { - sourceType = SourceType.YAMLSCHEMA - source = files("${sourceSets.main.output.resourcesDir}/workers_models") - targetDirectory = new File(project.buildDir, 'generated/src/gen/java/') - removeOldOutput = true - - targetPackage = 'io.airbyte.persistence.job.models' - - useLongIntegers = true - generateBuilders = true - includeConstructors = false - includeSetters = true -} - Task publishArtifactsTask = getPublishArtifactsTask("$rootProject.ext.version", project) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/WorkerUtils.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/WorkerUtils.java index 5c580417bc201..81da1aab53ec6 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/WorkerUtils.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/WorkerUtils.java @@ -11,15 +11,12 @@ import io.airbyte.config.StandardSyncInput; import io.airbyte.config.WorkerDestinationConfig; import io.airbyte.config.WorkerSourceConfig; -import io.airbyte.config.helpers.LogClientSingleton; -import io.airbyte.persistence.job.models.JobRunConfig; import io.airbyte.protocol.models.AirbyteMessage; import io.airbyte.protocol.models.AirbyteMessage.Type; import io.airbyte.protocol.models.AirbyteTraceMessage; import io.airbyte.workers.exception.WorkerException; import io.airbyte.workers.helper.FailureHelper; import io.airbyte.workers.helper.FailureHelper.ConnectorCommand; -import java.nio.file.Path; import java.time.Duration; import java.time.temporal.ChronoUnit; import java.util.ArrayList; @@ -149,24 +146,4 @@ public static String streamNameWithNamespace(final @Nullable String namespace, f return Objects.toString(namespace, "").trim() + streamName.trim(); } - // todo (cgardens) - there are 2 sources of truth for job path. we need to reduce this down to one, - // once we are fully on temporal. - public static Path getJobRoot(final Path workspaceRoot, final JobRunConfig jobRunConfig) { - return getJobRoot(workspaceRoot, jobRunConfig.getJobId(), jobRunConfig.getAttemptId()); - } - - public static Path getLogPath(final Path jobRoot) { - return jobRoot.resolve(LogClientSingleton.LOG_FILENAME); - } - - public static Path getJobRoot(final Path workspaceRoot, final String jobId, final long attemptId) { - return getJobRoot(workspaceRoot, jobId, Math.toIntExact(attemptId)); - } - - public static Path getJobRoot(final Path workspaceRoot, final String jobId, final int attemptId) { - return workspaceRoot - .resolve(String.valueOf(jobId)) - .resolve(String.valueOf(attemptId)); - } - } diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/helper/ConnectionHelper.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/ConnectionHelper.java similarity index 99% rename from airbyte-workers/src/main/java/io/airbyte/workers/helper/ConnectionHelper.java rename to airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/ConnectionHelper.java index 3232b25abbecf..628b2b2f070fc 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/helper/ConnectionHelper.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/ConnectionHelper.java @@ -7,6 +7,7 @@ import com.google.common.base.Preconditions; import io.airbyte.commons.enums.Enums; import io.airbyte.commons.json.Jsons; +import io.airbyte.commons.temporal.config.WorkerMode; import io.airbyte.config.BasicSchedule; import io.airbyte.config.JobSyncConfig.NamespaceDefinitionType; import io.airbyte.config.Schedule; @@ -17,7 +18,6 @@ import io.airbyte.config.persistence.ConfigRepository; import io.airbyte.persistence.job.WorkspaceHelper; import io.airbyte.validation.json.JsonValidationException; -import io.airbyte.workers.config.WorkerMode; import io.micronaut.context.annotation.Requires; import jakarta.inject.Singleton; import java.io.IOException; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/helper/ProtocolConverters.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/ProtocolConverters.java similarity index 100% rename from airbyte-workers/src/main/java/io/airbyte/workers/helper/ProtocolConverters.java rename to airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/ProtocolConverters.java diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/helper/StateConverter.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/StateConverter.java similarity index 100% rename from airbyte-workers/src/main/java/io/airbyte/workers/helper/StateConverter.java rename to airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/StateConverter.java diff --git a/airbyte-container-orchestrator/build.gradle b/airbyte-container-orchestrator/build.gradle index 98c4b42bf3f1e..e22f5217a0386 100644 --- a/airbyte-container-orchestrator/build.gradle +++ b/airbyte-container-orchestrator/build.gradle @@ -21,6 +21,7 @@ dependencies { implementation project(':airbyte-protocol:protocol-models') implementation project(':airbyte-persistence:job-persistence') implementation project(':airbyte-metrics:metrics-lib') + implementation project(':airbyte-worker-models') testImplementation 'org.mockito:mockito-inline:2.13.0' testImplementation libs.postgresql diff --git a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/ContainerOrchestratorApp.java b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/ContainerOrchestratorApp.java index 1d22e8af5f6b4..22f82426b4947 100644 --- a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/ContainerOrchestratorApp.java +++ b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/ContainerOrchestratorApp.java @@ -8,13 +8,13 @@ import io.airbyte.commons.features.FeatureFlags; import io.airbyte.commons.logging.LoggingHelper; import io.airbyte.commons.logging.MdcScope; +import io.airbyte.commons.temporal.TemporalUtils; import io.airbyte.commons.temporal.sync.OrchestratorConstants; import io.airbyte.config.Configs; import io.airbyte.config.EnvConfigs; import io.airbyte.config.helpers.LogClientSingleton; import io.airbyte.persistence.job.models.JobRunConfig; import io.airbyte.workers.WorkerConfigs; -import io.airbyte.workers.WorkerUtils; import io.airbyte.workers.process.AsyncKubePodStatus; import io.airbyte.workers.process.AsyncOrchestratorPodProcess; import io.airbyte.workers.process.DockerProcessFactory; @@ -99,7 +99,7 @@ private void configureLogging() { logClient.setJobMdc( configs.getWorkerEnvironment(), configs.getLogConfigs(), - WorkerUtils.getJobRoot(configs.getWorkspaceRoot(), jobRunConfig.getJobId(), jobRunConfig.getAttemptId())); + TemporalUtils.getJobRoot(configs.getWorkspaceRoot(), jobRunConfig.getJobId(), jobRunConfig.getAttemptId())); } /** diff --git a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/DbtJobOrchestrator.java b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/DbtJobOrchestrator.java index b8c659e566f2b..589bbb23d5ff2 100644 --- a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/DbtJobOrchestrator.java +++ b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/DbtJobOrchestrator.java @@ -4,12 +4,12 @@ package io.airbyte.container_orchestrator; +import io.airbyte.commons.temporal.TemporalUtils; import io.airbyte.config.Configs; import io.airbyte.config.OperatorDbtInput; import io.airbyte.persistence.job.models.IntegrationLauncherConfig; import io.airbyte.persistence.job.models.JobRunConfig; import io.airbyte.workers.WorkerConfigs; -import io.airbyte.workers.WorkerUtils; import io.airbyte.workers.general.DbtTransformationRunner; import io.airbyte.workers.general.DbtTransformationWorker; import io.airbyte.workers.normalization.NormalizationRunnerFactory; @@ -64,7 +64,7 @@ public Optional runJob() throws Exception { NormalizationRunnerFactory.NORMALIZATION_VERSION))); log.info("Running dbt worker..."); - final Path jobRoot = WorkerUtils.getJobRoot(configs.getWorkspaceRoot(), jobRunConfig.getJobId(), jobRunConfig.getAttemptId()); + final Path jobRoot = TemporalUtils.getJobRoot(configs.getWorkspaceRoot(), jobRunConfig.getJobId(), jobRunConfig.getAttemptId()); worker.run(dbtInput, jobRoot); return Optional.empty(); diff --git a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/NormalizationJobOrchestrator.java b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/NormalizationJobOrchestrator.java index bf3387a1a11cc..34116b594601e 100644 --- a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/NormalizationJobOrchestrator.java +++ b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/NormalizationJobOrchestrator.java @@ -5,12 +5,12 @@ package io.airbyte.container_orchestrator; import io.airbyte.commons.json.Jsons; +import io.airbyte.commons.temporal.TemporalUtils; import io.airbyte.config.Configs; import io.airbyte.config.NormalizationInput; import io.airbyte.config.NormalizationSummary; import io.airbyte.persistence.job.models.IntegrationLauncherConfig; import io.airbyte.persistence.job.models.JobRunConfig; -import io.airbyte.workers.WorkerUtils; import io.airbyte.workers.general.DefaultNormalizationWorker; import io.airbyte.workers.normalization.NormalizationRunnerFactory; import io.airbyte.workers.normalization.NormalizationWorker; @@ -62,7 +62,7 @@ public Optional runJob() throws Exception { configs.getWorkerEnvironment()); log.info("Running normalization worker..."); - final Path jobRoot = WorkerUtils.getJobRoot(configs.getWorkspaceRoot(), jobRunConfig.getJobId(), jobRunConfig.getAttemptId()); + final Path jobRoot = TemporalUtils.getJobRoot(configs.getWorkspaceRoot(), jobRunConfig.getJobId(), jobRunConfig.getAttemptId()); final NormalizationSummary normalizationSummary = normalizationWorker.run(normalizationInput, jobRoot); return Optional.of(Jsons.serialize(normalizationSummary)); diff --git a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/ReplicationJobOrchestrator.java b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/ReplicationJobOrchestrator.java index 728851a740a0d..4c71979210ee6 100644 --- a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/ReplicationJobOrchestrator.java +++ b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/ReplicationJobOrchestrator.java @@ -6,6 +6,7 @@ import io.airbyte.commons.features.FeatureFlags; import io.airbyte.commons.json.Jsons; +import io.airbyte.commons.temporal.TemporalUtils; import io.airbyte.config.Configs; import io.airbyte.config.ReplicationOutput; import io.airbyte.config.StandardSyncInput; @@ -112,7 +113,7 @@ public Optional runJob() throws Exception { metricReporter); log.info("Running replication worker..."); - final Path jobRoot = WorkerUtils.getJobRoot(configs.getWorkspaceRoot(), jobRunConfig.getJobId(), jobRunConfig.getAttemptId()); + final Path jobRoot = TemporalUtils.getJobRoot(configs.getWorkspaceRoot(), jobRunConfig.getJobId(), jobRunConfig.getAttemptId()); final ReplicationOutput replicationOutput = replicationWorker.run(syncInput, jobRoot); log.info("Returning output..."); diff --git a/airbyte-cron/build.gradle b/airbyte-cron/build.gradle index c718f52bab29f..548c5d13efb46 100644 --- a/airbyte-cron/build.gradle +++ b/airbyte-cron/build.gradle @@ -18,6 +18,7 @@ dependencies { implementation project(':airbyte-json-validation') implementation project(':airbyte-db:db-lib') implementation project(':airbyte-metrics:metrics-lib') + implementation project(':airbyte-persistence:job-persistence') annotationProcessor platform(libs.micronaut.bom) annotationProcessor libs.bundles.micronaut.annotation.processor diff --git a/airbyte-cron/src/main/java/io/airbyte/cron/config/DatabaseBeanFactory.java b/airbyte-cron/src/main/java/io/airbyte/cron/config/DatabaseBeanFactory.java index e546dc6acde74..d3ecc3a2de9d4 100644 --- a/airbyte-cron/src/main/java/io/airbyte/cron/config/DatabaseBeanFactory.java +++ b/airbyte-cron/src/main/java/io/airbyte/cron/config/DatabaseBeanFactory.java @@ -4,14 +4,19 @@ package io.airbyte.cron.config; +import io.airbyte.commons.temporal.config.WorkerMode; import io.airbyte.config.persistence.ConfigPersistence; import io.airbyte.config.persistence.ConfigRepository; import io.airbyte.config.persistence.DatabaseConfigPersistence; +import io.airbyte.config.persistence.StreamResetPersistence; import io.airbyte.config.persistence.split_secrets.JsonSecretsProcessor; import io.airbyte.db.Database; import io.airbyte.db.check.DatabaseMigrationCheck; import io.airbyte.db.factory.DatabaseCheckFactory; +import io.airbyte.persistence.job.DefaultJobPersistence; +import io.airbyte.persistence.job.JobPersistence; import io.micronaut.context.annotation.Factory; +import io.micronaut.context.annotation.Requires; import io.micronaut.context.annotation.Value; import io.micronaut.flyway.FlywayConfigurationProperties; import jakarta.inject.Named; @@ -40,6 +45,13 @@ public Database configDatabase(@Named("config") final DSLContext dslContext) thr return new Database(dslContext); } + @Singleton + @Requires(env = WorkerMode.CONTROL_PLANE) + @Named("jobsDatabase") + public Database jobsDatabase(@Named("jobs") final DSLContext dslContext) throws IOException { + return new Database(dslContext); + } + @Singleton @Named("configFlyway") public Flyway configFlyway(@Named("config") final FlywayConfigurationProperties configFlywayConfigurationProperties, @@ -79,4 +91,16 @@ public DatabaseMigrationCheck configsDatabaseMigrationCheck(@Named("config") fin configsDatabaseInitializationTimeoutMs); } + @Singleton + @Requires(env = WorkerMode.CONTROL_PLANE) + public StreamResetPersistence streamResetPersistence(@Named("configDatabase") final Database configDatabase) { + return new StreamResetPersistence(configDatabase); + } + + @Singleton + @Requires(env = WorkerMode.CONTROL_PLANE) + public JobPersistence jobPersistence(@Named("jobsDatabase") final Database jobDatabase) { + return new DefaultJobPersistence(jobDatabase); + } + } diff --git a/airbyte-cron/src/main/java/io/airbyte/cron/selfhealing/Temporal.java b/airbyte-cron/src/main/java/io/airbyte/cron/selfhealing/Temporal.java index 0b3848b0f26b7..39a0bc87f2d8b 100644 --- a/airbyte-cron/src/main/java/io/airbyte/cron/selfhealing/Temporal.java +++ b/airbyte-cron/src/main/java/io/airbyte/cron/selfhealing/Temporal.java @@ -7,7 +7,6 @@ import io.airbyte.commons.temporal.TemporalClient; import io.micronaut.scheduling.annotation.Scheduled; import io.temporal.api.enums.v1.WorkflowExecutionStatus; -import jakarta.inject.Named; import jakarta.inject.Singleton; import lombok.extern.slf4j.Slf4j; @@ -17,7 +16,7 @@ public class Temporal { private final TemporalClient temporalClient; - public Temporal(@Named("temporalClient") final TemporalClient temporalClient) { + public Temporal(final TemporalClient temporalClient) { log.debug("Creating temporal self-healing"); this.temporalClient = temporalClient; } diff --git a/airbyte-cron/src/main/resources/application.yml b/airbyte-cron/src/main/resources/application.yml index 3d18e233014e3..f291a5e7dd38d 100644 --- a/airbyte-cron/src/main/resources/application.yml +++ b/airbyte-cron/src/main/resources/application.yml @@ -15,7 +15,18 @@ datasources: connection-test-query: SELECT 1 connection-timeout: 30000 idle-timeout: 600000 - maximum-pool-size: 10 + maximum-pool-size: 5 + minimum-idle: 0 + url: ${DATABASE_URL} + driverClassName: org.postgresql.Driver + username: ${DATABASE_USER} + password: ${DATABASE_PASSWORD} + jobs: + connection-test-query: SELECT 1 + connection-timeout: 30000 + idle-timeout: 600000 + maximum-pool-size: 5 + minimum-idle: 0 url: ${DATABASE_URL} driverClassName: org.postgresql.Driver username: ${DATABASE_USER} diff --git a/airbyte-server/build.gradle b/airbyte-server/build.gradle index b39aad4326968..b38e879fcb5bd 100644 --- a/airbyte-server/build.gradle +++ b/airbyte-server/build.gradle @@ -4,6 +4,7 @@ plugins { configurations.all { exclude group: 'io.micronaut.jaxrs' + exclude group: 'io.micronaut.sql' } dependencies { @@ -22,14 +23,6 @@ dependencies { implementation project(':airbyte-oauth') implementation project(':airbyte-protocol:protocol-models') implementation project(':airbyte-persistence:job-persistence') - implementation(project(':airbyte-workers')) { - // Temporary hack to avoid dependency conflicts - exclude group: 'io.micronaut' - exclude group: 'io.micronaut.flyway' - exclude group: 'io.micronaut.jaxrs' - exclude group: 'io.micronaut.security' - exclude group: 'io.micronaut.sql' - } implementation libs.flyway.core implementation 'com.github.slugify:slugify:2.4' diff --git a/airbyte-server/src/main/java/io/airbyte/server/ServerApp.java b/airbyte-server/src/main/java/io/airbyte/server/ServerApp.java index c50b443e203da..ee59c365d776b 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/ServerApp.java +++ b/airbyte-server/src/main/java/io/airbyte/server/ServerApp.java @@ -10,6 +10,9 @@ import io.airbyte.analytics.TrackingClientSingleton; import io.airbyte.commons.lang.CloseableShutdownHook; import io.airbyte.commons.resources.MoreResources; +import io.airbyte.commons.temporal.ConnectionManagerUtils; +import io.airbyte.commons.temporal.StreamResetRecordsHelper; +import io.airbyte.commons.temporal.TemporalClient; import io.airbyte.commons.temporal.TemporalUtils; import io.airbyte.commons.temporal.TemporalWorkflowUtils; import io.airbyte.commons.version.AirbyteVersion; @@ -56,9 +59,6 @@ import io.airbyte.server.scheduler.TemporalEventRunner; import io.airbyte.validation.json.JsonValidationException; import io.airbyte.workers.normalization.NormalizationRunnerFactory; -import io.airbyte.workers.temporal.ConnectionManagerUtils; -import io.airbyte.workers.temporal.StreamResetRecordsHelper; -import io.airbyte.workers.temporal.TemporalClient; import io.temporal.serviceclient.WorkflowServiceStubs; import java.io.IOException; import java.net.http.HttpClient; diff --git a/airbyte-server/src/main/java/io/airbyte/server/handlers/SchedulerHandler.java b/airbyte-server/src/main/java/io/airbyte/server/handlers/SchedulerHandler.java index 3cfa7983d45b4..c936e5768625a 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/handlers/SchedulerHandler.java +++ b/airbyte-server/src/main/java/io/airbyte/server/handlers/SchedulerHandler.java @@ -35,6 +35,8 @@ import io.airbyte.commons.docker.DockerUtils; import io.airbyte.commons.enums.Enums; import io.airbyte.commons.json.Jsons; +import io.airbyte.commons.temporal.ErrorCode; +import io.airbyte.commons.temporal.TemporalClient.ManualOperationResult; import io.airbyte.commons.version.Version; import io.airbyte.config.ActorCatalog; import io.airbyte.config.Configs.WorkerEnvironment; @@ -64,8 +66,6 @@ import io.airbyte.server.scheduler.SynchronousSchedulerClient; import io.airbyte.validation.json.JsonSchemaValidator; import io.airbyte.validation.json.JsonValidationException; -import io.airbyte.workers.temporal.ErrorCode; -import io.airbyte.workers.temporal.TemporalClient.ManualOperationResult; import java.io.IOException; import java.util.ArrayList; import java.util.Optional; diff --git a/airbyte-server/src/main/java/io/airbyte/server/scheduler/DefaultSynchronousSchedulerClient.java b/airbyte-server/src/main/java/io/airbyte/server/scheduler/DefaultSynchronousSchedulerClient.java index d49a3efdd29bf..8ad8e23dda809 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/scheduler/DefaultSynchronousSchedulerClient.java +++ b/airbyte-server/src/main/java/io/airbyte/server/scheduler/DefaultSynchronousSchedulerClient.java @@ -11,6 +11,8 @@ import com.google.common.hash.Hashing; import io.airbyte.commons.json.Jsons; import io.airbyte.commons.lang.Exceptions; +import io.airbyte.commons.temporal.TemporalClient; +import io.airbyte.commons.temporal.TemporalResponse; import io.airbyte.commons.version.Version; import io.airbyte.config.ConnectorJobOutput; import io.airbyte.config.DestinationConnection; @@ -26,8 +28,6 @@ import io.airbyte.persistence.job.tracker.JobTracker; import io.airbyte.persistence.job.tracker.JobTracker.JobState; import io.airbyte.protocol.models.ConnectorSpecification; -import io.airbyte.workers.temporal.TemporalClient; -import io.airbyte.workers.temporal.TemporalResponse; import java.io.IOException; import java.time.Instant; import java.util.Optional; diff --git a/airbyte-server/src/main/java/io/airbyte/server/scheduler/EventRunner.java b/airbyte-server/src/main/java/io/airbyte/server/scheduler/EventRunner.java index 56d8fb8cae448..ca2b47bb6cc32 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/scheduler/EventRunner.java +++ b/airbyte-server/src/main/java/io/airbyte/server/scheduler/EventRunner.java @@ -4,8 +4,8 @@ package io.airbyte.server.scheduler; +import io.airbyte.commons.temporal.TemporalClient.ManualOperationResult; import io.airbyte.protocol.models.StreamDescriptor; -import io.airbyte.workers.temporal.TemporalClient.ManualOperationResult; import java.util.List; import java.util.Set; import java.util.UUID; diff --git a/airbyte-server/src/main/java/io/airbyte/server/scheduler/SynchronousJobMetadata.java b/airbyte-server/src/main/java/io/airbyte/server/scheduler/SynchronousJobMetadata.java index 196b009711513..37d862fb2ac96 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/scheduler/SynchronousJobMetadata.java +++ b/airbyte-server/src/main/java/io/airbyte/server/scheduler/SynchronousJobMetadata.java @@ -4,8 +4,8 @@ package io.airbyte.server.scheduler; +import io.airbyte.commons.temporal.JobMetadata; import io.airbyte.config.JobConfig.ConfigType; -import io.airbyte.workers.temporal.JobMetadata; import java.nio.file.Path; import java.time.Instant; import java.util.Objects; diff --git a/airbyte-server/src/main/java/io/airbyte/server/scheduler/SynchronousResponse.java b/airbyte-server/src/main/java/io/airbyte/server/scheduler/SynchronousResponse.java index 3905a3466fed4..5227c5b51a84c 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/scheduler/SynchronousResponse.java +++ b/airbyte-server/src/main/java/io/airbyte/server/scheduler/SynchronousResponse.java @@ -4,8 +4,8 @@ package io.airbyte.server.scheduler; +import io.airbyte.commons.temporal.TemporalResponse; import io.airbyte.config.JobConfig.ConfigType; -import io.airbyte.workers.temporal.TemporalResponse; import java.util.Objects; import java.util.UUID; diff --git a/airbyte-server/src/main/java/io/airbyte/server/scheduler/TemporalEventRunner.java b/airbyte-server/src/main/java/io/airbyte/server/scheduler/TemporalEventRunner.java index 1ff3cef84a016..79b54ce9a0580 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/scheduler/TemporalEventRunner.java +++ b/airbyte-server/src/main/java/io/airbyte/server/scheduler/TemporalEventRunner.java @@ -4,9 +4,9 @@ package io.airbyte.server.scheduler; +import io.airbyte.commons.temporal.TemporalClient; +import io.airbyte.commons.temporal.TemporalClient.ManualOperationResult; import io.airbyte.protocol.models.StreamDescriptor; -import io.airbyte.workers.temporal.TemporalClient; -import io.airbyte.workers.temporal.TemporalClient.ManualOperationResult; import java.util.List; import java.util.Set; import java.util.UUID; diff --git a/airbyte-server/src/test/java/io/airbyte/server/handlers/SchedulerHandlerTest.java b/airbyte-server/src/test/java/io/airbyte/server/handlers/SchedulerHandlerTest.java index d7997987a9024..5e2953190af06 100644 --- a/airbyte-server/src/test/java/io/airbyte/server/handlers/SchedulerHandlerTest.java +++ b/airbyte-server/src/test/java/io/airbyte/server/handlers/SchedulerHandlerTest.java @@ -40,6 +40,8 @@ import io.airbyte.commons.enums.Enums; import io.airbyte.commons.json.Jsons; import io.airbyte.commons.lang.Exceptions; +import io.airbyte.commons.temporal.ErrorCode; +import io.airbyte.commons.temporal.TemporalClient.ManualOperationResult; import io.airbyte.commons.version.Version; import io.airbyte.config.ActorCatalog; import io.airbyte.config.Configs.WorkerEnvironment; @@ -73,8 +75,6 @@ import io.airbyte.server.scheduler.SynchronousSchedulerClient; import io.airbyte.validation.json.JsonSchemaValidator; import io.airbyte.validation.json.JsonValidationException; -import io.airbyte.workers.temporal.ErrorCode; -import io.airbyte.workers.temporal.TemporalClient.ManualOperationResult; import java.io.IOException; import java.net.URI; import java.util.HashMap; diff --git a/airbyte-server/src/test/java/io/airbyte/server/handlers/WebBackendConnectionsHandlerTest.java b/airbyte-server/src/test/java/io/airbyte/server/handlers/WebBackendConnectionsHandlerTest.java index bfd50d3863d9a..72754b65c6311 100644 --- a/airbyte-server/src/test/java/io/airbyte/server/handlers/WebBackendConnectionsHandlerTest.java +++ b/airbyte-server/src/test/java/io/airbyte/server/handlers/WebBackendConnectionsHandlerTest.java @@ -66,6 +66,7 @@ import io.airbyte.api.model.generated.WebBackendWorkspaceState; import io.airbyte.api.model.generated.WorkspaceIdRequestBody; import io.airbyte.commons.enums.Enums; +import io.airbyte.commons.temporal.TemporalClient.ManualOperationResult; import io.airbyte.config.DestinationConnection; import io.airbyte.config.SourceConnection; import io.airbyte.config.StandardDestinationDefinition; @@ -87,7 +88,6 @@ import io.airbyte.server.helpers.SourceHelpers; import io.airbyte.server.scheduler.EventRunner; import io.airbyte.validation.json.JsonValidationException; -import io.airbyte.workers.temporal.TemporalClient.ManualOperationResult; import java.io.IOException; import java.lang.reflect.Method; import java.time.Instant; diff --git a/airbyte-server/src/test/java/io/airbyte/server/scheduler/DefaultSynchronousSchedulerClientTest.java b/airbyte-server/src/test/java/io/airbyte/server/scheduler/DefaultSynchronousSchedulerClientTest.java index 85415dfa2065a..8cf86db022c0e 100644 --- a/airbyte-server/src/test/java/io/airbyte/server/scheduler/DefaultSynchronousSchedulerClientTest.java +++ b/airbyte-server/src/test/java/io/airbyte/server/scheduler/DefaultSynchronousSchedulerClientTest.java @@ -20,6 +20,9 @@ import com.fasterxml.jackson.databind.JsonNode; import com.google.common.collect.ImmutableMap; import io.airbyte.commons.json.Jsons; +import io.airbyte.commons.temporal.JobMetadata; +import io.airbyte.commons.temporal.TemporalClient; +import io.airbyte.commons.temporal.TemporalResponse; import io.airbyte.commons.version.Version; import io.airbyte.config.ConnectorJobOutput; import io.airbyte.config.DestinationConnection; @@ -35,9 +38,6 @@ import io.airbyte.persistence.job.tracker.JobTracker; import io.airbyte.persistence.job.tracker.JobTracker.JobState; import io.airbyte.protocol.models.ConnectorSpecification; -import io.airbyte.workers.temporal.JobMetadata; -import io.airbyte.workers.temporal.TemporalClient; -import io.airbyte.workers.temporal.TemporalResponse; import java.io.IOException; import java.nio.file.Path; import java.util.UUID; diff --git a/airbyte-worker-models/build.gradle b/airbyte-worker-models/build.gradle new file mode 100644 index 0000000000000..57796fd0246fb --- /dev/null +++ b/airbyte-worker-models/build.gradle @@ -0,0 +1,22 @@ +import org.jsonschema2pojo.SourceType + +plugins { + id "java-library" + id 'com.github.eirnym.js2p' version '1.0' +} + +jsonSchema2Pojo { + sourceType = SourceType.YAMLSCHEMA + source = files("${sourceSets.main.output.resourcesDir}/workers_models") + targetDirectory = new File(project.buildDir, 'generated/src/gen/java/') + removeOldOutput = true + + targetPackage = 'io.airbyte.persistence.job.models' + + useLongIntegers = true + generateBuilders = true + includeConstructors = false + includeSetters = true +} + +Task publishArtifactsTask = getPublishArtifactsTask("$rootProject.ext.version", project) diff --git a/airbyte-commons-worker/src/main/resources/workers_models/IntegrationLauncherConfig.yaml b/airbyte-worker-models/src/main/resources/workers_models/IntegrationLauncherConfig.yaml similarity index 100% rename from airbyte-commons-worker/src/main/resources/workers_models/IntegrationLauncherConfig.yaml rename to airbyte-worker-models/src/main/resources/workers_models/IntegrationLauncherConfig.yaml diff --git a/airbyte-commons-temporal/src/main/resources/workers_models/JobRunConfig.yaml b/airbyte-worker-models/src/main/resources/workers_models/JobRunConfig.yaml similarity index 100% rename from airbyte-commons-temporal/src/main/resources/workers_models/JobRunConfig.yaml rename to airbyte-worker-models/src/main/resources/workers_models/JobRunConfig.yaml diff --git a/airbyte-workers/build.gradle b/airbyte-workers/build.gradle index 94626fdaab9fe..885400e6363f8 100644 --- a/airbyte-workers/build.gradle +++ b/airbyte-workers/build.gradle @@ -58,7 +58,7 @@ dependencies { exclude group: 'io.micronaut.security' exclude group: 'io.micronaut.sql' } - implementation project(':airbyte-api') + implementation project(':airbyte-worker-models') testAnnotationProcessor platform(libs.micronaut.bom) testAnnotationProcessor libs.bundles.micronaut.test.annotation.processor diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/ApplicationInitializer.java b/airbyte-workers/src/main/java/io/airbyte/workers/ApplicationInitializer.java index d183f4a3b816c..bef334c310dc1 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/ApplicationInitializer.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/ApplicationInitializer.java @@ -7,6 +7,7 @@ import io.airbyte.commons.temporal.TemporalInitializationUtils; import io.airbyte.commons.temporal.TemporalJobType; import io.airbyte.commons.temporal.TemporalUtils; +import io.airbyte.commons.temporal.config.WorkerMode; import io.airbyte.config.Configs.WorkerEnvironment; import io.airbyte.config.MaxWorkersConfig; import io.airbyte.config.helpers.LogClientSingleton; @@ -16,7 +17,6 @@ import io.airbyte.db.check.impl.JobsDatabaseAvailabilityCheck; import io.airbyte.metrics.lib.MetricClientFactory; import io.airbyte.metrics.lib.MetricEmittingApps; -import io.airbyte.workers.config.WorkerMode; import io.airbyte.workers.process.KubePortManagerSingleton; import io.airbyte.workers.temporal.check.connection.CheckConnectionWorkflowImpl; import io.airbyte.workers.temporal.discover.catalog.DiscoverCatalogWorkflowImpl; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/config/ActivityBeanFactory.java b/airbyte-workers/src/main/java/io/airbyte/workers/config/ActivityBeanFactory.java index 04f86556a5010..946cd936e562c 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/config/ActivityBeanFactory.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/config/ActivityBeanFactory.java @@ -5,6 +5,7 @@ package io.airbyte.workers.config; import io.airbyte.commons.temporal.TemporalUtils; +import io.airbyte.commons.temporal.config.WorkerMode; import io.airbyte.workers.exception.WorkerException; import io.airbyte.workers.temporal.check.connection.CheckConnectionActivity; import io.airbyte.workers.temporal.discover.catalog.DiscoverCatalogActivity; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/config/ApiClientBeanFactory.java b/airbyte-workers/src/main/java/io/airbyte/workers/config/ApiClientBeanFactory.java index 4d7e319b0f4de..c30415dd468c1 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/config/ApiClientBeanFactory.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/config/ApiClientBeanFactory.java @@ -9,6 +9,7 @@ import com.auth0.jwt.algorithms.Algorithm; import com.google.auth.oauth2.ServiceAccountCredentials; import io.airbyte.api.client.AirbyteApiClient; +import io.airbyte.commons.temporal.config.WorkerMode; import io.micronaut.context.BeanProvider; import io.micronaut.context.annotation.Factory; import io.micronaut.context.annotation.Prototype; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/config/ApplicationBeanFactory.java b/airbyte-workers/src/main/java/io/airbyte/workers/config/ApplicationBeanFactory.java index e0e7c650cacf8..79efd27cf14ec 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/config/ApplicationBeanFactory.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/config/ApplicationBeanFactory.java @@ -7,6 +7,7 @@ import io.airbyte.analytics.TrackingClient; import io.airbyte.commons.features.EnvVariableFeatureFlags; import io.airbyte.commons.features.FeatureFlags; +import io.airbyte.commons.temporal.config.WorkerMode; import io.airbyte.commons.version.AirbyteVersion; import io.airbyte.config.AirbyteConfigValidator; import io.airbyte.config.Configs.DeploymentMode; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/config/DatabaseBeanFactory.java b/airbyte-workers/src/main/java/io/airbyte/workers/config/DatabaseBeanFactory.java index 9403a1c21014b..6f91a8de5931b 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/config/DatabaseBeanFactory.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/config/DatabaseBeanFactory.java @@ -4,6 +4,7 @@ package io.airbyte.workers.config; +import io.airbyte.commons.temporal.config.WorkerMode; import io.airbyte.config.persistence.ConfigPersistence; import io.airbyte.config.persistence.ConfigRepository; import io.airbyte.config.persistence.DatabaseConfigPersistence; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/config/JobErrorReportingBeanFactory.java b/airbyte-workers/src/main/java/io/airbyte/workers/config/JobErrorReportingBeanFactory.java index f5ba3350d35a3..f9f6c8aeb03d7 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/config/JobErrorReportingBeanFactory.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/config/JobErrorReportingBeanFactory.java @@ -4,6 +4,7 @@ package io.airbyte.workers.config; +import io.airbyte.commons.temporal.config.WorkerMode; import io.airbyte.config.Configs.DeploymentMode; import io.airbyte.config.persistence.ConfigRepository; import io.airbyte.persistence.job.WebUrlHelper; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/config/ProcessFactoryBeanFactory.java b/airbyte-workers/src/main/java/io/airbyte/workers/config/ProcessFactoryBeanFactory.java index e3f499624fc46..dab9f21505c19 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/config/ProcessFactoryBeanFactory.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/config/ProcessFactoryBeanFactory.java @@ -4,6 +4,7 @@ package io.airbyte.workers.config; +import io.airbyte.commons.temporal.config.WorkerMode; import io.airbyte.workers.WorkerConfigs; import io.airbyte.workers.process.DockerProcessFactory; import io.airbyte.workers.process.KubeProcessFactory; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/config/SecretPersistenceBeanFactory.java b/airbyte-workers/src/main/java/io/airbyte/workers/config/SecretPersistenceBeanFactory.java index 1b15ce5b2c4b3..ab067f4de4b0b 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/config/SecretPersistenceBeanFactory.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/config/SecretPersistenceBeanFactory.java @@ -4,6 +4,7 @@ package io.airbyte.workers.config; +import io.airbyte.commons.temporal.config.WorkerMode; import io.airbyte.config.persistence.split_secrets.GoogleSecretManagerPersistence; import io.airbyte.config.persistence.split_secrets.LocalTestingSecretPersistence; import io.airbyte.config.persistence.split_secrets.RealSecretsHydrator; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/config/TemporalBeanFactory.java b/airbyte-workers/src/main/java/io/airbyte/workers/config/TemporalBeanFactory.java index 96aa20611129a..22d2a01f6e152 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/config/TemporalBeanFactory.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/config/TemporalBeanFactory.java @@ -8,7 +8,9 @@ import io.airbyte.analytics.TrackingClient; import io.airbyte.analytics.TrackingClientSingleton; import io.airbyte.commons.features.FeatureFlags; +import io.airbyte.commons.temporal.TemporalClient; import io.airbyte.commons.temporal.TemporalUtils; +import io.airbyte.commons.temporal.config.WorkerMode; import io.airbyte.commons.version.AirbyteVersion; import io.airbyte.config.Configs.DeploymentMode; import io.airbyte.config.Configs.TrackingStrategy; @@ -20,7 +22,6 @@ import io.airbyte.persistence.job.factory.OAuthConfigSupplier; import io.airbyte.persistence.job.factory.SyncJobFactory; import io.airbyte.workers.run.TemporalWorkerRunFactory; -import io.airbyte.workers.temporal.TemporalClient; import io.micronaut.context.annotation.Factory; import io.micronaut.context.annotation.Property; import io.micronaut.context.annotation.Requires; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/config/WorkerConfigurationBeanFactory.java b/airbyte-workers/src/main/java/io/airbyte/workers/config/WorkerConfigurationBeanFactory.java index f418b84235a69..fbee2323e0f91 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/config/WorkerConfigurationBeanFactory.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/config/WorkerConfigurationBeanFactory.java @@ -7,6 +7,7 @@ import com.google.common.base.Splitter; import com.google.common.base.Strings; import io.airbyte.commons.map.MoreMaps; +import io.airbyte.commons.temporal.config.WorkerMode; import io.airbyte.config.Configs.DeploymentMode; import io.airbyte.config.Configs.WorkerEnvironment; import io.airbyte.config.ResourceRequirements; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/run/TemporalWorkerRunFactory.java b/airbyte-workers/src/main/java/io/airbyte/workers/run/TemporalWorkerRunFactory.java index 44bcb8a1a400d..046e06c214945 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/run/TemporalWorkerRunFactory.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/run/TemporalWorkerRunFactory.java @@ -7,7 +7,9 @@ import io.airbyte.commons.features.FeatureFlags; import io.airbyte.commons.functional.CheckedSupplier; import io.airbyte.commons.json.Jsons; +import io.airbyte.commons.temporal.TemporalClient; import io.airbyte.commons.temporal.TemporalJobType; +import io.airbyte.commons.temporal.TemporalResponse; import io.airbyte.config.JobConfig.ConfigType; import io.airbyte.config.JobOutput; import io.airbyte.config.JobResetConnectionConfig; @@ -18,8 +20,6 @@ import io.airbyte.workers.JobStatus; import io.airbyte.workers.OutputAndStatus; import io.airbyte.workers.WorkerConstants; -import io.airbyte.workers.temporal.TemporalClient; -import io.airbyte.workers.temporal.TemporalResponse; import java.nio.file.Path; import java.util.UUID; import lombok.AllArgsConstructor; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/run/WorkerRun.java b/airbyte-workers/src/main/java/io/airbyte/workers/run/WorkerRun.java index 33dd11a693060..2a9bda5518984 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/run/WorkerRun.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/run/WorkerRun.java @@ -5,9 +5,9 @@ package io.airbyte.workers.run; import io.airbyte.commons.functional.CheckedSupplier; +import io.airbyte.commons.temporal.TemporalUtils; import io.airbyte.config.JobOutput; import io.airbyte.workers.OutputAndStatus; -import io.airbyte.workers.WorkerUtils; import java.nio.file.Files; import java.nio.file.Path; import java.util.concurrent.Callable; @@ -33,7 +33,7 @@ public static WorkerRun create(final Path workspaceRoot, final int attempt, final CheckedSupplier, Exception> workerRun, final String airbyteVersionOrWarnings) { - final Path jobRoot = WorkerUtils.getJobRoot(workspaceRoot, String.valueOf(jobId), attempt); + final Path jobRoot = TemporalUtils.getJobRoot(workspaceRoot, String.valueOf(jobId), attempt); return new WorkerRun(jobRoot, workerRun, airbyteVersionOrWarnings); } diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/ConnectionManagerUtils.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/ConnectionManagerUtils.java deleted file mode 100644 index 13b532d0044ca..0000000000000 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/ConnectionManagerUtils.java +++ /dev/null @@ -1,247 +0,0 @@ -/* - * Copyright (c) 2022 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.workers.temporal; - -import io.airbyte.commons.temporal.TemporalJobType; -import io.airbyte.commons.temporal.TemporalWorkflowUtils; -import io.airbyte.commons.temporal.scheduling.ConnectionManagerWorkflow; -import io.airbyte.commons.temporal.scheduling.ConnectionUpdaterInput; -import io.airbyte.commons.temporal.scheduling.state.WorkflowState; -import io.airbyte.workers.temporal.exception.DeletedWorkflowException; -import io.airbyte.workers.temporal.exception.UnreachableWorkflowException; -import io.airbyte.workers.temporal.scheduling.ConnectionManagerWorkflowImpl; -import io.temporal.api.common.v1.WorkflowExecution; -import io.temporal.api.enums.v1.WorkflowExecutionStatus; -import io.temporal.api.workflowservice.v1.DescribeWorkflowExecutionRequest; -import io.temporal.api.workflowservice.v1.DescribeWorkflowExecutionResponse; -import io.temporal.client.BatchRequest; -import io.temporal.client.WorkflowClient; -import io.temporal.workflow.Functions.Proc; -import io.temporal.workflow.Functions.Proc1; -import io.temporal.workflow.Functions.TemporalFunctionalInterfaceMarker; -import jakarta.inject.Singleton; -import java.util.Optional; -import java.util.UUID; -import java.util.function.Function; -import lombok.NoArgsConstructor; -import lombok.extern.slf4j.Slf4j; - -/** - * Encapsulates logic specific to retrieving, starting, and signaling the ConnectionManagerWorkflow. - */ -@NoArgsConstructor -@Singleton -@Slf4j -public class ConnectionManagerUtils { - - /** - * Attempts to send a signal to the existing ConnectionManagerWorkflow for the provided connection. - * - * If the workflow is unreachable, this will restart the workflow and send the signal in a single - * batched request. Batching is used to avoid race conditions between starting the workflow and - * executing the signal. - * - * @param client the WorkflowClient for interacting with temporal - * @param connectionId the connection ID to execute this operation for - * @param signalMethod a function that takes in a connection manager workflow and executes a signal - * method on it, with no arguments - * @return the healthy connection manager workflow that was signaled - * @throws DeletedWorkflowException if the connection manager workflow was deleted - */ - public ConnectionManagerWorkflow signalWorkflowAndRepairIfNecessary(final WorkflowClient client, - final UUID connectionId, - final Function signalMethod) - throws DeletedWorkflowException { - return signalWorkflowAndRepairIfNecessary(client, connectionId, signalMethod, Optional.empty()); - } - - /** - * Attempts to send a signal to the existing ConnectionManagerWorkflow for the provided connection. - * - * If the workflow is unreachable, this will restart the workflow and send the signal in a single - * batched request. Batching is used to avoid race conditions between starting the workflow and - * executing the signal. - * - * @param client the WorkflowClient for interacting with temporal - * @param connectionId the connection ID to execute this operation for - * @param signalMethod a function that takes in a connection manager workflow and executes a signal - * method on it, with 1 argument - * @param signalArgument the single argument to be input to the signal - * @return the healthy connection manager workflow that was signaled - * @throws DeletedWorkflowException if the connection manager workflow was deleted - */ - public ConnectionManagerWorkflow signalWorkflowAndRepairIfNecessary(final WorkflowClient client, - final UUID connectionId, - final Function> signalMethod, - final T signalArgument) - throws DeletedWorkflowException { - return signalWorkflowAndRepairIfNecessary(client, connectionId, signalMethod, Optional.of(signalArgument)); - } - - // This method unifies the logic of the above two, by using the optional signalArgument parameter to - // indicate if an argument is being provided to the signal or not. - // Keeping this private and only exposing the above methods outside this class provides a strict - // type enforcement for external calls, and means this method can assume consistent type - // implementations for both cases. - private ConnectionManagerWorkflow signalWorkflowAndRepairIfNecessary(final WorkflowClient client, - final UUID connectionId, - final Function signalMethod, - final Optional signalArgument) - throws DeletedWorkflowException { - try { - final ConnectionManagerWorkflow connectionManagerWorkflow = getConnectionManagerWorkflow(client, connectionId); - log.info("Retrieved existing connection manager workflow for connection {}. Executing signal.", connectionId); - // retrieve the signal from the lambda - final TemporalFunctionalInterfaceMarker signal = signalMethod.apply(connectionManagerWorkflow); - // execute the signal - if (signalArgument.isPresent()) { - ((Proc1) signal).apply(signalArgument.get()); - } else { - ((Proc) signal).apply(); - } - return connectionManagerWorkflow; - } catch (final UnreachableWorkflowException e) { - log.error( - String.format( - "Failed to retrieve ConnectionManagerWorkflow for connection %s. Repairing state by creating new workflow and starting with the signal.", - connectionId), - e); - - // in case there is an existing workflow in a bad state, attempt to terminate it first before - // starting a new workflow - safeTerminateWorkflow(client, connectionId, "Terminating workflow in unreachable state before starting a new workflow for this connection"); - - final ConnectionManagerWorkflow connectionManagerWorkflow = newConnectionManagerWorkflowStub(client, connectionId); - final ConnectionUpdaterInput startWorkflowInput = TemporalWorkflowUtils.buildStartWorkflowInput(connectionId); - - final BatchRequest batchRequest = client.newSignalWithStartRequest(); - batchRequest.add(connectionManagerWorkflow::run, startWorkflowInput); - - // retrieve the signal from the lambda - final TemporalFunctionalInterfaceMarker signal = signalMethod.apply(connectionManagerWorkflow); - // add signal to batch request - if (signalArgument.isPresent()) { - batchRequest.add((Proc1) signal, signalArgument.get()); - } else { - batchRequest.add((Proc) signal); - } - - client.signalWithStart(batchRequest); - log.info("Connection manager workflow for connection {} has been started and signaled.", connectionId); - - return connectionManagerWorkflow; - } - } - - void safeTerminateWorkflow(final WorkflowClient client, final String workflowId, final String reason) { - log.info("Attempting to terminate existing workflow for workflowId {}.", workflowId); - try { - client.newUntypedWorkflowStub(workflowId).terminate(reason); - } catch (final Exception e) { - log.warn( - "Could not terminate temporal workflow due to the following error; " - + "this may be because there is currently no running workflow for this connection.", - e); - } - } - - public void safeTerminateWorkflow(final WorkflowClient client, final UUID connectionId, final String reason) { - safeTerminateWorkflow(client, getConnectionManagerName(connectionId), reason); - } - - public ConnectionManagerWorkflow startConnectionManagerNoSignal(final WorkflowClient client, final UUID connectionId) { - final ConnectionManagerWorkflow connectionManagerWorkflow = newConnectionManagerWorkflowStub(client, connectionId); - final ConnectionUpdaterInput input = TemporalWorkflowUtils.buildStartWorkflowInput(connectionId); - WorkflowClient.start(connectionManagerWorkflow::run, input); - - return connectionManagerWorkflow; - } - - /** - * Attempts to retrieve the connection manager workflow for the provided connection. - * - * @param connectionId the ID of the connection whose workflow should be retrieved - * @return the healthy ConnectionManagerWorkflow - * @throws DeletedWorkflowException if the workflow was deleted, according to the workflow state - * @throws UnreachableWorkflowException if the workflow is in an unreachable state - */ - public ConnectionManagerWorkflow getConnectionManagerWorkflow(final WorkflowClient client, final UUID connectionId) - throws DeletedWorkflowException, UnreachableWorkflowException { - - final ConnectionManagerWorkflow connectionManagerWorkflow; - final WorkflowState workflowState; - final WorkflowExecutionStatus workflowExecutionStatus; - try { - connectionManagerWorkflow = client.newWorkflowStub(ConnectionManagerWorkflow.class, getConnectionManagerName(connectionId)); - workflowState = connectionManagerWorkflow.getState(); - workflowExecutionStatus = getConnectionManagerWorkflowStatus(client, connectionId); - } catch (final Exception e) { - throw new UnreachableWorkflowException( - String.format("Failed to retrieve ConnectionManagerWorkflow for connection %s due to the following error:", connectionId), - e); - } - - if (WorkflowExecutionStatus.WORKFLOW_EXECUTION_STATUS_COMPLETED.equals(workflowExecutionStatus)) { - if (workflowState.isDeleted()) { - throw new DeletedWorkflowException(String.format( - "The connection manager workflow for connection %s is deleted, so no further operations cannot be performed on it.", - connectionId)); - } - - // A non-deleted workflow being in a COMPLETED state is unexpected, and should be corrected - throw new UnreachableWorkflowException( - String.format("ConnectionManagerWorkflow for connection %s is unreachable due to having COMPLETED status.", connectionId)); - } - - if (workflowState.isQuarantined()) { - throw new UnreachableWorkflowException( - String.format("ConnectionManagerWorkflow for connection %s is unreachable due to being in a quarantined state.", connectionId)); - } - - return connectionManagerWorkflow; - } - - boolean isWorkflowStateRunning(final WorkflowClient client, final UUID connectionId) { - try { - final ConnectionManagerWorkflow connectionManagerWorkflow = client.newWorkflowStub(ConnectionManagerWorkflow.class, - getConnectionManagerName(connectionId)); - return connectionManagerWorkflow.getState().isRunning(); - } catch (final Exception e) { - return false; - } - } - - public WorkflowExecutionStatus getConnectionManagerWorkflowStatus(final WorkflowClient workflowClient, final UUID connectionId) { - final DescribeWorkflowExecutionRequest describeWorkflowExecutionRequest = DescribeWorkflowExecutionRequest.newBuilder() - .setExecution(WorkflowExecution.newBuilder() - .setWorkflowId(getConnectionManagerName(connectionId)) - .build()) - .setNamespace(workflowClient.getOptions().getNamespace()).build(); - - final DescribeWorkflowExecutionResponse describeWorkflowExecutionResponse = workflowClient.getWorkflowServiceStubs().blockingStub() - .describeWorkflowExecution(describeWorkflowExecutionRequest); - - return describeWorkflowExecutionResponse.getWorkflowExecutionInfo().getStatus(); - } - - public long getCurrentJobId(final WorkflowClient client, final UUID connectionId) { - try { - final ConnectionManagerWorkflow connectionManagerWorkflow = getConnectionManagerWorkflow(client, connectionId); - return connectionManagerWorkflow.getJobInformation().getJobId(); - } catch (final Exception e) { - return ConnectionManagerWorkflowImpl.NON_RUNNING_JOB_ID; - } - } - - public ConnectionManagerWorkflow newConnectionManagerWorkflowStub(final WorkflowClient client, final UUID connectionId) { - return client.newWorkflowStub(ConnectionManagerWorkflow.class, - TemporalWorkflowUtils.buildWorkflowOptions(TemporalJobType.CONNECTION_UPDATER, getConnectionManagerName(connectionId))); - } - - public String getConnectionManagerName(final UUID connectionId) { - return "connection_manager_" + connectionId; - } - -} diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/TemporalAttemptExecution.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/TemporalAttemptExecution.java index a15b109bbb3c9..b463b8fc8ce5c 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/TemporalAttemptExecution.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/TemporalAttemptExecution.java @@ -16,7 +16,6 @@ import io.airbyte.config.helpers.LogConfigs; import io.airbyte.persistence.job.models.JobRunConfig; import io.airbyte.workers.Worker; -import io.airbyte.workers.WorkerUtils; import io.temporal.activity.Activity; import io.temporal.activity.ActivityExecutionContext; import java.nio.file.Path; @@ -88,7 +87,7 @@ public TemporalAttemptExecution(final Path workspaceRoot, final String airbyteVersion) { this.jobRunConfig = jobRunConfig; - this.jobRoot = WorkerUtils.getJobRoot(workspaceRoot, jobRunConfig.getJobId(), jobRunConfig.getAttemptId()); + this.jobRoot = TemporalUtils.getJobRoot(workspaceRoot, jobRunConfig.getJobId(), jobRunConfig.getAttemptId()); this.workerSupplier = workerSupplier; this.inputSupplier = inputSupplier; this.mdcSetter = mdcSetter; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/TemporalClient.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/TemporalClient.java deleted file mode 100644 index b85a32f79ba83..0000000000000 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/TemporalClient.java +++ /dev/null @@ -1,549 +0,0 @@ -/* - * Copyright (c) 2022 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.workers.temporal; - -import static io.airbyte.workers.temporal.scheduling.ConnectionManagerWorkflowImpl.NON_RUNNING_JOB_ID; - -import com.google.common.annotations.VisibleForTesting; -import com.google.protobuf.ByteString; -import io.airbyte.commons.temporal.TemporalJobType; -import io.airbyte.commons.temporal.TemporalWorkflowUtils; -import io.airbyte.commons.temporal.scheduling.ConnectionManagerWorkflow; -import io.airbyte.config.ConnectorJobOutput; -import io.airbyte.config.JobCheckConnectionConfig; -import io.airbyte.config.JobDiscoverCatalogConfig; -import io.airbyte.config.JobGetSpecConfig; -import io.airbyte.config.JobSyncConfig; -import io.airbyte.config.StandardCheckConnectionInput; -import io.airbyte.config.StandardDiscoverCatalogInput; -import io.airbyte.config.StandardSyncInput; -import io.airbyte.config.StandardSyncOutput; -import io.airbyte.config.persistence.StreamResetPersistence; -import io.airbyte.persistence.job.models.IntegrationLauncherConfig; -import io.airbyte.persistence.job.models.JobRunConfig; -import io.airbyte.protocol.models.StreamDescriptor; -import io.airbyte.workers.WorkerUtils; -import io.airbyte.workers.config.WorkerMode; -import io.airbyte.workers.temporal.check.connection.CheckConnectionWorkflow; -import io.airbyte.workers.temporal.discover.catalog.DiscoverCatalogWorkflow; -import io.airbyte.workers.temporal.exception.DeletedWorkflowException; -import io.airbyte.workers.temporal.exception.UnreachableWorkflowException; -import io.airbyte.workers.temporal.spec.SpecWorkflow; -import io.airbyte.workers.temporal.sync.SyncWorkflow; -import io.micronaut.context.annotation.Requires; -import io.temporal.api.common.v1.WorkflowType; -import io.temporal.api.enums.v1.WorkflowExecutionStatus; -import io.temporal.api.workflowservice.v1.ListClosedWorkflowExecutionsRequest; -import io.temporal.api.workflowservice.v1.ListClosedWorkflowExecutionsResponse; -import io.temporal.api.workflowservice.v1.ListOpenWorkflowExecutionsRequest; -import io.temporal.api.workflowservice.v1.ListOpenWorkflowExecutionsResponse; -import io.temporal.client.WorkflowClient; -import io.temporal.serviceclient.WorkflowServiceStubs; -import jakarta.inject.Named; -import jakarta.inject.Singleton; -import java.io.IOException; -import java.nio.file.Path; -import java.time.Duration; -import java.time.Instant; -import java.util.HashSet; -import java.util.List; -import java.util.Optional; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.function.Supplier; -import java.util.stream.Collectors; -import lombok.Builder; -import lombok.Value; -import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.StringUtils; -import org.apache.commons.lang3.time.StopWatch; - -@Slf4j -@Singleton -@Requires(env = WorkerMode.CONTROL_PLANE) -public class TemporalClient { - - /** - * This is used to sleep between 2 temporal queries. The query is needed to ensure that the cancel - * and start manual sync methods wait before returning. Since temporal signals are async, we need to - * use the queries to make sure that we are in a state in which we want to continue with. - */ - private static final int DELAY_BETWEEN_QUERY_MS = 10; - - private final Path workspaceRoot; - private final WorkflowClient client; - private final WorkflowServiceStubs service; - private final StreamResetPersistence streamResetPersistence; - private final ConnectionManagerUtils connectionManagerUtils; - private final StreamResetRecordsHelper streamResetRecordsHelper; - - public TemporalClient(@Named("workspaceRoot") final Path workspaceRoot, - final WorkflowClient client, - final WorkflowServiceStubs service, - final StreamResetPersistence streamResetPersistence, - final ConnectionManagerUtils connectionManagerUtils, - final StreamResetRecordsHelper streamResetRecordsHelper) { - this.workspaceRoot = workspaceRoot; - this.client = client; - this.service = service; - this.streamResetPersistence = streamResetPersistence; - this.connectionManagerUtils = connectionManagerUtils; - this.streamResetRecordsHelper = streamResetRecordsHelper; - } - - /** - * Direct termination of Temporal Workflows should generally be avoided. This method exists for some - * rare circumstances where this may be required. Originally added to facilitate Airbyte's migration - * to Temporal Cloud. TODO consider deleting this after Temporal Cloud migration - */ - public void dangerouslyTerminateWorkflow(final String workflowId, final String reason) { - this.client.newUntypedWorkflowStub(workflowId).terminate(reason); - } - - public TemporalResponse submitGetSpec(final UUID jobId, final int attempt, final JobGetSpecConfig config) { - final JobRunConfig jobRunConfig = TemporalWorkflowUtils.createJobRunConfig(jobId, attempt); - - final IntegrationLauncherConfig launcherConfig = new IntegrationLauncherConfig() - .withJobId(jobId.toString()) - .withAttemptId((long) attempt) - .withDockerImage(config.getDockerImage()); - return execute(jobRunConfig, - () -> getWorkflowStub(SpecWorkflow.class, TemporalJobType.GET_SPEC).run(jobRunConfig, launcherConfig)); - - } - - public TemporalResponse submitCheckConnection(final UUID jobId, - final int attempt, - final JobCheckConnectionConfig config) { - final JobRunConfig jobRunConfig = TemporalWorkflowUtils.createJobRunConfig(jobId, attempt); - final IntegrationLauncherConfig launcherConfig = new IntegrationLauncherConfig() - .withJobId(jobId.toString()) - .withAttemptId((long) attempt) - .withDockerImage(config.getDockerImage()) - .withProtocolVersion(config.getProtocolVersion()); - final StandardCheckConnectionInput input = new StandardCheckConnectionInput().withConnectionConfiguration(config.getConnectionConfiguration()); - - return execute(jobRunConfig, - () -> getWorkflowStub(CheckConnectionWorkflow.class, TemporalJobType.CHECK_CONNECTION).run(jobRunConfig, launcherConfig, input)); - } - - public TemporalResponse submitDiscoverSchema(final UUID jobId, - final int attempt, - final JobDiscoverCatalogConfig config) { - final JobRunConfig jobRunConfig = TemporalWorkflowUtils.createJobRunConfig(jobId, attempt); - final IntegrationLauncherConfig launcherConfig = new IntegrationLauncherConfig() - .withJobId(jobId.toString()) - .withAttemptId((long) attempt) - .withDockerImage(config.getDockerImage()); - final StandardDiscoverCatalogInput input = new StandardDiscoverCatalogInput().withConnectionConfiguration(config.getConnectionConfiguration()) - .withSourceId(config.getSourceId()).withConnectorVersion(config.getConnectorVersion()).withConfigHash(config.getConfigHash()); - - return execute(jobRunConfig, - () -> getWorkflowStub(DiscoverCatalogWorkflow.class, TemporalJobType.DISCOVER_SCHEMA).run(jobRunConfig, launcherConfig, input)); - } - - public TemporalResponse submitSync(final long jobId, final int attempt, final JobSyncConfig config, final UUID connectionId) { - final JobRunConfig jobRunConfig = TemporalWorkflowUtils.createJobRunConfig(jobId, attempt); - - final IntegrationLauncherConfig sourceLauncherConfig = new IntegrationLauncherConfig() - .withJobId(String.valueOf(jobId)) - .withAttemptId((long) attempt) - .withDockerImage(config.getSourceDockerImage()); - - final IntegrationLauncherConfig destinationLauncherConfig = new IntegrationLauncherConfig() - .withJobId(String.valueOf(jobId)) - .withAttemptId((long) attempt) - .withDockerImage(config.getDestinationDockerImage()); - - final StandardSyncInput input = new StandardSyncInput() - .withNamespaceDefinition(config.getNamespaceDefinition()) - .withNamespaceFormat(config.getNamespaceFormat()) - .withPrefix(config.getPrefix()) - .withSourceConfiguration(config.getSourceConfiguration()) - .withDestinationConfiguration(config.getDestinationConfiguration()) - .withOperationSequence(config.getOperationSequence()) - .withCatalog(config.getConfiguredAirbyteCatalog()) - .withState(config.getState()) - .withResourceRequirements(config.getResourceRequirements()) - .withSourceResourceRequirements(config.getSourceResourceRequirements()) - .withDestinationResourceRequirements(config.getDestinationResourceRequirements()); - - return execute(jobRunConfig, - () -> getWorkflowStub(SyncWorkflow.class, TemporalJobType.SYNC).run( - jobRunConfig, - sourceLauncherConfig, - destinationLauncherConfig, - input, - connectionId)); - } - - public void migrateSyncIfNeeded(final Set connectionIds) { - final StopWatch globalMigrationWatch = new StopWatch(); - globalMigrationWatch.start(); - refreshRunningWorkflow(); - - connectionIds.forEach((connectionId) -> { - final StopWatch singleSyncMigrationWatch = new StopWatch(); - singleSyncMigrationWatch.start(); - if (!isInRunningWorkflowCache(connectionManagerUtils.getConnectionManagerName(connectionId))) { - log.info("Migrating: " + connectionId); - try { - submitConnectionUpdaterAsync(connectionId); - } catch (final Exception e) { - log.error("New workflow submission failed, retrying", e); - refreshRunningWorkflow(); - submitConnectionUpdaterAsync(connectionId); - } - } - singleSyncMigrationWatch.stop(); - log.info("Sync migration took: " + singleSyncMigrationWatch.formatTime()); - }); - globalMigrationWatch.stop(); - - log.info("The migration to the new scheduler took: " + globalMigrationWatch.formatTime()); - } - - private final Set workflowNames = new HashSet<>(); - - boolean isInRunningWorkflowCache(final String workflowName) { - return workflowNames.contains(workflowName); - } - - @VisibleForTesting - void refreshRunningWorkflow() { - workflowNames.clear(); - ByteString token; - ListOpenWorkflowExecutionsRequest openWorkflowExecutionsRequest = - ListOpenWorkflowExecutionsRequest.newBuilder() - .setNamespace(client.getOptions().getNamespace()) - .build(); - do { - final ListOpenWorkflowExecutionsResponse listOpenWorkflowExecutionsRequest = - service.blockingStub().listOpenWorkflowExecutions(openWorkflowExecutionsRequest); - final Set workflowExecutionInfos = listOpenWorkflowExecutionsRequest.getExecutionsList().stream() - .map((workflowExecutionInfo -> workflowExecutionInfo.getExecution().getWorkflowId())) - .collect(Collectors.toSet()); - workflowNames.addAll(workflowExecutionInfos); - token = listOpenWorkflowExecutionsRequest.getNextPageToken(); - - openWorkflowExecutionsRequest = - ListOpenWorkflowExecutionsRequest.newBuilder() - .setNamespace(client.getOptions().getNamespace()) - .setNextPageToken(token) - .build(); - - } while (token != null && token.size() > 0); - } - - /** - * Refreshes the cache of running workflows, and returns their names. Currently called by the - * Temporal Cloud migrator to generate a list of workflows that should be migrated. After the - * Temporal Migration is complete, this could be removed, though it may be handy for a future use - * case. - */ - public Set getAllRunningWorkflows() { - final var startTime = Instant.now(); - refreshRunningWorkflow(); - final var endTime = Instant.now(); - log.info("getAllRunningWorkflows took {} milliseconds", Duration.between(startTime, endTime).toMillis()); - return workflowNames; - } - - public ConnectionManagerWorkflow submitConnectionUpdaterAsync(final UUID connectionId) { - log.info("Starting the scheduler temporal wf"); - final ConnectionManagerWorkflow connectionManagerWorkflow = - connectionManagerUtils.startConnectionManagerNoSignal(client, connectionId); - try { - CompletableFuture.supplyAsync(() -> { - try { - do { - Thread.sleep(DELAY_BETWEEN_QUERY_MS); - } while (!isWorkflowReachable(connectionId)); - } catch (final InterruptedException e) {} - return null; - }).get(60, TimeUnit.SECONDS); - } catch (final InterruptedException | ExecutionException e) { - log.error("Failed to create a new connection manager workflow", e); - } catch (final TimeoutException e) { - log.error("Can't create a new connection manager workflow due to timeout", e); - } - - return connectionManagerWorkflow; - } - - public void deleteConnection(final UUID connectionId) { - try { - connectionManagerUtils.signalWorkflowAndRepairIfNecessary(client, connectionId, - connectionManagerWorkflow -> connectionManagerWorkflow::deleteConnection); - } catch (final DeletedWorkflowException e) { - log.info("Connection {} has already been deleted.", connectionId); - } - } - - public void update(final UUID connectionId) { - final ConnectionManagerWorkflow connectionManagerWorkflow; - try { - connectionManagerWorkflow = connectionManagerUtils.getConnectionManagerWorkflow(client, connectionId); - } catch (final DeletedWorkflowException e) { - log.info("Connection {} is deleted, and therefore cannot be updated.", connectionId); - return; - } catch (final UnreachableWorkflowException e) { - log.error( - String.format("Failed to retrieve ConnectionManagerWorkflow for connection %s. Repairing state by creating new workflow.", connectionId), - e); - connectionManagerUtils.safeTerminateWorkflow(client, connectionId, - "Terminating workflow in unreachable state before starting a new workflow for this connection"); - submitConnectionUpdaterAsync(connectionId); - return; - } - - connectionManagerWorkflow.connectionUpdated(); - } - - @Value - @Builder - public static class ManualOperationResult { - - final Optional failingReason; - final Optional jobId; - final Optional errorCode; - - } - - public ManualOperationResult startNewManualSync(final UUID connectionId) { - log.info("Manual sync request"); - - if (connectionManagerUtils.isWorkflowStateRunning(client, connectionId)) { - // TODO Bmoric: Error is running - return new ManualOperationResult( - Optional.of("A sync is already running for: " + connectionId), - Optional.empty(), Optional.of(ErrorCode.WORKFLOW_RUNNING)); - } - - try { - connectionManagerUtils.signalWorkflowAndRepairIfNecessary(client, connectionId, workflow -> workflow::submitManualSync); - } catch (final DeletedWorkflowException e) { - log.error("Can't sync a deleted connection.", e); - return new ManualOperationResult( - Optional.of(e.getMessage()), - Optional.empty(), Optional.of(ErrorCode.WORKFLOW_DELETED)); - } - - do { - try { - Thread.sleep(DELAY_BETWEEN_QUERY_MS); - } catch (final InterruptedException e) { - return new ManualOperationResult( - Optional.of("Didn't managed to start a sync for: " + connectionId), - Optional.empty(), Optional.of(ErrorCode.UNKNOWN)); - } - } while (!connectionManagerUtils.isWorkflowStateRunning(client, connectionId)); - - log.info("end of manual schedule"); - - final long jobId = connectionManagerUtils.getCurrentJobId(client, connectionId); - - return new ManualOperationResult( - Optional.empty(), - Optional.of(jobId), Optional.empty()); - } - - public ManualOperationResult startNewCancellation(final UUID connectionId) { - log.info("Manual cancellation request"); - - final long jobId = connectionManagerUtils.getCurrentJobId(client, connectionId); - - try { - connectionManagerUtils.signalWorkflowAndRepairIfNecessary(client, connectionId, workflow -> workflow::cancelJob); - } catch (final DeletedWorkflowException e) { - log.error("Can't cancel a deleted workflow", e); - return new ManualOperationResult( - Optional.of(e.getMessage()), - Optional.empty(), Optional.of(ErrorCode.WORKFLOW_DELETED)); - } - - do { - try { - Thread.sleep(DELAY_BETWEEN_QUERY_MS); - } catch (final InterruptedException e) { - return new ManualOperationResult( - Optional.of("Didn't manage to cancel a sync for: " + connectionId), - Optional.empty(), Optional.of(ErrorCode.UNKNOWN)); - } - } while (connectionManagerUtils.isWorkflowStateRunning(client, connectionId)); - - streamResetRecordsHelper.deleteStreamResetRecordsForJob(jobId, connectionId); - - log.info("end of manual cancellation"); - - return new ManualOperationResult( - Optional.empty(), - Optional.of(jobId), Optional.empty()); - } - - public ManualOperationResult resetConnection(final UUID connectionId, - final List streamsToReset, - final boolean syncImmediatelyAfter) { - log.info("reset sync request"); - - try { - streamResetPersistence.createStreamResets(connectionId, streamsToReset); - } catch (final IOException e) { - log.error("Could not persist streams to reset.", e); - return new ManualOperationResult( - Optional.of(e.getMessage()), - Optional.empty(), Optional.of(ErrorCode.UNKNOWN)); - } - - // get the job ID before the reset, defaulting to NON_RUNNING_JOB_ID if workflow is unreachable - final long oldJobId = connectionManagerUtils.getCurrentJobId(client, connectionId); - - try { - if (syncImmediatelyAfter) { - connectionManagerUtils.signalWorkflowAndRepairIfNecessary(client, connectionId, workflow -> workflow::resetConnectionAndSkipNextScheduling); - } else { - connectionManagerUtils.signalWorkflowAndRepairIfNecessary(client, connectionId, workflow -> workflow::resetConnection); - } - } catch (final DeletedWorkflowException e) { - log.error("Can't reset a deleted workflow", e); - return new ManualOperationResult( - Optional.of(e.getMessage()), - Optional.empty(), Optional.of(ErrorCode.UNKNOWN)); - } - - Optional newJobId; - - do { - try { - Thread.sleep(DELAY_BETWEEN_QUERY_MS); - } catch (final InterruptedException e) { - return new ManualOperationResult( - Optional.of("Didn't manage to reset a sync for: " + connectionId), - Optional.empty(), Optional.of(ErrorCode.UNKNOWN)); - } - newJobId = getNewJobId(connectionId, oldJobId); - } while (newJobId.isEmpty()); - - log.info("end of reset submission"); - - return new ManualOperationResult( - Optional.empty(), - newJobId, Optional.empty()); - } - - private Optional getNewJobId(final UUID connectionId, final long oldJobId) { - final long currentJobId = connectionManagerUtils.getCurrentJobId(client, connectionId); - if (currentJobId == NON_RUNNING_JOB_ID || currentJobId == oldJobId) { - return Optional.empty(); - } else { - return Optional.of(currentJobId); - } - } - - /** - * This should be in the class {@li} - * - * @param workflowId - * @return - */ - Optional extractConnectionIdFromWorkflowId(final String workflowId) { - if (!workflowId.startsWith("connection_manager_")) { - return Optional.empty(); - } - return Optional.ofNullable(StringUtils.removeStart(workflowId, "connection_manager_")) - .map( - stringUUID -> UUID.fromString(stringUUID)); - } - - Set fetchClosedWorkflowsByStatus(final WorkflowExecutionStatus executionStatus) { - ByteString token; - ListClosedWorkflowExecutionsRequest workflowExecutionsRequest = - ListClosedWorkflowExecutionsRequest.newBuilder() - .setNamespace(client.getOptions().getNamespace()) - .build(); - - final Set workflowExecutionInfos = new HashSet<>(); - do { - final ListClosedWorkflowExecutionsResponse listOpenWorkflowExecutionsRequest = - service.blockingStub().listClosedWorkflowExecutions(workflowExecutionsRequest); - final WorkflowType connectionManagerWorkflowType = WorkflowType.newBuilder().setName(ConnectionManagerWorkflow.class.getSimpleName()).build(); - workflowExecutionInfos.addAll(listOpenWorkflowExecutionsRequest.getExecutionsList().stream() - .filter(workflowExecutionInfo -> workflowExecutionInfo.getType() == connectionManagerWorkflowType || - workflowExecutionInfo.getStatus() == executionStatus) - .flatMap((workflowExecutionInfo -> extractConnectionIdFromWorkflowId(workflowExecutionInfo.getExecution().getWorkflowId()).stream())) - .collect(Collectors.toSet())); - token = listOpenWorkflowExecutionsRequest.getNextPageToken(); - - workflowExecutionsRequest = - ListClosedWorkflowExecutionsRequest.newBuilder() - .setNamespace(client.getOptions().getNamespace()) - .setNextPageToken(token) - .build(); - - } while (token != null && token.size() > 0); - - return workflowExecutionInfos; - } - - @VisibleForTesting - Set filterOutRunningWorkspaceId(final Set workflowIds) { - refreshRunningWorkflow(); - - final Set runningWorkflowByUUID = - workflowNames.stream().flatMap(name -> extractConnectionIdFromWorkflowId(name).stream()).collect(Collectors.toSet()); - - return workflowIds.stream().filter(workflowId -> !runningWorkflowByUUID.contains(workflowId)).collect(Collectors.toSet()); - } - - private T getWorkflowStub(final Class workflowClass, final TemporalJobType jobType) { - return client.newWorkflowStub(workflowClass, TemporalWorkflowUtils.buildWorkflowOptions(jobType)); - } - - private boolean getConnectorJobSucceeded(final ConnectorJobOutput output) { - return output.getFailureReason() == null; - } - - @VisibleForTesting - TemporalResponse execute(final JobRunConfig jobRunConfig, final Supplier executor) { - final Path jobRoot = WorkerUtils.getJobRoot(workspaceRoot, jobRunConfig); - final Path logPath = WorkerUtils.getLogPath(jobRoot); - - T operationOutput = null; - RuntimeException exception = null; - - try { - operationOutput = executor.get(); - } catch (final RuntimeException e) { - exception = e; - } - - boolean succeeded = exception == null; - if (succeeded && operationOutput instanceof ConnectorJobOutput) { - succeeded = getConnectorJobSucceeded((ConnectorJobOutput) operationOutput); - } - - final JobMetadata metadata = new JobMetadata(succeeded, logPath); - return new TemporalResponse<>(operationOutput, metadata); - } - - /** - * Check if a workflow is reachable for signal calls by attempting to query for current state. If - * the query succeeds, and the workflow is not marked as deleted, the workflow is reachable. - */ - @VisibleForTesting - boolean isWorkflowReachable(final UUID connectionId) { - try { - connectionManagerUtils.getConnectionManagerWorkflow(client, connectionId); - return true; - } catch (final Exception e) { - return false; - } - } - -} diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/check/connection/CheckConnectionActivityImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/check/connection/CheckConnectionActivityImpl.java index 8448b1014aef8..2ed068e633abe 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/check/connection/CheckConnectionActivityImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/check/connection/CheckConnectionActivityImpl.java @@ -10,6 +10,7 @@ import io.airbyte.commons.protocol.AirbyteMessageSerDeProvider; import io.airbyte.commons.protocol.AirbyteMessageVersionedMigratorFactory; import io.airbyte.commons.temporal.CancellationHandler; +import io.airbyte.commons.temporal.config.WorkerMode; import io.airbyte.config.Configs.WorkerEnvironment; import io.airbyte.config.ConnectorJobOutput; import io.airbyte.config.StandardCheckConnectionInput; @@ -20,7 +21,6 @@ import io.airbyte.persistence.job.models.IntegrationLauncherConfig; import io.airbyte.workers.Worker; import io.airbyte.workers.WorkerConfigs; -import io.airbyte.workers.config.WorkerMode; import io.airbyte.workers.general.DefaultCheckConnectionWorker; import io.airbyte.workers.internal.AirbyteStreamFactory; import io.airbyte.workers.internal.DefaultAirbyteStreamFactory; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/check/connection/CheckConnectionWorkflowImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/check/connection/CheckConnectionWorkflowImpl.java index 070a5b01758ec..e8706c91a89bd 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/check/connection/CheckConnectionWorkflowImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/check/connection/CheckConnectionWorkflowImpl.java @@ -4,6 +4,7 @@ package io.airbyte.workers.temporal.check.connection; +import io.airbyte.commons.temporal.scheduling.CheckConnectionWorkflow; import io.airbyte.config.ConnectorJobOutput; import io.airbyte.config.ConnectorJobOutput.OutputType; import io.airbyte.config.StandardCheckConnectionInput; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/discover/catalog/DiscoverCatalogActivityImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/discover/catalog/DiscoverCatalogActivityImpl.java index 76a1f6235b6c9..f631de372f4b3 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/discover/catalog/DiscoverCatalogActivityImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/discover/catalog/DiscoverCatalogActivityImpl.java @@ -8,6 +8,7 @@ import io.airbyte.api.client.AirbyteApiClient; import io.airbyte.commons.functional.CheckedSupplier; import io.airbyte.commons.temporal.CancellationHandler; +import io.airbyte.commons.temporal.config.WorkerMode; import io.airbyte.config.Configs.WorkerEnvironment; import io.airbyte.config.ConnectorJobOutput; import io.airbyte.config.StandardDiscoverCatalogInput; @@ -18,7 +19,6 @@ import io.airbyte.persistence.job.models.JobRunConfig; import io.airbyte.workers.Worker; import io.airbyte.workers.WorkerConfigs; -import io.airbyte.workers.config.WorkerMode; import io.airbyte.workers.general.DefaultDiscoverCatalogWorker; import io.airbyte.workers.internal.AirbyteStreamFactory; import io.airbyte.workers.internal.DefaultAirbyteStreamFactory; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/discover/catalog/DiscoverCatalogWorkflowImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/discover/catalog/DiscoverCatalogWorkflowImpl.java index 273acfb210c21..b216dbc8f0263 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/discover/catalog/DiscoverCatalogWorkflowImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/discover/catalog/DiscoverCatalogWorkflowImpl.java @@ -4,6 +4,7 @@ package io.airbyte.workers.temporal.discover.catalog; +import io.airbyte.commons.temporal.scheduling.DiscoverCatalogWorkflow; import io.airbyte.config.ConnectorJobOutput; import io.airbyte.config.StandardDiscoverCatalogInput; import io.airbyte.persistence.job.models.IntegrationLauncherConfig; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/ConnectionManagerWorkflowImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/ConnectionManagerWorkflowImpl.java index d35f6c515ed4c..4b3a2ca80f1ee 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/ConnectionManagerWorkflowImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/ConnectionManagerWorkflowImpl.java @@ -7,8 +7,10 @@ import com.fasterxml.jackson.databind.JsonNode; import io.airbyte.commons.temporal.TemporalJobType; import io.airbyte.commons.temporal.TemporalWorkflowUtils; +import io.airbyte.commons.temporal.exception.RetryableException; import io.airbyte.commons.temporal.scheduling.ConnectionManagerWorkflow; import io.airbyte.commons.temporal.scheduling.ConnectionUpdaterInput; +import io.airbyte.commons.temporal.scheduling.SyncWorkflow; import io.airbyte.commons.temporal.scheduling.state.WorkflowInternalState; import io.airbyte.commons.temporal.scheduling.state.WorkflowState; import io.airbyte.commons.temporal.scheduling.state.listener.NoopStateListener; @@ -31,7 +33,6 @@ import io.airbyte.workers.temporal.annotations.TemporalActivityStub; import io.airbyte.workers.temporal.check.connection.CheckConnectionActivity; import io.airbyte.workers.temporal.check.connection.CheckConnectionActivity.CheckConnectionInput; -import io.airbyte.workers.temporal.exception.RetryableException; import io.airbyte.workers.temporal.scheduling.activities.AutoDisableConnectionActivity; import io.airbyte.workers.temporal.scheduling.activities.AutoDisableConnectionActivity.AutoDisableConnectionActivityInput; import io.airbyte.workers.temporal.scheduling.activities.AutoDisableConnectionActivity.AutoDisableConnectionOutput; @@ -68,7 +69,6 @@ import io.airbyte.workers.temporal.scheduling.activities.StreamResetActivity; import io.airbyte.workers.temporal.scheduling.activities.StreamResetActivity.DeleteStreamResetRecordsForJobInput; import io.airbyte.workers.temporal.scheduling.activities.WorkflowConfigActivity; -import io.airbyte.workers.temporal.sync.SyncWorkflow; import io.temporal.api.enums.v1.ParentClosePolicy; import io.temporal.failure.ActivityFailure; import io.temporal.failure.CanceledFailure; @@ -89,9 +89,6 @@ @SuppressWarnings("PMD.AvoidDuplicateLiterals") public class ConnectionManagerWorkflowImpl implements ConnectionManagerWorkflow { - public static final long NON_RUNNING_JOB_ID = -1; - public static final int NON_RUNNING_ATTEMPT_ID = -1; - private static final int TASK_QUEUE_CHANGE_CURRENT_VERSION = 1; private static final int AUTO_DISABLE_FAILING_CONNECTION_CHANGE_CURRENT_VERSION = 1; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/AutoDisableConnectionActivityImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/AutoDisableConnectionActivityImpl.java index 67809505267f9..ca361d079705f 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/AutoDisableConnectionActivityImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/AutoDisableConnectionActivityImpl.java @@ -10,6 +10,8 @@ import static java.time.temporal.ChronoUnit.DAYS; import io.airbyte.commons.features.FeatureFlags; +import io.airbyte.commons.temporal.config.WorkerMode; +import io.airbyte.commons.temporal.exception.RetryableException; import io.airbyte.config.StandardSync; import io.airbyte.config.StandardSync.Status; import io.airbyte.config.persistence.ConfigRepository; @@ -19,8 +21,6 @@ import io.airbyte.persistence.job.models.JobStatus; import io.airbyte.persistence.job.models.JobWithStatusAndTimestamp; import io.airbyte.validation.json.JsonValidationException; -import io.airbyte.workers.config.WorkerMode; -import io.airbyte.workers.temporal.exception.RetryableException; import io.micronaut.context.annotation.Requires; import io.micronaut.context.annotation.Value; import jakarta.inject.Singleton; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/ConfigFetchActivityImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/ConfigFetchActivityImpl.java index c6909a4c28753..88776de0bac08 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/ConfigFetchActivityImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/ConfigFetchActivityImpl.java @@ -4,6 +4,8 @@ package io.airbyte.workers.temporal.scheduling.activities; +import io.airbyte.commons.temporal.config.WorkerMode; +import io.airbyte.commons.temporal.exception.RetryableException; import io.airbyte.config.Cron; import io.airbyte.config.StandardSync; import io.airbyte.config.StandardSync.ScheduleType; @@ -14,8 +16,6 @@ import io.airbyte.persistence.job.JobPersistence; import io.airbyte.persistence.job.models.Job; import io.airbyte.validation.json.JsonValidationException; -import io.airbyte.workers.config.WorkerMode; -import io.airbyte.workers.temporal.exception.RetryableException; import io.micronaut.context.annotation.Requires; import io.micronaut.context.annotation.Value; import jakarta.inject.Named; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/ConnectionDeletionActivityImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/ConnectionDeletionActivityImpl.java index 5b138443b74d0..9bde493d8a2cc 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/ConnectionDeletionActivityImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/ConnectionDeletionActivityImpl.java @@ -4,11 +4,11 @@ package io.airbyte.workers.temporal.scheduling.activities; +import io.airbyte.commons.temporal.config.WorkerMode; +import io.airbyte.commons.temporal.exception.RetryableException; import io.airbyte.config.persistence.ConfigNotFoundException; import io.airbyte.validation.json.JsonValidationException; -import io.airbyte.workers.config.WorkerMode; import io.airbyte.workers.helper.ConnectionHelper; -import io.airbyte.workers.temporal.exception.RetryableException; import io.micronaut.context.annotation.Requires; import jakarta.inject.Singleton; import java.io.IOException; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/GenerateInputActivityImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/GenerateInputActivityImpl.java index 503ba9bc8e023..74a622cf27497 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/GenerateInputActivityImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/GenerateInputActivityImpl.java @@ -6,6 +6,8 @@ import io.airbyte.commons.json.Jsons; import io.airbyte.commons.temporal.TemporalWorkflowUtils; +import io.airbyte.commons.temporal.config.WorkerMode; +import io.airbyte.commons.temporal.exception.RetryableException; import io.airbyte.config.JobConfig.ConfigType; import io.airbyte.config.JobResetConnectionConfig; import io.airbyte.config.JobSyncConfig; @@ -16,8 +18,6 @@ import io.airbyte.persistence.job.models.Job; import io.airbyte.persistence.job.models.JobRunConfig; import io.airbyte.workers.WorkerConstants; -import io.airbyte.workers.config.WorkerMode; -import io.airbyte.workers.temporal.exception.RetryableException; import io.micronaut.context.annotation.Requires; import jakarta.inject.Singleton; import java.util.List; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/JobCreationAndStatusUpdateActivity.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/JobCreationAndStatusUpdateActivity.java index 6393d2643ff89..6d394b2dc5e5c 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/JobCreationAndStatusUpdateActivity.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/JobCreationAndStatusUpdateActivity.java @@ -4,9 +4,9 @@ package io.airbyte.workers.temporal.scheduling.activities; +import io.airbyte.commons.temporal.exception.RetryableException; import io.airbyte.config.AttemptFailureSummary; import io.airbyte.config.StandardSyncOutput; -import io.airbyte.workers.temporal.exception.RetryableException; import io.temporal.activity.ActivityInterface; import io.temporal.activity.ActivityMethod; import java.util.UUID; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/JobCreationAndStatusUpdateActivityImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/JobCreationAndStatusUpdateActivityImpl.java index 354b563638677..66533409192fd 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/JobCreationAndStatusUpdateActivityImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/JobCreationAndStatusUpdateActivityImpl.java @@ -7,6 +7,8 @@ import com.google.common.collect.Lists; import io.airbyte.commons.docker.DockerUtils; import io.airbyte.commons.enums.Enums; +import io.airbyte.commons.temporal.config.WorkerMode; +import io.airbyte.commons.temporal.exception.RetryableException; import io.airbyte.config.AttemptFailureSummary; import io.airbyte.config.Configs.WorkerEnvironment; import io.airbyte.config.DestinationConnection; @@ -41,11 +43,9 @@ import io.airbyte.protocol.models.StreamDescriptor; import io.airbyte.validation.json.JsonValidationException; import io.airbyte.workers.JobStatus; -import io.airbyte.workers.config.WorkerMode; import io.airbyte.workers.helper.FailureHelper; import io.airbyte.workers.run.TemporalWorkerRunFactory; import io.airbyte.workers.run.WorkerRun; -import io.airbyte.workers.temporal.exception.RetryableException; import io.micronaut.context.annotation.Requires; import jakarta.inject.Singleton; import java.io.IOException; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/RecordMetricActivityImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/RecordMetricActivityImpl.java index 870a96b0e9381..d65368b1877b9 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/RecordMetricActivityImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/RecordMetricActivityImpl.java @@ -4,11 +4,11 @@ package io.airbyte.workers.temporal.scheduling.activities; +import io.airbyte.commons.temporal.config.WorkerMode; import io.airbyte.commons.temporal.scheduling.ConnectionUpdaterInput; import io.airbyte.metrics.lib.MetricAttribute; import io.airbyte.metrics.lib.MetricClient; import io.airbyte.metrics.lib.MetricTags; -import io.airbyte.workers.config.WorkerMode; import io.micronaut.context.annotation.Requires; import jakarta.inject.Singleton; import java.util.ArrayList; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/StreamResetActivityImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/StreamResetActivityImpl.java index e4b2014dce23e..1499fa160467f 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/StreamResetActivityImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/StreamResetActivityImpl.java @@ -4,8 +4,8 @@ package io.airbyte.workers.temporal.scheduling.activities; -import io.airbyte.workers.config.WorkerMode; -import io.airbyte.workers.temporal.StreamResetRecordsHelper; +import io.airbyte.commons.temporal.StreamResetRecordsHelper; +import io.airbyte.commons.temporal.config.WorkerMode; import io.micronaut.context.annotation.Requires; import jakarta.inject.Singleton; import lombok.extern.slf4j.Slf4j; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/WorkflowConfigActivityImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/WorkflowConfigActivityImpl.java index 558868f04a4b1..19d8fa1135ee3 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/WorkflowConfigActivityImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/scheduling/activities/WorkflowConfigActivityImpl.java @@ -4,7 +4,7 @@ package io.airbyte.workers.temporal.scheduling.activities; -import io.airbyte.workers.config.WorkerMode; +import io.airbyte.commons.temporal.config.WorkerMode; import io.micronaut.context.annotation.Property; import io.micronaut.context.annotation.Requires; import jakarta.inject.Singleton; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/spec/SpecActivityImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/spec/SpecActivityImpl.java index 5cdb899fdfbac..3ad439a4c9bbd 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/spec/SpecActivityImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/spec/SpecActivityImpl.java @@ -7,6 +7,7 @@ import io.airbyte.api.client.AirbyteApiClient; import io.airbyte.commons.functional.CheckedSupplier; import io.airbyte.commons.temporal.CancellationHandler; +import io.airbyte.commons.temporal.config.WorkerMode; import io.airbyte.config.Configs.WorkerEnvironment; import io.airbyte.config.ConnectorJobOutput; import io.airbyte.config.JobGetSpecConfig; @@ -15,7 +16,6 @@ import io.airbyte.persistence.job.models.JobRunConfig; import io.airbyte.workers.Worker; import io.airbyte.workers.WorkerConfigs; -import io.airbyte.workers.config.WorkerMode; import io.airbyte.workers.general.DefaultGetSpecWorker; import io.airbyte.workers.process.AirbyteIntegrationLauncher; import io.airbyte.workers.process.IntegrationLauncher; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/spec/SpecWorkflowImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/spec/SpecWorkflowImpl.java index 5766301fe8169..24ddfe96951ce 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/spec/SpecWorkflowImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/spec/SpecWorkflowImpl.java @@ -4,6 +4,7 @@ package io.airbyte.workers.temporal.spec; +import io.airbyte.commons.temporal.scheduling.SpecWorkflow; import io.airbyte.config.ConnectorJobOutput; import io.airbyte.persistence.job.models.IntegrationLauncherConfig; import io.airbyte.persistence.job.models.JobRunConfig; diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/SyncWorkflowImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/SyncWorkflowImpl.java index 5aa7939948673..99d875968cf7c 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/SyncWorkflowImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/SyncWorkflowImpl.java @@ -4,6 +4,7 @@ package io.airbyte.workers.temporal.sync; +import io.airbyte.commons.temporal.scheduling.SyncWorkflow; import io.airbyte.config.NormalizationInput; import io.airbyte.config.NormalizationSummary; import io.airbyte.config.OperatorDbtInput; diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/run/TemporalWorkerRunFactoryTest.java b/airbyte-workers/src/test/java/io/airbyte/workers/run/TemporalWorkerRunFactoryTest.java index f199b60ac48f3..b522070da341a 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/run/TemporalWorkerRunFactoryTest.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/run/TemporalWorkerRunFactoryTest.java @@ -14,6 +14,8 @@ import com.google.common.collect.ImmutableMap; import io.airbyte.commons.features.FeatureFlags; import io.airbyte.commons.json.Jsons; +import io.airbyte.commons.temporal.TemporalClient; +import io.airbyte.commons.temporal.TemporalResponse; import io.airbyte.config.JobConfig.ConfigType; import io.airbyte.config.JobResetConnectionConfig; import io.airbyte.config.JobSyncConfig; @@ -22,8 +24,6 @@ import io.airbyte.persistence.job.models.Job; import io.airbyte.protocol.models.ConfiguredAirbyteCatalog; import io.airbyte.workers.WorkerConstants; -import io.airbyte.workers.temporal.TemporalClient; -import io.airbyte.workers.temporal.TemporalResponse; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/StreamResetRecordsHelperTest.java b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/StreamResetRecordsHelperTest.java index 7c839b2b0c4f5..fb64e3ecf9284 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/StreamResetRecordsHelperTest.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/StreamResetRecordsHelperTest.java @@ -9,6 +9,7 @@ import static org.mockito.Mockito.never; import static org.mockito.Mockito.when; +import io.airbyte.commons.temporal.StreamResetRecordsHelper; import io.airbyte.config.JobConfig.ConfigType; import io.airbyte.config.persistence.StreamResetPersistence; import io.airbyte.persistence.job.JobPersistence; diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/TemporalClientTest.java b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/TemporalClientTest.java deleted file mode 100644 index 59b1caa9ee74d..0000000000000 --- a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/TemporalClientTest.java +++ /dev/null @@ -1,839 +0,0 @@ -/* - * Copyright (c) 2022 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.workers.temporal; - -import static io.airbyte.workers.temporal.scheduling.ConnectionManagerWorkflowImpl.NON_RUNNING_JOB_ID; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.Mockito.doNothing; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoMoreInteractions; -import static org.mockito.Mockito.when; - -import com.google.common.collect.Sets; -import io.airbyte.commons.json.Jsons; -import io.airbyte.commons.temporal.TemporalJobType; -import io.airbyte.commons.temporal.TemporalWorkflowUtils; -import io.airbyte.commons.temporal.scheduling.ConnectionManagerWorkflow; -import io.airbyte.commons.temporal.scheduling.ConnectionManagerWorkflow.JobInformation; -import io.airbyte.commons.temporal.scheduling.state.WorkflowState; -import io.airbyte.config.ConnectorJobOutput; -import io.airbyte.config.FailureReason; -import io.airbyte.config.JobCheckConnectionConfig; -import io.airbyte.config.JobDiscoverCatalogConfig; -import io.airbyte.config.JobGetSpecConfig; -import io.airbyte.config.JobSyncConfig; -import io.airbyte.config.StandardCheckConnectionInput; -import io.airbyte.config.StandardDiscoverCatalogInput; -import io.airbyte.config.StandardSyncInput; -import io.airbyte.config.helpers.LogClientSingleton; -import io.airbyte.config.persistence.StreamResetPersistence; -import io.airbyte.persistence.job.models.IntegrationLauncherConfig; -import io.airbyte.persistence.job.models.JobRunConfig; -import io.airbyte.protocol.models.ConfiguredAirbyteCatalog; -import io.airbyte.protocol.models.StreamDescriptor; -import io.airbyte.workers.temporal.TemporalClient.ManualOperationResult; -import io.airbyte.workers.temporal.check.connection.CheckConnectionWorkflow; -import io.airbyte.workers.temporal.discover.catalog.DiscoverCatalogWorkflow; -import io.airbyte.workers.temporal.spec.SpecWorkflow; -import io.airbyte.workers.temporal.sync.SyncWorkflow; -import io.temporal.api.enums.v1.WorkflowExecutionStatus; -import io.temporal.api.workflow.v1.WorkflowExecutionInfo; -import io.temporal.api.workflowservice.v1.DescribeWorkflowExecutionResponse; -import io.temporal.api.workflowservice.v1.WorkflowServiceGrpc.WorkflowServiceBlockingStub; -import io.temporal.client.BatchRequest; -import io.temporal.client.WorkflowClient; -import io.temporal.client.WorkflowClientOptions; -import io.temporal.client.WorkflowOptions; -import io.temporal.client.WorkflowStub; -import io.temporal.serviceclient.WorkflowServiceStubs; -import io.temporal.workflow.Functions.Proc; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.List; -import java.util.UUID; -import java.util.function.Supplier; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.DisplayName; -import org.junit.jupiter.api.Nested; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; - -class TemporalClientTest { - - private static final UUID CONNECTION_ID = UUID.randomUUID(); - private static final UUID JOB_UUID = UUID.randomUUID(); - private static final long JOB_ID = 11L; - private static final int ATTEMPT_ID = 21; - private static final JobRunConfig JOB_RUN_CONFIG = new JobRunConfig() - .withJobId(String.valueOf(JOB_ID)) - .withAttemptId((long) ATTEMPT_ID); - private static final String IMAGE_NAME1 = "hms invincible"; - private static final String IMAGE_NAME2 = "hms defiant"; - private static final IntegrationLauncherConfig UUID_LAUNCHER_CONFIG = new IntegrationLauncherConfig() - .withJobId(String.valueOf(JOB_UUID)) - .withAttemptId((long) ATTEMPT_ID) - .withDockerImage(IMAGE_NAME1); - private static final IntegrationLauncherConfig LAUNCHER_CONFIG = new IntegrationLauncherConfig() - .withJobId(String.valueOf(JOB_ID)) - .withAttemptId((long) ATTEMPT_ID) - .withDockerImage(IMAGE_NAME1); - private static final String NAMESPACE = "namespace"; - private static final StreamDescriptor STREAM_DESCRIPTOR = new StreamDescriptor().withName("name"); - private static final String UNCHECKED = "unchecked"; - private static final String EXCEPTION_MESSAGE = "Force state exception to simulate workflow not running"; - - private WorkflowClient workflowClient; - private TemporalClient temporalClient; - private Path logPath; - private WorkflowServiceStubs workflowServiceStubs; - private WorkflowServiceBlockingStub workflowServiceBlockingStub; - private StreamResetPersistence streamResetPersistence; - private ConnectionManagerUtils connectionManagerUtils; - private StreamResetRecordsHelper streamResetRecordsHelper; - - @BeforeEach - void setup() throws IOException { - final Path workspaceRoot = Files.createTempDirectory(Path.of("/tmp"), "temporal_client_test"); - logPath = workspaceRoot.resolve(String.valueOf(JOB_ID)).resolve(String.valueOf(ATTEMPT_ID)).resolve(LogClientSingleton.LOG_FILENAME); - workflowClient = mock(WorkflowClient.class); - when(workflowClient.getOptions()).thenReturn(WorkflowClientOptions.newBuilder().setNamespace(NAMESPACE).build()); - workflowServiceStubs = mock(WorkflowServiceStubs.class); - when(workflowClient.getWorkflowServiceStubs()).thenReturn(workflowServiceStubs); - workflowServiceBlockingStub = mock(WorkflowServiceBlockingStub.class); - when(workflowServiceStubs.blockingStub()).thenReturn(workflowServiceBlockingStub); - streamResetPersistence = mock(StreamResetPersistence.class); - mockWorkflowStatus(WorkflowExecutionStatus.WORKFLOW_EXECUTION_STATUS_RUNNING); - connectionManagerUtils = spy(new ConnectionManagerUtils()); - streamResetRecordsHelper = mock(StreamResetRecordsHelper.class); - temporalClient = - spy(new TemporalClient(workspaceRoot, workflowClient, workflowServiceStubs, streamResetPersistence, connectionManagerUtils, - streamResetRecordsHelper)); - } - - @Nested - @DisplayName("Test execute method.") - class ExecuteJob { - - @SuppressWarnings(UNCHECKED) - @Test - void testExecute() { - final Supplier supplier = mock(Supplier.class); - when(supplier.get()).thenReturn("hello"); - - final TemporalResponse response = temporalClient.execute(JOB_RUN_CONFIG, supplier); - - assertNotNull(response); - assertTrue(response.getOutput().isPresent()); - assertEquals("hello", response.getOutput().get()); - assertTrue(response.getMetadata().isSucceeded()); - assertEquals(logPath, response.getMetadata().getLogPath()); - } - - @SuppressWarnings(UNCHECKED) - @Test - void testExecuteWithException() { - final Supplier supplier = mock(Supplier.class); - when(supplier.get()).thenThrow(IllegalStateException.class); - - final TemporalResponse response = temporalClient.execute(JOB_RUN_CONFIG, supplier); - - assertNotNull(response); - assertFalse(response.getOutput().isPresent()); - assertFalse(response.getMetadata().isSucceeded()); - assertEquals(logPath, response.getMetadata().getLogPath()); - } - - @Test - void testExecuteWithConnectorJobFailure() { - final Supplier supplier = mock(Supplier.class); - final FailureReason mockFailureReason = mock(FailureReason.class); - final ConnectorJobOutput connectorJobOutput = new ConnectorJobOutput() - .withFailureReason(mockFailureReason); - when(supplier.get()).thenReturn(connectorJobOutput); - - final TemporalResponse response = temporalClient.execute(JOB_RUN_CONFIG, supplier); - - assertNotNull(response); - assertTrue(response.getOutput().isPresent()); - assertEquals(connectorJobOutput, response.getOutput().get()); - assertFalse(response.getMetadata().isSucceeded()); - assertEquals(logPath, response.getMetadata().getLogPath()); - } - - } - - @Nested - @DisplayName("Test job creation for each configuration type.") - class TestJobSubmission { - - @Test - void testSubmitGetSpec() { - final SpecWorkflow specWorkflow = mock(SpecWorkflow.class); - when(workflowClient.newWorkflowStub(SpecWorkflow.class, TemporalWorkflowUtils.buildWorkflowOptions(TemporalJobType.GET_SPEC))) - .thenReturn(specWorkflow); - final JobGetSpecConfig getSpecConfig = new JobGetSpecConfig().withDockerImage(IMAGE_NAME1); - - temporalClient.submitGetSpec(JOB_UUID, ATTEMPT_ID, getSpecConfig); - specWorkflow.run(JOB_RUN_CONFIG, UUID_LAUNCHER_CONFIG); - verify(workflowClient).newWorkflowStub(SpecWorkflow.class, TemporalWorkflowUtils.buildWorkflowOptions(TemporalJobType.GET_SPEC)); - } - - @Test - void testSubmitCheckConnection() { - final CheckConnectionWorkflow checkConnectionWorkflow = mock(CheckConnectionWorkflow.class); - when( - workflowClient.newWorkflowStub(CheckConnectionWorkflow.class, TemporalWorkflowUtils.buildWorkflowOptions(TemporalJobType.CHECK_CONNECTION))) - .thenReturn(checkConnectionWorkflow); - final JobCheckConnectionConfig checkConnectionConfig = new JobCheckConnectionConfig() - .withDockerImage(IMAGE_NAME1) - .withConnectionConfiguration(Jsons.emptyObject()); - final StandardCheckConnectionInput input = new StandardCheckConnectionInput() - .withConnectionConfiguration(checkConnectionConfig.getConnectionConfiguration()); - - temporalClient.submitCheckConnection(JOB_UUID, ATTEMPT_ID, checkConnectionConfig); - checkConnectionWorkflow.run(JOB_RUN_CONFIG, UUID_LAUNCHER_CONFIG, input); - verify(workflowClient).newWorkflowStub(CheckConnectionWorkflow.class, - TemporalWorkflowUtils.buildWorkflowOptions(TemporalJobType.CHECK_CONNECTION)); - } - - @Test - void testSubmitDiscoverSchema() { - final DiscoverCatalogWorkflow discoverCatalogWorkflow = mock(DiscoverCatalogWorkflow.class); - when(workflowClient.newWorkflowStub(DiscoverCatalogWorkflow.class, TemporalWorkflowUtils.buildWorkflowOptions(TemporalJobType.DISCOVER_SCHEMA))) - .thenReturn(discoverCatalogWorkflow); - final JobDiscoverCatalogConfig checkConnectionConfig = new JobDiscoverCatalogConfig() - .withDockerImage(IMAGE_NAME1) - .withConnectionConfiguration(Jsons.emptyObject()); - final StandardDiscoverCatalogInput input = new StandardDiscoverCatalogInput() - .withConnectionConfiguration(checkConnectionConfig.getConnectionConfiguration()); - - temporalClient.submitDiscoverSchema(JOB_UUID, ATTEMPT_ID, checkConnectionConfig); - discoverCatalogWorkflow.run(JOB_RUN_CONFIG, UUID_LAUNCHER_CONFIG, input); - verify(workflowClient).newWorkflowStub(DiscoverCatalogWorkflow.class, - TemporalWorkflowUtils.buildWorkflowOptions(TemporalJobType.DISCOVER_SCHEMA)); - } - - @Test - void testSubmitSync() { - final SyncWorkflow discoverCatalogWorkflow = mock(SyncWorkflow.class); - when(workflowClient.newWorkflowStub(SyncWorkflow.class, TemporalWorkflowUtils.buildWorkflowOptions(TemporalJobType.SYNC))) - .thenReturn(discoverCatalogWorkflow); - final JobSyncConfig syncConfig = new JobSyncConfig() - .withSourceDockerImage(IMAGE_NAME1) - .withSourceDockerImage(IMAGE_NAME2) - .withSourceConfiguration(Jsons.emptyObject()) - .withDestinationConfiguration(Jsons.emptyObject()) - .withOperationSequence(List.of()) - .withConfiguredAirbyteCatalog(new ConfiguredAirbyteCatalog()); - final StandardSyncInput input = new StandardSyncInput() - .withNamespaceDefinition(syncConfig.getNamespaceDefinition()) - .withNamespaceFormat(syncConfig.getNamespaceFormat()) - .withPrefix(syncConfig.getPrefix()) - .withSourceConfiguration(syncConfig.getSourceConfiguration()) - .withDestinationConfiguration(syncConfig.getDestinationConfiguration()) - .withOperationSequence(syncConfig.getOperationSequence()) - .withCatalog(syncConfig.getConfiguredAirbyteCatalog()) - .withState(syncConfig.getState()); - - final IntegrationLauncherConfig destinationLauncherConfig = new IntegrationLauncherConfig() - .withJobId(String.valueOf(JOB_ID)) - .withAttemptId((long) ATTEMPT_ID) - .withDockerImage(IMAGE_NAME2); - - temporalClient.submitSync(JOB_ID, ATTEMPT_ID, syncConfig, CONNECTION_ID); - discoverCatalogWorkflow.run(JOB_RUN_CONFIG, LAUNCHER_CONFIG, destinationLauncherConfig, input, CONNECTION_ID); - verify(workflowClient).newWorkflowStub(SyncWorkflow.class, TemporalWorkflowUtils.buildWorkflowOptions(TemporalJobType.SYNC)); - } - - } - - @Nested - @DisplayName("Test related to the migration to the new scheduler") - class TestMigration { - - @DisplayName("Test that the migration is properly done if needed") - @Test - void migrateCalled() { - final UUID nonMigratedId = UUID.randomUUID(); - final UUID migratedId = UUID.randomUUID(); - - when(temporalClient.isInRunningWorkflowCache(connectionManagerUtils.getConnectionManagerName(nonMigratedId))).thenReturn(false); - when(temporalClient.isInRunningWorkflowCache(connectionManagerUtils.getConnectionManagerName(migratedId))).thenReturn(true); - - doNothing() - .when(temporalClient).refreshRunningWorkflow(); - final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - doReturn(mConnectionManagerWorkflow) - .when(temporalClient).submitConnectionUpdaterAsync(nonMigratedId); - - temporalClient.migrateSyncIfNeeded(Sets.newHashSet(nonMigratedId, migratedId)); - - verify(temporalClient, times(1)).submitConnectionUpdaterAsync(nonMigratedId); - verify(temporalClient, times(0)).submitConnectionUpdaterAsync(migratedId); - } - - } - - @Nested - @DisplayName("Test delete connection method.") - class DeleteConnection { - - @Test - @SuppressWarnings(UNCHECKED) - @DisplayName("Test delete connection method when workflow is in a running state.") - void testDeleteConnection() { - final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - final WorkflowState mWorkflowState = mock(WorkflowState.class); - when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); - when(mWorkflowState.isDeleted()).thenReturn(false); - - doReturn(true).when(temporalClient).isWorkflowReachable(any(UUID.class)); - when(workflowClient.newWorkflowStub(any(Class.class), anyString())).thenReturn(mConnectionManagerWorkflow); - - final JobSyncConfig syncConfig = new JobSyncConfig() - .withSourceDockerImage(IMAGE_NAME1) - .withSourceDockerImage(IMAGE_NAME2) - .withSourceConfiguration(Jsons.emptyObject()) - .withDestinationConfiguration(Jsons.emptyObject()) - .withOperationSequence(List.of()) - .withConfiguredAirbyteCatalog(new ConfiguredAirbyteCatalog()); - - temporalClient.submitSync(JOB_ID, ATTEMPT_ID, syncConfig, CONNECTION_ID); - temporalClient.deleteConnection(CONNECTION_ID); - - verify(workflowClient, Mockito.never()).newSignalWithStartRequest(); - verify(mConnectionManagerWorkflow).deleteConnection(); - } - - @Test - @SuppressWarnings(UNCHECKED) - @DisplayName("Test delete connection method when workflow is in an unexpected state") - void testDeleteConnectionInUnexpectedState() { - final ConnectionManagerWorkflow mTerminatedConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - when(mTerminatedConnectionManagerWorkflow.getState()) - .thenThrow(new IllegalStateException(EXCEPTION_MESSAGE)); - when(workflowClient.newWorkflowStub(any(Class.class), any(String.class))).thenReturn(mTerminatedConnectionManagerWorkflow); - - final ConnectionManagerWorkflow mNewConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - when(workflowClient.newWorkflowStub(any(Class.class), any(WorkflowOptions.class))).thenReturn(mNewConnectionManagerWorkflow); - final BatchRequest mBatchRequest = mock(BatchRequest.class); - when(workflowClient.newSignalWithStartRequest()).thenReturn(mBatchRequest); - - temporalClient.deleteConnection(CONNECTION_ID); - verify(workflowClient).signalWithStart(mBatchRequest); - - // Verify that the deleteConnection signal was passed to the batch request by capturing the - // argument, - // executing the signal, and verifying that the desired signal was executed - final ArgumentCaptor batchRequestAddArgCaptor = ArgumentCaptor.forClass(Proc.class); - verify(mBatchRequest).add(batchRequestAddArgCaptor.capture()); - final Proc signal = batchRequestAddArgCaptor.getValue(); - signal.apply(); - verify(mNewConnectionManagerWorkflow).deleteConnection(); - } - - @Test - @SuppressWarnings(UNCHECKED) - @DisplayName("Test delete connection method when workflow has already been deleted") - void testDeleteConnectionOnDeletedWorkflow() { - final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - final WorkflowState mWorkflowState = mock(WorkflowState.class); - when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); - when(mWorkflowState.isDeleted()).thenReturn(true); - when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow); - mockWorkflowStatus(WorkflowExecutionStatus.WORKFLOW_EXECUTION_STATUS_COMPLETED); - - temporalClient.deleteConnection(CONNECTION_ID); - - verify(temporalClient).deleteConnection(CONNECTION_ID); - verifyNoMoreInteractions(temporalClient); - } - - } - - @Nested - @DisplayName("Test update connection behavior") - class UpdateConnection { - - @Test - @SuppressWarnings(UNCHECKED) - @DisplayName("Test update connection when workflow is running") - void testUpdateConnection() { - final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - final WorkflowState mWorkflowState = mock(WorkflowState.class); - - when(mWorkflowState.isRunning()).thenReturn(true); - when(mWorkflowState.isDeleted()).thenReturn(false); - when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); - when(workflowClient.newWorkflowStub(any(Class.class), any(String.class))).thenReturn(mConnectionManagerWorkflow); - - temporalClient.update(CONNECTION_ID); - - verify(mConnectionManagerWorkflow, Mockito.times(1)).connectionUpdated(); - } - - @Test - @SuppressWarnings(UNCHECKED) - @DisplayName("Test update connection method starts a new workflow when workflow is in an unexpected state") - void testUpdateConnectionInUnexpectedState() { - final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - - when(mConnectionManagerWorkflow.getState()).thenThrow(new IllegalStateException(EXCEPTION_MESSAGE)); - when(workflowClient.newWorkflowStub(any(Class.class), any(String.class))).thenReturn(mConnectionManagerWorkflow); - doReturn(mConnectionManagerWorkflow).when(temporalClient).submitConnectionUpdaterAsync(CONNECTION_ID); - - final WorkflowStub untypedWorkflowStub = mock(WorkflowStub.class); - when(workflowClient.newUntypedWorkflowStub(anyString())).thenReturn(untypedWorkflowStub); - - temporalClient.update(CONNECTION_ID); - - // this is only called when updating an existing workflow - verify(mConnectionManagerWorkflow, Mockito.never()).connectionUpdated(); - - verify(untypedWorkflowStub, Mockito.times(1)).terminate(anyString()); - verify(temporalClient, Mockito.times(1)).submitConnectionUpdaterAsync(CONNECTION_ID); - } - - @Test - @SuppressWarnings(UNCHECKED) - @DisplayName("Test update connection method does nothing when connection is deleted") - void testUpdateConnectionDeletedWorkflow() { - final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - final WorkflowState mWorkflowState = mock(WorkflowState.class); - when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); - when(mWorkflowState.isDeleted()).thenReturn(true); - when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow); - mockWorkflowStatus(WorkflowExecutionStatus.WORKFLOW_EXECUTION_STATUS_COMPLETED); - - temporalClient.update(CONNECTION_ID); - - // this is only called when updating an existing workflow - verify(mConnectionManagerWorkflow, Mockito.never()).connectionUpdated(); - verify(temporalClient).update(CONNECTION_ID); - verifyNoMoreInteractions(temporalClient); - } - - } - - @Nested - @DisplayName("Test manual sync behavior") - class ManualSync { - - @Test - @DisplayName("Test startNewManualSync successful") - void testStartNewManualSyncSuccess() { - final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - final WorkflowState mWorkflowState = mock(WorkflowState.class); - when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); - when(mWorkflowState.isDeleted()).thenReturn(false); - when(mWorkflowState.isRunning()).thenReturn(false).thenReturn(true); - when(mConnectionManagerWorkflow.getJobInformation()).thenReturn(new JobInformation(JOB_ID, ATTEMPT_ID)); - when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow); - - final ManualOperationResult result = temporalClient.startNewManualSync(CONNECTION_ID); - - assertTrue(result.getJobId().isPresent()); - assertEquals(JOB_ID, result.getJobId().get()); - assertFalse(result.getFailingReason().isPresent()); - verify(mConnectionManagerWorkflow).submitManualSync(); - } - - @Test - @DisplayName("Test startNewManualSync fails if job is already running") - void testStartNewManualSyncAlreadyRunning() { - final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - final WorkflowState mWorkflowState = mock(WorkflowState.class); - when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); - when(mWorkflowState.isDeleted()).thenReturn(false); - when(mWorkflowState.isRunning()).thenReturn(true); - when(mConnectionManagerWorkflow.getJobInformation()).thenReturn(new JobInformation(JOB_ID, ATTEMPT_ID)); - when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow); - - final ManualOperationResult result = temporalClient.startNewManualSync(CONNECTION_ID); - - assertFalse(result.getJobId().isPresent()); - assertTrue(result.getFailingReason().isPresent()); - verify(mConnectionManagerWorkflow, times(0)).submitManualSync(); - } - - @Test - @DisplayName("Test startNewManualSync repairs the workflow if it is in a bad state") - void testStartNewManualSyncRepairsBadWorkflowState() { - final ConnectionManagerWorkflow mTerminatedConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - when(mTerminatedConnectionManagerWorkflow.getState()) - .thenThrow(new IllegalStateException(EXCEPTION_MESSAGE)); - when(mTerminatedConnectionManagerWorkflow.getJobInformation()).thenReturn(new JobInformation(JOB_ID, ATTEMPT_ID)); - - final ConnectionManagerWorkflow mNewConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - final WorkflowState mWorkflowState = mock(WorkflowState.class); - when(mNewConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); - when(mWorkflowState.isDeleted()).thenReturn(false); - when(mWorkflowState.isRunning()).thenReturn(false).thenReturn(true); - when(mNewConnectionManagerWorkflow.getJobInformation()).thenReturn(new JobInformation(JOB_ID, ATTEMPT_ID)); - when(workflowClient.newWorkflowStub(any(Class.class), any(WorkflowOptions.class))).thenReturn(mNewConnectionManagerWorkflow); - final BatchRequest mBatchRequest = mock(BatchRequest.class); - when(workflowClient.newSignalWithStartRequest()).thenReturn(mBatchRequest); - - when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mTerminatedConnectionManagerWorkflow, mTerminatedConnectionManagerWorkflow, - mNewConnectionManagerWorkflow); - - final ManualOperationResult result = temporalClient.startNewManualSync(CONNECTION_ID); - - assertTrue(result.getJobId().isPresent()); - assertEquals(JOB_ID, result.getJobId().get()); - assertFalse(result.getFailingReason().isPresent()); - verify(workflowClient).signalWithStart(mBatchRequest); - - // Verify that the submitManualSync signal was passed to the batch request by capturing the - // argument, - // executing the signal, and verifying that the desired signal was executed - final ArgumentCaptor batchRequestAddArgCaptor = ArgumentCaptor.forClass(Proc.class); - verify(mBatchRequest).add(batchRequestAddArgCaptor.capture()); - final Proc signal = batchRequestAddArgCaptor.getValue(); - signal.apply(); - verify(mNewConnectionManagerWorkflow).submitManualSync(); - } - - @Test - @SuppressWarnings(UNCHECKED) - @DisplayName("Test startNewManualSync returns a failure reason when connection is deleted") - void testStartNewManualSyncDeletedWorkflow() { - final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - final WorkflowState mWorkflowState = mock(WorkflowState.class); - when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); - when(mWorkflowState.isDeleted()).thenReturn(true); - when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow); - mockWorkflowStatus(WorkflowExecutionStatus.WORKFLOW_EXECUTION_STATUS_COMPLETED); - - final ManualOperationResult result = temporalClient.startNewManualSync(CONNECTION_ID); - - // this is only called when updating an existing workflow - assertFalse(result.getJobId().isPresent()); - assertTrue(result.getFailingReason().isPresent()); - verify(mConnectionManagerWorkflow, times(0)).submitManualSync(); - } - - } - - @Nested - @DisplayName("Test cancellation behavior") - class Cancellation { - - @Test - @DisplayName("Test startNewCancellation successful") - void testStartNewCancellationSuccess() { - final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - final WorkflowState mWorkflowState = mock(WorkflowState.class); - when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); - when(mWorkflowState.isDeleted()).thenReturn(false); - when(mWorkflowState.isRunning()).thenReturn(true).thenReturn(false); - when(mConnectionManagerWorkflow.getJobInformation()).thenReturn(new JobInformation(JOB_ID, ATTEMPT_ID)); - when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow); - - final ManualOperationResult result = temporalClient.startNewCancellation(CONNECTION_ID); - - assertTrue(result.getJobId().isPresent()); - assertEquals(JOB_ID, result.getJobId().get()); - assertFalse(result.getFailingReason().isPresent()); - verify(mConnectionManagerWorkflow).cancelJob(); - verify(streamResetRecordsHelper).deleteStreamResetRecordsForJob(JOB_ID, CONNECTION_ID); - } - - @Test - @DisplayName("Test startNewCancellation repairs the workflow if it is in a bad state") - void testStartNewCancellationRepairsBadWorkflowState() { - final ConnectionManagerWorkflow mTerminatedConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - when(mTerminatedConnectionManagerWorkflow.getState()) - .thenThrow(new IllegalStateException(EXCEPTION_MESSAGE)); - when(mTerminatedConnectionManagerWorkflow.getJobInformation()).thenReturn(new JobInformation(JOB_ID, ATTEMPT_ID)); - - final ConnectionManagerWorkflow mNewConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - final WorkflowState mWorkflowState = mock(WorkflowState.class); - when(mNewConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); - when(mWorkflowState.isDeleted()).thenReturn(false); - when(mWorkflowState.isRunning()).thenReturn(true).thenReturn(false); - when(mNewConnectionManagerWorkflow.getJobInformation()).thenReturn(new JobInformation(JOB_ID, ATTEMPT_ID)); - when(workflowClient.newWorkflowStub(any(Class.class), any(WorkflowOptions.class))).thenReturn(mNewConnectionManagerWorkflow); - final BatchRequest mBatchRequest = mock(BatchRequest.class); - when(workflowClient.newSignalWithStartRequest()).thenReturn(mBatchRequest); - - when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mTerminatedConnectionManagerWorkflow, mTerminatedConnectionManagerWorkflow, - mNewConnectionManagerWorkflow); - - final ManualOperationResult result = temporalClient.startNewCancellation(CONNECTION_ID); - - assertTrue(result.getJobId().isPresent()); - assertEquals(NON_RUNNING_JOB_ID, result.getJobId().get()); - assertFalse(result.getFailingReason().isPresent()); - verify(workflowClient).signalWithStart(mBatchRequest); - - // Verify that the cancelJob signal was passed to the batch request by capturing the argument, - // executing the signal, and verifying that the desired signal was executed - final ArgumentCaptor batchRequestAddArgCaptor = ArgumentCaptor.forClass(Proc.class); - verify(mBatchRequest).add(batchRequestAddArgCaptor.capture()); - final Proc signal = batchRequestAddArgCaptor.getValue(); - signal.apply(); - verify(mNewConnectionManagerWorkflow).cancelJob(); - } - - @Test - @SuppressWarnings(UNCHECKED) - @DisplayName("Test startNewCancellation returns a failure reason when connection is deleted") - void testStartNewCancellationDeletedWorkflow() { - final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - final WorkflowState mWorkflowState = mock(WorkflowState.class); - when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); - when(mWorkflowState.isDeleted()).thenReturn(true); - when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow); - mockWorkflowStatus(WorkflowExecutionStatus.WORKFLOW_EXECUTION_STATUS_COMPLETED); - - final ManualOperationResult result = temporalClient.startNewCancellation(CONNECTION_ID); - - // this is only called when updating an existing workflow - assertFalse(result.getJobId().isPresent()); - assertTrue(result.getFailingReason().isPresent()); - verify(mConnectionManagerWorkflow, times(0)).cancelJob(); - } - - } - - @Nested - @DisplayName("Test reset connection behavior") - class ResetConnection { - - @Test - @DisplayName("Test resetConnection successful") - void testResetConnectionSuccess() throws IOException { - final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - final WorkflowState mWorkflowState = mock(WorkflowState.class); - when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); - when(mWorkflowState.isDeleted()).thenReturn(false); - when(mWorkflowState.isRunning()).thenReturn(false); - final long jobId1 = 1; - final long jobId2 = 2; - when(mConnectionManagerWorkflow.getJobInformation()).thenReturn( - new JobInformation(jobId1, 0), - new JobInformation(jobId1, 0), - new JobInformation(NON_RUNNING_JOB_ID, 0), - new JobInformation(NON_RUNNING_JOB_ID, 0), - new JobInformation(jobId2, 0), - new JobInformation(jobId2, 0)); - when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow); - - final List streamsToReset = List.of(STREAM_DESCRIPTOR); - final ManualOperationResult result = temporalClient.resetConnection(CONNECTION_ID, streamsToReset, false); - - verify(streamResetPersistence).createStreamResets(CONNECTION_ID, streamsToReset); - - assertTrue(result.getJobId().isPresent()); - assertEquals(jobId2, result.getJobId().get()); - assertFalse(result.getFailingReason().isPresent()); - verify(mConnectionManagerWorkflow).resetConnection(); - } - - @Test - @DisplayName("Test resetConnection successful") - void testResetConnectionSuccessAndContinue() throws IOException { - final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - final WorkflowState mWorkflowState = mock(WorkflowState.class); - when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); - when(mWorkflowState.isDeleted()).thenReturn(false); - when(mWorkflowState.isRunning()).thenReturn(false); - final long jobId1 = 1; - final long jobId2 = 2; - when(mConnectionManagerWorkflow.getJobInformation()).thenReturn( - new JobInformation(jobId1, 0), - new JobInformation(jobId1, 0), - new JobInformation(NON_RUNNING_JOB_ID, 0), - new JobInformation(NON_RUNNING_JOB_ID, 0), - new JobInformation(jobId2, 0), - new JobInformation(jobId2, 0)); - when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow); - - final List streamsToReset = List.of(STREAM_DESCRIPTOR); - final ManualOperationResult result = temporalClient.resetConnection(CONNECTION_ID, streamsToReset, true); - - verify(streamResetPersistence).createStreamResets(CONNECTION_ID, streamsToReset); - - assertTrue(result.getJobId().isPresent()); - assertEquals(jobId2, result.getJobId().get()); - assertFalse(result.getFailingReason().isPresent()); - verify(mConnectionManagerWorkflow).resetConnectionAndSkipNextScheduling(); - } - - @Test - @DisplayName("Test resetConnection repairs the workflow if it is in a bad state") - void testResetConnectionRepairsBadWorkflowState() throws IOException { - final ConnectionManagerWorkflow mTerminatedConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - when(mTerminatedConnectionManagerWorkflow.getState()) - .thenThrow(new IllegalStateException(EXCEPTION_MESSAGE)); - when(mTerminatedConnectionManagerWorkflow.getJobInformation()).thenReturn(new JobInformation(JOB_ID, ATTEMPT_ID)); - - final ConnectionManagerWorkflow mNewConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - final WorkflowState mWorkflowState = mock(WorkflowState.class); - when(mNewConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); - when(mWorkflowState.isDeleted()).thenReturn(false); - when(mWorkflowState.isRunning()).thenReturn(false); - when(mNewConnectionManagerWorkflow.getJobInformation()).thenReturn( - new JobInformation(NON_RUNNING_JOB_ID, 0), - new JobInformation(NON_RUNNING_JOB_ID, 0), - new JobInformation(JOB_ID, 0), - new JobInformation(JOB_ID, 0)); - when(workflowClient.newWorkflowStub(any(Class.class), any(WorkflowOptions.class))).thenReturn(mNewConnectionManagerWorkflow); - final BatchRequest mBatchRequest = mock(BatchRequest.class); - when(workflowClient.newSignalWithStartRequest()).thenReturn(mBatchRequest); - - when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mTerminatedConnectionManagerWorkflow, mTerminatedConnectionManagerWorkflow, - mNewConnectionManagerWorkflow); - - final List streamsToReset = List.of(STREAM_DESCRIPTOR); - final ManualOperationResult result = temporalClient.resetConnection(CONNECTION_ID, streamsToReset, false); - - verify(streamResetPersistence).createStreamResets(CONNECTION_ID, streamsToReset); - - assertTrue(result.getJobId().isPresent()); - assertEquals(JOB_ID, result.getJobId().get()); - assertFalse(result.getFailingReason().isPresent()); - verify(workflowClient).signalWithStart(mBatchRequest); - - // Verify that the resetConnection signal was passed to the batch request by capturing the argument, - // executing the signal, and verifying that the desired signal was executed - final ArgumentCaptor batchRequestAddArgCaptor = ArgumentCaptor.forClass(Proc.class); - verify(mBatchRequest).add(batchRequestAddArgCaptor.capture()); - final Proc signal = batchRequestAddArgCaptor.getValue(); - signal.apply(); - verify(mNewConnectionManagerWorkflow).resetConnection(); - } - - @Test - @SuppressWarnings(UNCHECKED) - @DisplayName("Test resetConnection returns a failure reason when connection is deleted") - void testResetConnectionDeletedWorkflow() throws IOException { - final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - final WorkflowState mWorkflowState = mock(WorkflowState.class); - when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); - when(mWorkflowState.isDeleted()).thenReturn(true); - when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow); - mockWorkflowStatus(WorkflowExecutionStatus.WORKFLOW_EXECUTION_STATUS_COMPLETED); - - final List streamsToReset = List.of(STREAM_DESCRIPTOR); - final ManualOperationResult result = temporalClient.resetConnection(CONNECTION_ID, streamsToReset, false); - - verify(streamResetPersistence).createStreamResets(CONNECTION_ID, streamsToReset); - - // this is only called when updating an existing workflow - assertFalse(result.getJobId().isPresent()); - assertTrue(result.getFailingReason().isPresent()); - verify(mConnectionManagerWorkflow, times(0)).resetConnection(); - } - - } - - @Test - @DisplayName("Test manual operation on quarantined workflow causes a restart") - void testManualOperationOnQuarantinedWorkflow() { - final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - final WorkflowState mWorkflowState = mock(WorkflowState.class); - when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); - when(mWorkflowState.isQuarantined()).thenReturn(true); - - final ConnectionManagerWorkflow mNewConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - final WorkflowState mNewWorkflowState = mock(WorkflowState.class); - when(mNewConnectionManagerWorkflow.getState()).thenReturn(mNewWorkflowState); - when(mNewWorkflowState.isRunning()).thenReturn(false).thenReturn(true); - when(mNewConnectionManagerWorkflow.getJobInformation()).thenReturn(new JobInformation(JOB_ID, ATTEMPT_ID)); - when(workflowClient.newWorkflowStub(any(Class.class), any(WorkflowOptions.class))).thenReturn(mNewConnectionManagerWorkflow); - final BatchRequest mBatchRequest = mock(BatchRequest.class); - when(workflowClient.newSignalWithStartRequest()).thenReturn(mBatchRequest); - - when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow, mConnectionManagerWorkflow, - mNewConnectionManagerWorkflow); - - final WorkflowStub mWorkflowStub = mock(WorkflowStub.class); - when(workflowClient.newUntypedWorkflowStub(anyString())).thenReturn(mWorkflowStub); - - final ManualOperationResult result = temporalClient.startNewManualSync(CONNECTION_ID); - - assertTrue(result.getJobId().isPresent()); - assertEquals(JOB_ID, result.getJobId().get()); - assertFalse(result.getFailingReason().isPresent()); - verify(workflowClient).signalWithStart(mBatchRequest); - verify(mWorkflowStub).terminate(anyString()); - - // Verify that the submitManualSync signal was passed to the batch request by capturing the - // argument, - // executing the signal, and verifying that the desired signal was executed - final ArgumentCaptor batchRequestAddArgCaptor = ArgumentCaptor.forClass(Proc.class); - verify(mBatchRequest).add(batchRequestAddArgCaptor.capture()); - final Proc signal = batchRequestAddArgCaptor.getValue(); - signal.apply(); - verify(mNewConnectionManagerWorkflow).submitManualSync(); - } - - @Test - @DisplayName("Test manual operation on completed workflow causes a restart") - void testManualOperationOnCompletedWorkflow() { - final ConnectionManagerWorkflow mConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - final WorkflowState mWorkflowState = mock(WorkflowState.class); - when(mConnectionManagerWorkflow.getState()).thenReturn(mWorkflowState); - when(mWorkflowState.isQuarantined()).thenReturn(false); - when(mWorkflowState.isDeleted()).thenReturn(false); - when(workflowServiceBlockingStub.describeWorkflowExecution(any())) - .thenReturn(DescribeWorkflowExecutionResponse.newBuilder().setWorkflowExecutionInfo( - WorkflowExecutionInfo.newBuilder().setStatus(WorkflowExecutionStatus.WORKFLOW_EXECUTION_STATUS_COMPLETED).buildPartial()).build()) - .thenReturn(DescribeWorkflowExecutionResponse.newBuilder().setWorkflowExecutionInfo( - WorkflowExecutionInfo.newBuilder().setStatus(WorkflowExecutionStatus.WORKFLOW_EXECUTION_STATUS_RUNNING).buildPartial()).build()); - - final ConnectionManagerWorkflow mNewConnectionManagerWorkflow = mock(ConnectionManagerWorkflow.class); - final WorkflowState mNewWorkflowState = mock(WorkflowState.class); - when(mNewConnectionManagerWorkflow.getState()).thenReturn(mNewWorkflowState); - when(mNewWorkflowState.isRunning()).thenReturn(false).thenReturn(true); - when(mNewConnectionManagerWorkflow.getJobInformation()).thenReturn(new JobInformation(JOB_ID, ATTEMPT_ID)); - when(workflowClient.newWorkflowStub(any(Class.class), any(WorkflowOptions.class))).thenReturn(mNewConnectionManagerWorkflow); - final BatchRequest mBatchRequest = mock(BatchRequest.class); - when(workflowClient.newSignalWithStartRequest()).thenReturn(mBatchRequest); - - when(workflowClient.newWorkflowStub(any(), anyString())).thenReturn(mConnectionManagerWorkflow, mConnectionManagerWorkflow, - mNewConnectionManagerWorkflow); - - final WorkflowStub mWorkflowStub = mock(WorkflowStub.class); - when(workflowClient.newUntypedWorkflowStub(anyString())).thenReturn(mWorkflowStub); - - final ManualOperationResult result = temporalClient.startNewManualSync(CONNECTION_ID); - - assertTrue(result.getJobId().isPresent()); - assertEquals(JOB_ID, result.getJobId().get()); - assertFalse(result.getFailingReason().isPresent()); - verify(workflowClient).signalWithStart(mBatchRequest); - verify(mWorkflowStub).terminate(anyString()); - - // Verify that the submitManualSync signal was passed to the batch request by capturing the - // argument, - // executing the signal, and verifying that the desired signal was executed - final ArgumentCaptor batchRequestAddArgCaptor = ArgumentCaptor.forClass(Proc.class); - verify(mBatchRequest).add(batchRequestAddArgCaptor.capture()); - final Proc signal = batchRequestAddArgCaptor.getValue(); - signal.apply(); - verify(mNewConnectionManagerWorkflow).submitManualSync(); - } - - private void mockWorkflowStatus(final WorkflowExecutionStatus status) { - when(workflowServiceBlockingStub.describeWorkflowExecution(any())).thenReturn( - DescribeWorkflowExecutionResponse.newBuilder().setWorkflowExecutionInfo( - WorkflowExecutionInfo.newBuilder().setStatus(status).buildPartial()).build()); - } - -} diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/ConnectionManagerWorkflowTest.java b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/ConnectionManagerWorkflowTest.java index 8c91948b05f2b..a75e9275459df 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/ConnectionManagerWorkflowTest.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/ConnectionManagerWorkflowTest.java @@ -11,6 +11,7 @@ import io.airbyte.commons.temporal.TemporalJobType; import io.airbyte.commons.temporal.scheduling.ConnectionManagerWorkflow; import io.airbyte.commons.temporal.scheduling.ConnectionUpdaterInput; +import io.airbyte.commons.temporal.scheduling.SyncWorkflow; import io.airbyte.commons.temporal.scheduling.state.WorkflowState; import io.airbyte.commons.temporal.scheduling.state.listener.TestStateListener; import io.airbyte.commons.temporal.scheduling.state.listener.WorkflowStateChangedListener.ChangedStateEvent; @@ -57,7 +58,6 @@ import io.airbyte.workers.temporal.scheduling.testsyncworkflow.SleepingSyncWorkflow; import io.airbyte.workers.temporal.scheduling.testsyncworkflow.SourceAndDestinationFailureSyncWorkflow; import io.airbyte.workers.temporal.support.TemporalProxyHelper; -import io.airbyte.workers.temporal.sync.SyncWorkflow; import io.micronaut.context.BeanRegistration; import io.micronaut.inject.BeanIdentifier; import io.temporal.activity.ActivityOptions; diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/activities/ConnectionDeletionActivityTest.java b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/activities/ConnectionDeletionActivityTest.java index b38f5bd1157dd..326b79447fd44 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/activities/ConnectionDeletionActivityTest.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/activities/ConnectionDeletionActivityTest.java @@ -4,10 +4,10 @@ package io.airbyte.workers.temporal.scheduling.activities; +import io.airbyte.commons.temporal.exception.RetryableException; import io.airbyte.config.persistence.ConfigNotFoundException; import io.airbyte.validation.json.JsonValidationException; import io.airbyte.workers.helper.ConnectionHelper; -import io.airbyte.workers.temporal.exception.RetryableException; import io.airbyte.workers.temporal.scheduling.activities.ConnectionDeletionActivity.ConnectionDeletionInput; import java.io.IOException; import java.util.UUID; diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/activities/JobCreationAndStatusUpdateActivityTest.java b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/activities/JobCreationAndStatusUpdateActivityTest.java index 876bafbdf2192..57d856a111d3a 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/activities/JobCreationAndStatusUpdateActivityTest.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/activities/JobCreationAndStatusUpdateActivityTest.java @@ -10,6 +10,7 @@ import static org.mockito.Mockito.verify; import io.airbyte.commons.docker.DockerUtils; +import io.airbyte.commons.temporal.exception.RetryableException; import io.airbyte.config.AttemptFailureSummary; import io.airbyte.config.Configs.WorkerEnvironment; import io.airbyte.config.DestinationConnection; @@ -45,7 +46,6 @@ import io.airbyte.validation.json.JsonValidationException; import io.airbyte.workers.run.TemporalWorkerRunFactory; import io.airbyte.workers.run.WorkerRun; -import io.airbyte.workers.temporal.exception.RetryableException; import io.airbyte.workers.temporal.scheduling.activities.JobCreationAndStatusUpdateActivity.AttemptCreationInput; import io.airbyte.workers.temporal.scheduling.activities.JobCreationAndStatusUpdateActivity.AttemptCreationOutput; import io.airbyte.workers.temporal.scheduling.activities.JobCreationAndStatusUpdateActivity.AttemptFailureInput; diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/activities/StreamResetActivityTest.java b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/activities/StreamResetActivityTest.java index d2fc66d761847..15808733acdb7 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/activities/StreamResetActivityTest.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/activities/StreamResetActivityTest.java @@ -7,7 +7,7 @@ import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.verify; -import io.airbyte.workers.temporal.StreamResetRecordsHelper; +import io.airbyte.commons.temporal.StreamResetRecordsHelper; import io.airbyte.workers.temporal.scheduling.activities.StreamResetActivity.DeleteStreamResetRecordsForJobInput; import java.util.UUID; import org.junit.jupiter.api.Test; diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/DbtFailureSyncWorkflow.java b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/DbtFailureSyncWorkflow.java index e91a60d532383..677740d2f796f 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/DbtFailureSyncWorkflow.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/DbtFailureSyncWorkflow.java @@ -4,11 +4,11 @@ package io.airbyte.workers.temporal.scheduling.testsyncworkflow; +import io.airbyte.commons.temporal.scheduling.SyncWorkflow; import io.airbyte.config.StandardSyncInput; import io.airbyte.config.StandardSyncOutput; import io.airbyte.persistence.job.models.IntegrationLauncherConfig; import io.airbyte.persistence.job.models.JobRunConfig; -import io.airbyte.workers.temporal.sync.SyncWorkflow; import io.temporal.api.enums.v1.RetryState; import io.temporal.failure.ActivityFailure; import java.util.UUID; diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/EmptySyncWorkflow.java b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/EmptySyncWorkflow.java index 27bb555550639..3f93c6ba87d24 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/EmptySyncWorkflow.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/EmptySyncWorkflow.java @@ -4,11 +4,11 @@ package io.airbyte.workers.temporal.scheduling.testsyncworkflow; +import io.airbyte.commons.temporal.scheduling.SyncWorkflow; import io.airbyte.config.StandardSyncInput; import io.airbyte.config.StandardSyncOutput; import io.airbyte.persistence.job.models.IntegrationLauncherConfig; import io.airbyte.persistence.job.models.JobRunConfig; -import io.airbyte.workers.temporal.sync.SyncWorkflow; import java.util.UUID; public class EmptySyncWorkflow implements SyncWorkflow { diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/NormalizationFailureSyncWorkflow.java b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/NormalizationFailureSyncWorkflow.java index 8f6e9d0ae69e6..99e52c941c206 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/NormalizationFailureSyncWorkflow.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/NormalizationFailureSyncWorkflow.java @@ -4,11 +4,11 @@ package io.airbyte.workers.temporal.scheduling.testsyncworkflow; +import io.airbyte.commons.temporal.scheduling.SyncWorkflow; import io.airbyte.config.StandardSyncInput; import io.airbyte.config.StandardSyncOutput; import io.airbyte.persistence.job.models.IntegrationLauncherConfig; import io.airbyte.persistence.job.models.JobRunConfig; -import io.airbyte.workers.temporal.sync.SyncWorkflow; import io.temporal.api.enums.v1.RetryState; import io.temporal.failure.ActivityFailure; import java.util.UUID; diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/NormalizationTraceFailureSyncWorkflow.java b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/NormalizationTraceFailureSyncWorkflow.java index 4df2b66e92b20..1845f0ac0e459 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/NormalizationTraceFailureSyncWorkflow.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/NormalizationTraceFailureSyncWorkflow.java @@ -5,6 +5,7 @@ package io.airbyte.workers.temporal.scheduling.testsyncworkflow; import com.google.common.annotations.VisibleForTesting; +import io.airbyte.commons.temporal.scheduling.SyncWorkflow; import io.airbyte.config.FailureReason; import io.airbyte.config.FailureReason.FailureOrigin; import io.airbyte.config.NormalizationSummary; @@ -12,7 +13,6 @@ import io.airbyte.config.StandardSyncOutput; import io.airbyte.persistence.job.models.IntegrationLauncherConfig; import io.airbyte.persistence.job.models.JobRunConfig; -import io.airbyte.workers.temporal.sync.SyncWorkflow; import java.util.List; import java.util.UUID; diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/PersistFailureSyncWorkflow.java b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/PersistFailureSyncWorkflow.java index 54a97b3f2988e..e8b704a6512c0 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/PersistFailureSyncWorkflow.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/PersistFailureSyncWorkflow.java @@ -4,11 +4,11 @@ package io.airbyte.workers.temporal.scheduling.testsyncworkflow; +import io.airbyte.commons.temporal.scheduling.SyncWorkflow; import io.airbyte.config.StandardSyncInput; import io.airbyte.config.StandardSyncOutput; import io.airbyte.persistence.job.models.IntegrationLauncherConfig; import io.airbyte.persistence.job.models.JobRunConfig; -import io.airbyte.workers.temporal.sync.SyncWorkflow; import io.temporal.api.enums.v1.RetryState; import io.temporal.failure.ActivityFailure; import java.util.UUID; diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/ReplicateFailureSyncWorkflow.java b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/ReplicateFailureSyncWorkflow.java index 7706b860e1f22..d4c64152e2cc4 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/ReplicateFailureSyncWorkflow.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/ReplicateFailureSyncWorkflow.java @@ -4,11 +4,11 @@ package io.airbyte.workers.temporal.scheduling.testsyncworkflow; +import io.airbyte.commons.temporal.scheduling.SyncWorkflow; import io.airbyte.config.StandardSyncInput; import io.airbyte.config.StandardSyncOutput; import io.airbyte.persistence.job.models.IntegrationLauncherConfig; import io.airbyte.persistence.job.models.JobRunConfig; -import io.airbyte.workers.temporal.sync.SyncWorkflow; import io.temporal.api.enums.v1.RetryState; import io.temporal.failure.ActivityFailure; import java.util.UUID; diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/SleepingSyncWorkflow.java b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/SleepingSyncWorkflow.java index 59dba751228d1..c2f24529de127 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/SleepingSyncWorkflow.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/SleepingSyncWorkflow.java @@ -4,11 +4,11 @@ package io.airbyte.workers.temporal.scheduling.testsyncworkflow; +import io.airbyte.commons.temporal.scheduling.SyncWorkflow; import io.airbyte.config.StandardSyncInput; import io.airbyte.config.StandardSyncOutput; import io.airbyte.persistence.job.models.IntegrationLauncherConfig; import io.airbyte.persistence.job.models.JobRunConfig; -import io.airbyte.workers.temporal.sync.SyncWorkflow; import io.temporal.workflow.Workflow; import java.time.Duration; import java.util.UUID; diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/SourceAndDestinationFailureSyncWorkflow.java b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/SourceAndDestinationFailureSyncWorkflow.java index 7f4d1664d4c7a..19a8bdabbe1c6 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/SourceAndDestinationFailureSyncWorkflow.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/SourceAndDestinationFailureSyncWorkflow.java @@ -5,6 +5,7 @@ package io.airbyte.workers.temporal.scheduling.testsyncworkflow; import com.google.common.annotations.VisibleForTesting; +import io.airbyte.commons.temporal.scheduling.SyncWorkflow; import io.airbyte.config.FailureReason; import io.airbyte.config.FailureReason.FailureOrigin; import io.airbyte.config.StandardSyncInput; @@ -14,7 +15,6 @@ import io.airbyte.config.SyncStats; import io.airbyte.persistence.job.models.IntegrationLauncherConfig; import io.airbyte.persistence.job.models.JobRunConfig; -import io.airbyte.workers.temporal.sync.SyncWorkflow; import java.util.Set; import java.util.UUID; import org.assertj.core.util.Sets; diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/SyncWorkflowFailingOutputWorkflow.java b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/SyncWorkflowFailingOutputWorkflow.java index 87b4c18f2a162..07925086dcc7e 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/SyncWorkflowFailingOutputWorkflow.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/SyncWorkflowFailingOutputWorkflow.java @@ -4,11 +4,11 @@ package io.airbyte.workers.temporal.scheduling.testsyncworkflow; +import io.airbyte.commons.temporal.scheduling.SyncWorkflow; import io.airbyte.config.StandardSyncInput; import io.airbyte.config.StandardSyncOutput; import io.airbyte.persistence.job.models.IntegrationLauncherConfig; import io.airbyte.persistence.job.models.JobRunConfig; -import io.airbyte.workers.temporal.sync.SyncWorkflow; import java.util.UUID; public class SyncWorkflowFailingOutputWorkflow implements SyncWorkflow { diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/SyncWorkflowFailingWithHearbeatTimeoutException.java b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/SyncWorkflowFailingWithHearbeatTimeoutException.java index c854bbb621b22..b09a20163e95d 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/SyncWorkflowFailingWithHearbeatTimeoutException.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/SyncWorkflowFailingWithHearbeatTimeoutException.java @@ -4,11 +4,11 @@ package io.airbyte.workers.temporal.scheduling.testsyncworkflow; +import io.airbyte.commons.temporal.scheduling.SyncWorkflow; import io.airbyte.config.StandardSyncInput; import io.airbyte.config.StandardSyncOutput; import io.airbyte.persistence.job.models.IntegrationLauncherConfig; import io.airbyte.persistence.job.models.JobRunConfig; -import io.airbyte.workers.temporal.sync.SyncWorkflow; import io.temporal.api.enums.v1.TimeoutType; import io.temporal.failure.TimeoutFailure; import io.temporal.workflow.Workflow; diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/SyncWorkflowWithActivityFailureException.java b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/SyncWorkflowWithActivityFailureException.java index 35e53678a99b3..2a897d73dabd1 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/SyncWorkflowWithActivityFailureException.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/scheduling/testsyncworkflow/SyncWorkflowWithActivityFailureException.java @@ -4,11 +4,11 @@ package io.airbyte.workers.temporal.scheduling.testsyncworkflow; +import io.airbyte.commons.temporal.scheduling.SyncWorkflow; import io.airbyte.config.StandardSyncInput; import io.airbyte.config.StandardSyncOutput; import io.airbyte.persistence.job.models.IntegrationLauncherConfig; import io.airbyte.persistence.job.models.JobRunConfig; -import io.airbyte.workers.temporal.sync.SyncWorkflow; import io.temporal.api.enums.v1.RetryState; import io.temporal.failure.ActivityFailure; import io.temporal.workflow.Workflow; diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/stubs/ErrorTestWorkflowImpl.java b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/stubs/ErrorTestWorkflowImpl.java index ed419ba481f96..71b97086ede58 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/stubs/ErrorTestWorkflowImpl.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/stubs/ErrorTestWorkflowImpl.java @@ -4,7 +4,7 @@ package io.airbyte.workers.temporal.stubs; -import io.airbyte.workers.temporal.exception.RetryableException; +import io.airbyte.commons.temporal.exception.RetryableException; public class ErrorTestWorkflowImpl implements TestWorkflow { diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/stubs/InvalidTestWorkflowImpl.java b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/stubs/InvalidTestWorkflowImpl.java index 2a749650d91cb..afc43d56ce531 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/stubs/InvalidTestWorkflowImpl.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/stubs/InvalidTestWorkflowImpl.java @@ -4,8 +4,8 @@ package io.airbyte.workers.temporal.stubs; +import io.airbyte.commons.temporal.exception.RetryableException; import io.airbyte.workers.temporal.annotations.TemporalActivityStub; -import io.airbyte.workers.temporal.exception.RetryableException; @SuppressWarnings("PMD.UnusedPrivateField") public class InvalidTestWorkflowImpl implements TestWorkflow { diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/support/TemporalActivityStubInterceptorTest.java b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/support/TemporalActivityStubInterceptorTest.java index bbf5c75f017a9..6b4a2f33e6cf5 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/support/TemporalActivityStubInterceptorTest.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/support/TemporalActivityStubInterceptorTest.java @@ -7,7 +7,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -import io.airbyte.workers.temporal.exception.RetryableException; +import io.airbyte.commons.temporal.exception.RetryableException; import io.airbyte.workers.temporal.stubs.ErrorTestWorkflowImpl; import io.airbyte.workers.temporal.stubs.InvalidTestWorkflowImpl; import io.airbyte.workers.temporal.stubs.TestActivity; diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/sync/SyncWorkflowTest.java b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/sync/SyncWorkflowTest.java index 9c062c38c55bd..d0d692645f9bc 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/temporal/sync/SyncWorkflowTest.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/temporal/sync/SyncWorkflowTest.java @@ -16,6 +16,7 @@ import static org.mockito.Mockito.when; import io.airbyte.commons.temporal.TemporalUtils; +import io.airbyte.commons.temporal.scheduling.SyncWorkflow; import io.airbyte.config.NormalizationInput; import io.airbyte.config.NormalizationSummary; import io.airbyte.config.OperatorDbtInput; diff --git a/charts/airbyte/templates/env-configmap.yaml b/charts/airbyte/templates/env-configmap.yaml index b9945de80ed21..19006ff3043e9 100644 --- a/charts/airbyte/templates/env-configmap.yaml +++ b/charts/airbyte/templates/env-configmap.yaml @@ -64,6 +64,7 @@ data: CONTAINER_ORCHESTRATOR_ENABLED: {{ .Values.worker.containerOrchestrator.enabled | quote }} CONTAINER_ORCHESTRATOR_IMAGE: {{ .Values.worker.containerOrchestrator.image | quote }} WORKERS_MICRONAUT_ENVIRONMENTS: "control-plane" + CRON_MICRONAUT_ENVIRONMENTS: "control-plane" WORKER_LOGS_STORAGE_TYPE: {{ .Values.global.logs.storage.type | quote }} WORKER_STATE_STORAGE_TYPE: {{ .Values.global.state.storage.type | quote }} {{- end }} diff --git a/docker-compose.yaml b/docker-compose.yaml index 44408bdec3625..b8d18220030f9 100644 --- a/docker-compose.yaml +++ b/docker-compose.yaml @@ -194,6 +194,7 @@ services: - TEMPORAL_HISTORY_RETENTION_IN_DAYS=${TEMPORAL_HISTORY_RETENTION_IN_DAYS} - UPDATE_DEFINITIONS_CRON_ENABLED=${UPDATE_DEFINITIONS_CRON_ENABLED} - WORKSPACE_ROOT=${WORKSPACE_ROOT} + - MICRONAUT_ENVIRONMENTS=${CRON_MICRONAUT_ENVIRONMENTS} volumes: - workspace:${WORKSPACE_ROOT} volumes: diff --git a/kube/overlays/dev-integration-test/.env b/kube/overlays/dev-integration-test/.env index 4f3226e4eb0f5..2b3674e89f118 100644 --- a/kube/overlays/dev-integration-test/.env +++ b/kube/overlays/dev-integration-test/.env @@ -60,6 +60,9 @@ NORMALIZATION_JOB_MAIN_CONTAINER_CPU_REQUEST= # Worker # WORKERS_MICRONAUT_ENVIRONMENTS=control-plane +# Cron # +CRON_MICRONAUT_ENVIRONMENTS=control-plane + # Worker pod tolerations, annotations and node selectors JOB_KUBE_TOLERATIONS= JOB_KUBE_ANNOTATIONS= diff --git a/kube/overlays/dev/.env b/kube/overlays/dev/.env index 0ddf0bb2800b3..839afbaa47b1a 100644 --- a/kube/overlays/dev/.env +++ b/kube/overlays/dev/.env @@ -62,6 +62,9 @@ NORMALIZATION_JOB_MAIN_CONTAINER_CPU_REQUEST= # Worker # WORKERS_MICRONAUT_ENVIRONMENTS=control-plane +# Cron # +CRON_MICRONAUT_ENVIRONMENTS=control-plane + # Worker pod tolerations, annotations and node selectors JOB_KUBE_TOLERATIONS= JOB_KUBE_ANNOTATIONS= diff --git a/kube/overlays/stable-with-resource-limits/.env b/kube/overlays/stable-with-resource-limits/.env index e69f951c1c20e..375c93d34d8af 100644 --- a/kube/overlays/stable-with-resource-limits/.env +++ b/kube/overlays/stable-with-resource-limits/.env @@ -62,6 +62,9 @@ NORMALIZATION_JOB_MAIN_CONTAINER_CPU_REQUEST= # Worker # WORKERS_MICRONAUT_ENVIRONMENTS=control-plane +# Cron # +CRON_MICRONAUT_ENVIRONMENTS=control-plane + # Worker pod tolerations, annotations and node selectors JOB_KUBE_TOLERATIONS= JOB_KUBE_ANNOTATIONS= diff --git a/kube/overlays/stable/.env b/kube/overlays/stable/.env index 24ab591f212a4..5daa65701e111 100644 --- a/kube/overlays/stable/.env +++ b/kube/overlays/stable/.env @@ -62,6 +62,9 @@ NORMALIZATION_JOB_MAIN_CONTAINER_CPU_REQUEST= # Worker # WORKERS_MICRONAUT_ENVIRONMENTS=control-plane +# Cron # +CRON_MICRONAUT_ENVIRONMENTS=control-plane + # Worker pod tolerations, annotations and node selectors JOB_KUBE_TOLERATIONS= JOB_KUBE_ANNOTATIONS= diff --git a/kube/resources/cron.yaml b/kube/resources/cron.yaml index 76ba09d37759d..47ce4853d7107 100644 --- a/kube/resources/cron.yaml +++ b/kube/resources/cron.yaml @@ -56,6 +56,11 @@ spec: configMapKeyRef: name: airbyte-env key: WORKSPACE_ROOT + - name: MICRONAUT_ENVIRONMENTS + valueFrom: + configMapKeyRef: + name: airbyte-env + key: CRON_MICRONAUT_ENVIRONMENTS volumeMounts: - name: airbyte-volume-configs mountPath: /configs diff --git a/settings.gradle b/settings.gradle index c0233a62ce0f5..4ba0c20041f87 100644 --- a/settings.gradle +++ b/settings.gradle @@ -84,6 +84,7 @@ include ':airbyte-config:config-persistence' // transitively used by airbyte-wor include ':airbyte-persistence:job-persistence' // transitively used by airbyte-workers. include ':airbyte-db:jooq' // transitively used by airbyte-workers. include ':airbyte-notification' // transitively used by airbyte-workers. +include ':airbyte-worker-models' // platform if (!System.getenv().containsKey("SUB_BUILD") || System.getenv().get("SUB_BUILD") == "PLATFORM") {