From 736ac5764210b2ebf3633fce1c33b57cdd2c8af7 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Sat, 26 Nov 2022 03:40:23 -0400 Subject: [PATCH 01/63] track latest config message --- .../internal/AirbyteMessageTracker.java | 25 ++++++++++++++----- .../internal/DefaultAirbyteSource.java | 3 ++- .../workers/internal/MessageTracker.java | 17 +++++++++++++ 3 files changed, 38 insertions(+), 7 deletions(-) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/AirbyteMessageTracker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/AirbyteMessageTracker.java index f7d8caa4be69..bcd42262697d 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/AirbyteMessageTracker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/AirbyteMessageTracker.java @@ -25,6 +25,7 @@ import io.airbyte.protocol.models.AirbyteStateMessage.AirbyteStateType; import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair; import io.airbyte.protocol.models.AirbyteTraceMessage; +import io.airbyte.protocol.models.Config; import io.airbyte.workers.helper.FailureHelper; import io.airbyte.workers.internal.StateMetricsTracker.StateMetricsTrackerNoStateMatchException; import io.airbyte.workers.internal.state_aggregator.DefaultStateAggregator; @@ -48,6 +49,8 @@ public class AirbyteMessageTracker implements MessageTracker { private final AtomicReference sourceOutputState; private final AtomicReference destinationOutputState; + private final AtomicReference sourceOutputConfig; + private final AtomicReference destinationOutputConfig; private final Map streamToRunningCount; private final HashFunction hashFunction; private final BiMap nameNamespacePairToIndex; @@ -90,6 +93,8 @@ protected AirbyteMessageTracker(final StateDeltaTracker stateDeltaTracker, final StateMetricsTracker stateMetricsTracker) { this.sourceOutputState = new AtomicReference<>(); this.destinationOutputState = new AtomicReference<>(); + this.sourceOutputConfig = new AtomicReference<>(); + this.destinationOutputConfig = new AtomicReference<>(); this.streamToRunningCount = new HashMap<>(); this.nameNamespacePairToIndex = HashBiMap.create(); this.hashFunction = Hashing.murmur3_32_fixed(); @@ -238,12 +243,10 @@ private void handleEmittedOrchestratorMessage(final AirbyteControlMessage contro @SuppressWarnings("PMD") // until method is implemented private void handleEmittedOrchestratorConnectorConfig(final AirbyteControlConnectorConfigMessage configMessage, final ConnectorType connectorType) { - // TODO: Update config here - /** - * Pseudocode: for (key in configMessage.getConfig()) { validateIsReallyConfig(key); - * persistConfigChange(connectorType, key, configMessage.getConfig().get(key)); // nuance here for - * secret storage or not. May need to be async over API for replication orchestrator } - */ + switch (connectorType) { + case SOURCE -> sourceOutputConfig.set(configMessage.getConfig()); + case DESTINATION -> destinationOutputConfig.set(configMessage.getConfig()); + } } /** @@ -343,6 +346,16 @@ public Optional getDestinationOutputState() { return Optional.ofNullable(destinationOutputState.get()); } + @Override + public Optional getSourceOutputConfig() { + return Optional.ofNullable(sourceOutputConfig.get()); + } + + @Override + public Optional getDestinationOutputConfig() { + return Optional.ofNullable(destinationOutputConfig.get()); + } + /** * Fetch committed stream index to record count from the {@link StateDeltaTracker}. Then, swap out * stream indices for stream names. If the delta tracker has exceeded its capacity, return empty diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/DefaultAirbyteSource.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/DefaultAirbyteSource.java index 452fb439ab14..2ebb4a022e33 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/DefaultAirbyteSource.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/DefaultAirbyteSource.java @@ -88,7 +88,8 @@ public void start(final WorkerSourceConfig sourceConfig, final Path jobRoot) thr messageIterator = streamFactory.create(IOs.newBufferedReader(sourceProcess.getInputStream())) .peek(message -> heartbeatMonitor.beat()) - .filter(message -> message.getType() == Type.RECORD || message.getType() == Type.STATE || message.getType() == Type.TRACE) + .filter(message -> message.getType() == Type.RECORD || message.getType() == Type.STATE || message.getType() == Type.TRACE + || message.getType() == Type.CONTROL) .iterator(); } diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/MessageTracker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/MessageTracker.java index 09507ec7a374..b2f3d412c712 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/MessageTracker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/MessageTracker.java @@ -9,6 +9,7 @@ import io.airbyte.protocol.models.AirbyteMessage; import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair; import io.airbyte.protocol.models.AirbyteTraceMessage; +import io.airbyte.protocol.models.Config; import java.util.Map; import java.util.Optional; @@ -50,6 +51,22 @@ public interface MessageTracker { */ Optional getDestinationOutputState(); + /** + * Get the current source config. + * + * @return returns the last Config that was accepted from the source via a control message. If no + * Config was accepted, empty. + */ + Optional getSourceOutputConfig(); + + /** + * Get the current destination config. + * + * @return returns the last Config that was accepted from the destination via a control message. If + * no StateMessage was accepted, empty. + */ + Optional getDestinationOutputConfig(); + /** * Get the per-stream committed record count. * From 715c17070061f67c6b2a9a95d066ae193719988f Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Sat, 26 Nov 2022 03:41:22 -0400 Subject: [PATCH 02/63] pass new config as part of outputs --- .../general/DefaultReplicationWorker.java | 44 +++++++++++++++++++ .../resources/types/ReplicationOutput.yaml | 4 ++ .../resources/types/StandardSyncOutput.yaml | 4 ++ .../sync/ReplicationActivityImpl.java | 3 ++ 4 files changed, 55 insertions(+) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java index 36f05be67751..f4f9948febb6 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java @@ -28,6 +28,7 @@ import io.airbyte.protocol.models.AirbyteMessage.Type; import io.airbyte.protocol.models.AirbyteRecordMessage; import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair; +import io.airbyte.protocol.models.Config; import io.airbyte.workers.RecordSchemaValidator; import io.airbyte.workers.WorkerMetricReporter; import io.airbyte.workers.WorkerUtils; @@ -82,6 +83,27 @@ public class DefaultReplicationWorker implements ReplicationWorker { private static final Logger LOGGER = LoggerFactory.getLogger(DefaultReplicationWorker.class); + // private static final Configs CONFIGS = new EnvConfigs(); + // private static final AirbyteApiClient CLIENT = getAirbyteApiClient(); + + // Passing env vars to the container orchestrator isn't working properly. Hack around this for now. + // TODO(Davin): This doesn't work for Kube. Need to figure it out. + // private static AirbyteApiClient getAirbyteApiClient() { + // if (CONFIGS.getWorkerEnvironment() == WorkerEnvironment.DOCKER) { + // return new AirbyteApiClient( + // new ApiClient().setScheme("http") + // .setHost(CONFIGS.getAirbyteApiHost()) + // .setPort(CONFIGS.getAirbyteApiPort()) + // .setBasePath("/api")); + // } + // + // return new AirbyteApiClient( + // new ApiClient().setScheme("http") + // .setHost("airbyte-server-svc") + // .setPort(8001) + // .setBasePath("/api")); + // } + private final String jobId; private final int attempt; private final AirbyteSource source; @@ -276,6 +298,15 @@ private static Runnable readFromDstRunnable(final AirbyteDestination destination }; } + // private static void saveConfig(final AirbyteSource source, final Config config) throws + // ApiException { + // var jobsApi = CLIENT.getJobsApi(); + // final JobIdRequestBody body = new JobIdRequestBody().id(1L); + // final JobInfoRead jobInfo = jobsApi.getJobInfo(body); + // jobsApi.getJobDebugInfo(body). + //// CLIENT.getSourceApi().updateSource(); + // } + @SuppressWarnings("PMD.AvoidInstanceofChecksInCatchClause") private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource source, final AirbyteDestination destination, @@ -404,6 +435,19 @@ else if (hasFailed.get()) { prepStateForLaterSaving(syncInput, output); + // pass along new configs we may have received + if (messageTracker.getDestinationOutputConfig().isPresent()) { + LOGGER.info("Captured updated source config"); + final Config config = messageTracker.getDestinationOutputConfig().get(); + output.withDestinationConfig(config); + } + + if (messageTracker.getSourceOutputConfig().isPresent()) { + LOGGER.info("Captured updated source config"); + final Config config = messageTracker.getSourceOutputConfig().get(); + output.withSourceConfig(config); + } + final ObjectMapper mapper = new ObjectMapper(); LOGGER.info("sync summary: {}", mapper.writerWithDefaultPrettyPrinter().writeValueAsString(summary)); LOGGER.info("failures: {}", mapper.writerWithDefaultPrettyPrinter().writeValueAsString(failures)); diff --git a/airbyte-config/config-models/src/main/resources/types/ReplicationOutput.yaml b/airbyte-config/config-models/src/main/resources/types/ReplicationOutput.yaml index 0182c0259405..9186089964f5 100644 --- a/airbyte-config/config-models/src/main/resources/types/ReplicationOutput.yaml +++ b/airbyte-config/config-models/src/main/resources/types/ReplicationOutput.yaml @@ -14,6 +14,10 @@ properties: "$ref": ReplicationAttemptSummary.yaml state: "$ref": State.yaml + sourceConfig: + existingJavaType: io.airbyte.protocol.models.Config + destinationConfig: + existingJavaType: io.airbyte.protocol.models.Config output_catalog: existingJavaType: io.airbyte.protocol.models.ConfiguredAirbyteCatalog failures: diff --git a/airbyte-config/config-models/src/main/resources/types/StandardSyncOutput.yaml b/airbyte-config/config-models/src/main/resources/types/StandardSyncOutput.yaml index ba0a14a340fd..ed8c014ee1a7 100644 --- a/airbyte-config/config-models/src/main/resources/types/StandardSyncOutput.yaml +++ b/airbyte-config/config-models/src/main/resources/types/StandardSyncOutput.yaml @@ -18,6 +18,10 @@ properties: "$ref": WebhookOperationSummary.yaml state: "$ref": State.yaml + sourceConfig: + existingJavaType: io.airbyte.protocol.models.Config + destinationConfig: + existingJavaType: io.airbyte.protocol.models.Config output_catalog: existingJavaType: io.airbyte.protocol.models.ConfiguredAirbyteCatalog failures: diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java index b7104497e71a..4e47e65fa2ad 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java @@ -228,6 +228,9 @@ private static StandardSyncOutput reduceReplicationOutput(final ReplicationOutpu standardSyncOutput.setStandardSyncSummary(syncSummary); standardSyncOutput.setFailures(output.getFailures()); + standardSyncOutput.setDestinationConfig(output.getDestinationConfig()); + standardSyncOutput.setSourceConfig(output.getSourceConfig()); + return standardSyncOutput; } From 2df503976dc2a10b6adc156b813edc29b94e181f Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Sat, 26 Nov 2022 03:42:02 -0400 Subject: [PATCH 03/63] persist new config --- .../sync/PersistStateActivityImpl.java | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/PersistStateActivityImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/PersistStateActivityImpl.java index f9ed3057066c..ded55ccca569 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/PersistStateActivityImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/PersistStateActivityImpl.java @@ -14,9 +14,14 @@ import io.airbyte.api.client.AirbyteApiClient; import io.airbyte.api.client.invoker.generated.ApiException; import io.airbyte.api.client.model.generated.ConnectionIdRequestBody; +import io.airbyte.api.client.model.generated.ConnectionRead; import io.airbyte.api.client.model.generated.ConnectionState; import io.airbyte.api.client.model.generated.ConnectionStateCreateOrUpdate; +import io.airbyte.api.client.model.generated.SourceIdRequestBody; +import io.airbyte.api.client.model.generated.SourceRead; +import io.airbyte.api.client.model.generated.SourceUpdate; import io.airbyte.commons.features.FeatureFlags; +import io.airbyte.commons.json.Jsons; import io.airbyte.config.StandardSyncOutput; import io.airbyte.config.State; import io.airbyte.config.StateType; @@ -24,6 +29,7 @@ import io.airbyte.config.helpers.StateMessageHelper; import io.airbyte.metrics.lib.ApmTraceUtils; import io.airbyte.protocol.models.CatalogHelpers; +import io.airbyte.protocol.models.Config; import io.airbyte.protocol.models.ConfiguredAirbyteCatalog; import io.airbyte.protocol.models.StreamDescriptor; import io.airbyte.workers.helper.StateConverter; @@ -44,10 +50,36 @@ public PersistStateActivityImpl(final AirbyteApiClient airbyteApiClient, final F this.featureFlags = featureFlags; } + public void persistConfig(final UUID connectionId, final StandardSyncOutput syncOutput) throws ApiException { + // TODO move this elsewhere, just doing this here for quick iteration + final Config sourceConfig = syncOutput.getSourceConfig(); + if (sourceConfig == null) + return; + + final ConnectionRead connection = airbyteApiClient.getConnectionApi().getConnection(new ConnectionIdRequestBody().connectionId(connectionId)); + final UUID sourceId = connection.getSourceId(); + + final SourceRead source = airbyteApiClient.getSourceApi().getSource(new SourceIdRequestBody().sourceId(sourceId)); + + airbyteApiClient.getSourceApi() + .updateSource(new SourceUpdate() + .sourceId(sourceId) + .name(source.getName()) + .connectionConfiguration(Jsons.jsonNode(sourceConfig.getAdditionalProperties()))); + + } + @Trace(operationName = ACTIVITY_TRACE_OPERATION_NAME) @Override public boolean persist(final UUID connectionId, final StandardSyncOutput syncOutput, final ConfiguredAirbyteCatalog configuredCatalog) { ApmTraceUtils.addTagsToTrace(Map.of(CONNECTION_ID_KEY, connectionId.toString())); + + try { + persistConfig(connectionId, syncOutput); + } catch (final ApiException e) { + throw new RuntimeException(e); + } + final State state = syncOutput.getState(); if (state != null) { // todo: these validation logic should happen on server side. From 59122d768d290e7c4bc2aa6e5ec3b6e001c7458d Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Mon, 28 Nov 2022 17:26:57 -0400 Subject: [PATCH 04/63] persist config as the messages come through, dont set output --- .../general/DefaultReplicationWorker.java | 81 +++++-------- .../workers/helper/PersistConfigHelper.java | 112 ++++++++++++++++++ 2 files changed, 143 insertions(+), 50 deletions(-) create mode 100644 airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/PersistConfigHelper.java diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java index f4f9948febb6..51dfdb5104d2 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java @@ -12,6 +12,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import datadog.trace.api.Trace; +import io.airbyte.api.client.invoker.generated.ApiException; import io.airbyte.commons.io.LineGobbler; import io.airbyte.config.FailureReason; import io.airbyte.config.ReplicationAttemptSummary; @@ -24,17 +25,18 @@ import io.airbyte.config.WorkerDestinationConfig; import io.airbyte.config.WorkerSourceConfig; import io.airbyte.metrics.lib.ApmTraceUtils; +import io.airbyte.protocol.models.AirbyteControlMessage; import io.airbyte.protocol.models.AirbyteMessage; import io.airbyte.protocol.models.AirbyteMessage.Type; import io.airbyte.protocol.models.AirbyteRecordMessage; import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair; -import io.airbyte.protocol.models.Config; import io.airbyte.workers.RecordSchemaValidator; import io.airbyte.workers.WorkerMetricReporter; import io.airbyte.workers.WorkerUtils; import io.airbyte.workers.exception.RecordSchemaValidationException; import io.airbyte.workers.exception.WorkerException; import io.airbyte.workers.helper.FailureHelper; +import io.airbyte.workers.helper.PersistConfigHelper; import io.airbyte.workers.helper.ThreadedTimeTracker; import io.airbyte.workers.internal.AirbyteDestination; import io.airbyte.workers.internal.AirbyteMapper; @@ -83,27 +85,6 @@ public class DefaultReplicationWorker implements ReplicationWorker { private static final Logger LOGGER = LoggerFactory.getLogger(DefaultReplicationWorker.class); - // private static final Configs CONFIGS = new EnvConfigs(); - // private static final AirbyteApiClient CLIENT = getAirbyteApiClient(); - - // Passing env vars to the container orchestrator isn't working properly. Hack around this for now. - // TODO(Davin): This doesn't work for Kube. Need to figure it out. - // private static AirbyteApiClient getAirbyteApiClient() { - // if (CONFIGS.getWorkerEnvironment() == WorkerEnvironment.DOCKER) { - // return new AirbyteApiClient( - // new ApiClient().setScheme("http") - // .setHost(CONFIGS.getAirbyteApiHost()) - // .setPort(CONFIGS.getAirbyteApiPort()) - // .setBasePath("/api")); - // } - // - // return new AirbyteApiClient( - // new ApiClient().setScheme("http") - // .setHost("airbyte-server-svc") - // .setPort(8001) - // .setBasePath("/api")); - // } - private final String jobId; private final int attempt; private final AirbyteSource source; @@ -206,7 +187,7 @@ private void replicate(final Path jobRoot, // note: `whenComplete` is used instead of `exceptionally` so that the original exception is still // thrown final CompletableFuture readFromDstThread = CompletableFuture.runAsync( - readFromDstRunnable(destination, cancelled, messageTracker, mdc, timeTracker), + readFromDstRunnable(destination, cancelled, messageTracker, mdc, timeTracker, Long.valueOf(jobId)), executors) .whenComplete((msg, ex) -> { if (ex != null) { @@ -221,7 +202,7 @@ private void replicate(final Path jobRoot, final CompletableFuture readSrcAndWriteDstThread = CompletableFuture.runAsync( readFromSrcAndWriteToDstRunnable(source, destination, cancelled, mapper, messageTracker, mdc, recordSchemaValidator, metricReporter, - timeTracker), + timeTracker, Long.valueOf(jobId)), executors) .whenComplete((msg, ex) -> { if (ex != null) { @@ -259,7 +240,8 @@ private static Runnable readFromDstRunnable(final AirbyteDestination destination final AtomicBoolean cancelled, final MessageTracker messageTracker, final Map mdc, - final ThreadedTimeTracker timeHolder) { + final ThreadedTimeTracker timeHolder, + final Long jobId) { return () -> { MDC.setContextMap(mdc); LOGGER.info("Destination output thread started."); @@ -272,8 +254,19 @@ private static Runnable readFromDstRunnable(final AirbyteDestination destination throw new DestinationException("Destination process read attempt failed", e); } if (messageOptional.isPresent()) { - LOGGER.info("State in DefaultReplicationWorker from destination: {}", messageOptional.get()); - messageTracker.acceptFromDestination(messageOptional.get()); + final AirbyteMessage message = messageOptional.get(); + LOGGER.info("State in DefaultReplicationWorker from destination: {}", message); + messageTracker.acceptFromDestination(message); + + try { + if (message.getType() == Type.CONTROL && message.getControl().getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG) { + PersistConfigHelper.persistDestinationConfig(jobId, message.getControl().getConnectorConfig().getConfig()); + } + } catch (final ApiException e) { + // TODO this should probably throw rather than just log and continue + LOGGER.error("Error trying to save updated destination config", e); + } + } } timeHolder.trackDestinationWriteEndTime(); @@ -298,15 +291,6 @@ private static Runnable readFromDstRunnable(final AirbyteDestination destination }; } - // private static void saveConfig(final AirbyteSource source, final Config config) throws - // ApiException { - // var jobsApi = CLIENT.getJobsApi(); - // final JobIdRequestBody body = new JobIdRequestBody().id(1L); - // final JobInfoRead jobInfo = jobsApi.getJobInfo(body); - // jobsApi.getJobDebugInfo(body). - //// CLIENT.getSourceApi().updateSource(); - // } - @SuppressWarnings("PMD.AvoidInstanceofChecksInCatchClause") private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource source, final AirbyteDestination destination, @@ -316,7 +300,8 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou final Map mdc, final RecordSchemaValidator recordSchemaValidator, final WorkerMetricReporter metricReporter, - final ThreadedTimeTracker timeHolder) { + final ThreadedTimeTracker timeHolder, + final Long jobId) { return () -> { MDC.setContextMap(mdc); LOGGER.info("Replication thread started."); @@ -338,6 +323,15 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou messageTracker.acceptFromSource(message); + if (message.getType() == Type.CONTROL && message.getControl().getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG) { + try { + PersistConfigHelper.persistSourceConfig(jobId, message.getControl().getConnectorConfig().getConfig()); + } catch (final ApiException e) { + // TODO this should probably throw rather than just log and continue + LOGGER.error("Error trying to save updated source config", e); + } + } + try { if (message.getType() == Type.RECORD || message.getType() == Type.STATE) { destination.accept(message); @@ -435,19 +429,6 @@ else if (hasFailed.get()) { prepStateForLaterSaving(syncInput, output); - // pass along new configs we may have received - if (messageTracker.getDestinationOutputConfig().isPresent()) { - LOGGER.info("Captured updated source config"); - final Config config = messageTracker.getDestinationOutputConfig().get(); - output.withDestinationConfig(config); - } - - if (messageTracker.getSourceOutputConfig().isPresent()) { - LOGGER.info("Captured updated source config"); - final Config config = messageTracker.getSourceOutputConfig().get(); - output.withSourceConfig(config); - } - final ObjectMapper mapper = new ObjectMapper(); LOGGER.info("sync summary: {}", mapper.writerWithDefaultPrettyPrinter().writeValueAsString(summary)); LOGGER.info("failures: {}", mapper.writerWithDefaultPrettyPrinter().writeValueAsString(failures)); diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/PersistConfigHelper.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/PersistConfigHelper.java new file mode 100644 index 000000000000..4a04bd7c8069 --- /dev/null +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/PersistConfigHelper.java @@ -0,0 +1,112 @@ +/* + * Copyright (c) 2022 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.workers.helper; + +import com.google.common.hash.Hashing; +import io.airbyte.api.client.AirbyteApiClient; +import io.airbyte.api.client.generated.DestinationApi; +import io.airbyte.api.client.generated.JobsApi; +import io.airbyte.api.client.generated.SourceApi; +import io.airbyte.api.client.invoker.generated.ApiClient; +import io.airbyte.api.client.invoker.generated.ApiException; +import io.airbyte.api.client.model.generated.ConnectionIdRequestBody; +import io.airbyte.api.client.model.generated.ConnectionRead; +import io.airbyte.api.client.model.generated.DestinationIdRequestBody; +import io.airbyte.api.client.model.generated.DestinationRead; +import io.airbyte.api.client.model.generated.DestinationUpdate; +import io.airbyte.api.client.model.generated.JobIdRequestBody; +import io.airbyte.api.client.model.generated.JobInfoLightRead; +import io.airbyte.api.client.model.generated.SourceIdRequestBody; +import io.airbyte.api.client.model.generated.SourceRead; +import io.airbyte.api.client.model.generated.SourceUpdate; +import io.airbyte.commons.json.Jsons; +import io.airbyte.config.Configs; +import io.airbyte.config.Configs.WorkerEnvironment; +import io.airbyte.config.EnvConfigs; +import io.airbyte.protocol.models.Config; +import java.nio.charset.StandardCharsets; +import java.util.UUID; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Helper class for workers to persist updates to Source/Destination configs emitted from + * AirbyteControlMessages. + */ +public class PersistConfigHelper { + + private static final Logger LOGGER = LoggerFactory.getLogger(PersistConfigHelper.class); + + private static final Configs CONFIGS = new EnvConfigs(); + private static final AirbyteApiClient CLIENT = getAirbyteApiClient(); + + // TODO inject this with micronaut, possibly making this class a Singleton + private static AirbyteApiClient getAirbyteApiClient() { + if (CONFIGS.getWorkerEnvironment() == WorkerEnvironment.DOCKER) { + return new AirbyteApiClient( + new ApiClient().setScheme("http") + .setHost(CONFIGS.getAirbyteApiHost()) + .setPort(CONFIGS.getAirbyteApiPort()) + .setBasePath("/api")); + } + + return new AirbyteApiClient( + new ApiClient().setScheme("http") + .setHost("airbyte-server-svc") + .setPort(8001) + .setBasePath("/api")); + } + + private static UUID getConnectionIdFromJobId(final Long jobId) throws ApiException { + final JobsApi jobsApi = CLIENT.getJobsApi(); + final JobIdRequestBody body = new JobIdRequestBody().id(jobId); + final JobInfoLightRead jobInfo = jobsApi.getJobInfoLight(body); + return UUID.fromString(jobInfo.getJob().getConfigId()); + } + + public static void persistSourceConfig(final Long jobId, final Config config) throws ApiException { + final UUID connectionId = getConnectionIdFromJobId(jobId); + + final ConnectionRead connection = CLIENT.getConnectionApi().getConnection(new ConnectionIdRequestBody().connectionId(connectionId)); + final UUID sourceId = connection.getSourceId(); + + final SourceApi sourceApi = CLIENT.getSourceApi(); + final SourceRead source = sourceApi.getSource(new SourceIdRequestBody().sourceId(sourceId)); + + // TODO might need to strip out OAuth params + + final SourceRead updatedSource = sourceApi + .updateSource(new SourceUpdate() + .sourceId(sourceId) + .name(source.getName()) + .connectionConfiguration(Jsons.jsonNode(config.getAdditionalProperties()))); + + LOGGER.info("Persisted updated configuration for source {}. New config hash: {}.", sourceId, + Hashing.sha256().hashString(updatedSource.getConnectionConfiguration().asText(), StandardCharsets.UTF_8)); + + } + + public static void persistDestinationConfig(final Long jobId, final Config config) throws ApiException { + final UUID connectionId = getConnectionIdFromJobId(jobId); + + final ConnectionRead connection = CLIENT.getConnectionApi().getConnection(new ConnectionIdRequestBody().connectionId(connectionId)); + final UUID destinationId = connection.getDestinationId(); + + final DestinationApi destinationApi = CLIENT.getDestinationApi(); + final DestinationRead destination = destinationApi.getDestination(new DestinationIdRequestBody().destinationId(destinationId)); + + // TODO might need to strip out OAuth params + + final DestinationRead updatedDestination = destinationApi + .updateDestination(new DestinationUpdate() + .destinationId(destinationId) + .name(destination.getName()) + .connectionConfiguration(Jsons.jsonNode(config.getAdditionalProperties()))); + + LOGGER.info("Persisted updated configuration for destination {}. New config hash: {}.", destinationId, + Hashing.sha256().hashString(updatedDestination.getConnectionConfiguration().asText(), StandardCharsets.UTF_8)); + } + +} From 7f5a4e756f5f21bc02c3a2e9abb1504847b1475b Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Mon, 28 Nov 2022 17:27:35 -0400 Subject: [PATCH 05/63] clean up old implementation --- .../internal/AirbyteMessageTracker.java | 21 ++----------- .../workers/internal/MessageTracker.java | 17 ---------- .../resources/types/ReplicationOutput.yaml | 4 --- .../resources/types/StandardSyncOutput.yaml | 4 --- .../sync/PersistStateActivityImpl.java | 31 ------------------- .../sync/ReplicationActivityImpl.java | 3 -- 6 files changed, 2 insertions(+), 78 deletions(-) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/AirbyteMessageTracker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/AirbyteMessageTracker.java index bcd42262697d..6c9e865612a9 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/AirbyteMessageTracker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/AirbyteMessageTracker.java @@ -25,7 +25,6 @@ import io.airbyte.protocol.models.AirbyteStateMessage.AirbyteStateType; import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair; import io.airbyte.protocol.models.AirbyteTraceMessage; -import io.airbyte.protocol.models.Config; import io.airbyte.workers.helper.FailureHelper; import io.airbyte.workers.internal.StateMetricsTracker.StateMetricsTrackerNoStateMatchException; import io.airbyte.workers.internal.state_aggregator.DefaultStateAggregator; @@ -49,8 +48,6 @@ public class AirbyteMessageTracker implements MessageTracker { private final AtomicReference sourceOutputState; private final AtomicReference destinationOutputState; - private final AtomicReference sourceOutputConfig; - private final AtomicReference destinationOutputConfig; private final Map streamToRunningCount; private final HashFunction hashFunction; private final BiMap nameNamespacePairToIndex; @@ -93,8 +90,6 @@ protected AirbyteMessageTracker(final StateDeltaTracker stateDeltaTracker, final StateMetricsTracker stateMetricsTracker) { this.sourceOutputState = new AtomicReference<>(); this.destinationOutputState = new AtomicReference<>(); - this.sourceOutputConfig = new AtomicReference<>(); - this.destinationOutputConfig = new AtomicReference<>(); this.streamToRunningCount = new HashMap<>(); this.nameNamespacePairToIndex = HashBiMap.create(); this.hashFunction = Hashing.murmur3_32_fixed(); @@ -243,10 +238,8 @@ private void handleEmittedOrchestratorMessage(final AirbyteControlMessage contro @SuppressWarnings("PMD") // until method is implemented private void handleEmittedOrchestratorConnectorConfig(final AirbyteControlConnectorConfigMessage configMessage, final ConnectorType connectorType) { - switch (connectorType) { - case SOURCE -> sourceOutputConfig.set(configMessage.getConfig()); - case DESTINATION -> destinationOutputConfig.set(configMessage.getConfig()); - } + // Currently being persisted as part of the DefaultReplicationWorker + // TODO decide if it should be persisted from here instead } /** @@ -346,16 +339,6 @@ public Optional getDestinationOutputState() { return Optional.ofNullable(destinationOutputState.get()); } - @Override - public Optional getSourceOutputConfig() { - return Optional.ofNullable(sourceOutputConfig.get()); - } - - @Override - public Optional getDestinationOutputConfig() { - return Optional.ofNullable(destinationOutputConfig.get()); - } - /** * Fetch committed stream index to record count from the {@link StateDeltaTracker}. Then, swap out * stream indices for stream names. If the delta tracker has exceeded its capacity, return empty diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/MessageTracker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/MessageTracker.java index b2f3d412c712..09507ec7a374 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/MessageTracker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/MessageTracker.java @@ -9,7 +9,6 @@ import io.airbyte.protocol.models.AirbyteMessage; import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair; import io.airbyte.protocol.models.AirbyteTraceMessage; -import io.airbyte.protocol.models.Config; import java.util.Map; import java.util.Optional; @@ -51,22 +50,6 @@ public interface MessageTracker { */ Optional getDestinationOutputState(); - /** - * Get the current source config. - * - * @return returns the last Config that was accepted from the source via a control message. If no - * Config was accepted, empty. - */ - Optional getSourceOutputConfig(); - - /** - * Get the current destination config. - * - * @return returns the last Config that was accepted from the destination via a control message. If - * no StateMessage was accepted, empty. - */ - Optional getDestinationOutputConfig(); - /** * Get the per-stream committed record count. * diff --git a/airbyte-config/config-models/src/main/resources/types/ReplicationOutput.yaml b/airbyte-config/config-models/src/main/resources/types/ReplicationOutput.yaml index 9186089964f5..0182c0259405 100644 --- a/airbyte-config/config-models/src/main/resources/types/ReplicationOutput.yaml +++ b/airbyte-config/config-models/src/main/resources/types/ReplicationOutput.yaml @@ -14,10 +14,6 @@ properties: "$ref": ReplicationAttemptSummary.yaml state: "$ref": State.yaml - sourceConfig: - existingJavaType: io.airbyte.protocol.models.Config - destinationConfig: - existingJavaType: io.airbyte.protocol.models.Config output_catalog: existingJavaType: io.airbyte.protocol.models.ConfiguredAirbyteCatalog failures: diff --git a/airbyte-config/config-models/src/main/resources/types/StandardSyncOutput.yaml b/airbyte-config/config-models/src/main/resources/types/StandardSyncOutput.yaml index ed8c014ee1a7..ba0a14a340fd 100644 --- a/airbyte-config/config-models/src/main/resources/types/StandardSyncOutput.yaml +++ b/airbyte-config/config-models/src/main/resources/types/StandardSyncOutput.yaml @@ -18,10 +18,6 @@ properties: "$ref": WebhookOperationSummary.yaml state: "$ref": State.yaml - sourceConfig: - existingJavaType: io.airbyte.protocol.models.Config - destinationConfig: - existingJavaType: io.airbyte.protocol.models.Config output_catalog: existingJavaType: io.airbyte.protocol.models.ConfiguredAirbyteCatalog failures: diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/PersistStateActivityImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/PersistStateActivityImpl.java index ded55ccca569..00d3959aad65 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/PersistStateActivityImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/PersistStateActivityImpl.java @@ -14,14 +14,9 @@ import io.airbyte.api.client.AirbyteApiClient; import io.airbyte.api.client.invoker.generated.ApiException; import io.airbyte.api.client.model.generated.ConnectionIdRequestBody; -import io.airbyte.api.client.model.generated.ConnectionRead; import io.airbyte.api.client.model.generated.ConnectionState; import io.airbyte.api.client.model.generated.ConnectionStateCreateOrUpdate; -import io.airbyte.api.client.model.generated.SourceIdRequestBody; -import io.airbyte.api.client.model.generated.SourceRead; -import io.airbyte.api.client.model.generated.SourceUpdate; import io.airbyte.commons.features.FeatureFlags; -import io.airbyte.commons.json.Jsons; import io.airbyte.config.StandardSyncOutput; import io.airbyte.config.State; import io.airbyte.config.StateType; @@ -29,7 +24,6 @@ import io.airbyte.config.helpers.StateMessageHelper; import io.airbyte.metrics.lib.ApmTraceUtils; import io.airbyte.protocol.models.CatalogHelpers; -import io.airbyte.protocol.models.Config; import io.airbyte.protocol.models.ConfiguredAirbyteCatalog; import io.airbyte.protocol.models.StreamDescriptor; import io.airbyte.workers.helper.StateConverter; @@ -50,36 +44,11 @@ public PersistStateActivityImpl(final AirbyteApiClient airbyteApiClient, final F this.featureFlags = featureFlags; } - public void persistConfig(final UUID connectionId, final StandardSyncOutput syncOutput) throws ApiException { - // TODO move this elsewhere, just doing this here for quick iteration - final Config sourceConfig = syncOutput.getSourceConfig(); - if (sourceConfig == null) - return; - - final ConnectionRead connection = airbyteApiClient.getConnectionApi().getConnection(new ConnectionIdRequestBody().connectionId(connectionId)); - final UUID sourceId = connection.getSourceId(); - - final SourceRead source = airbyteApiClient.getSourceApi().getSource(new SourceIdRequestBody().sourceId(sourceId)); - - airbyteApiClient.getSourceApi() - .updateSource(new SourceUpdate() - .sourceId(sourceId) - .name(source.getName()) - .connectionConfiguration(Jsons.jsonNode(sourceConfig.getAdditionalProperties()))); - - } - @Trace(operationName = ACTIVITY_TRACE_OPERATION_NAME) @Override public boolean persist(final UUID connectionId, final StandardSyncOutput syncOutput, final ConfiguredAirbyteCatalog configuredCatalog) { ApmTraceUtils.addTagsToTrace(Map.of(CONNECTION_ID_KEY, connectionId.toString())); - try { - persistConfig(connectionId, syncOutput); - } catch (final ApiException e) { - throw new RuntimeException(e); - } - final State state = syncOutput.getState(); if (state != null) { // todo: these validation logic should happen on server side. diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java index 4e47e65fa2ad..b7104497e71a 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java @@ -228,9 +228,6 @@ private static StandardSyncOutput reduceReplicationOutput(final ReplicationOutpu standardSyncOutput.setStandardSyncSummary(syncSummary); standardSyncOutput.setFailures(output.getFailures()); - standardSyncOutput.setDestinationConfig(output.getDestinationConfig()); - standardSyncOutput.setSourceConfig(output.getSourceConfig()); - return standardSyncOutput; } From 2637e62aec20943473bb0d7c57b25ea884214343 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Mon, 28 Nov 2022 17:27:57 -0400 Subject: [PATCH 06/63] accept control messages for destinations --- .../airbyte/workers/internal/DefaultAirbyteDestination.java | 4 +++- .../io/airbyte/workers/internal/DefaultAirbyteSource.java | 5 +++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/DefaultAirbyteDestination.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/DefaultAirbyteDestination.java index d776dfba1a95..96900bf05b64 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/DefaultAirbyteDestination.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/DefaultAirbyteDestination.java @@ -27,6 +27,7 @@ import java.io.OutputStreamWriter; import java.nio.file.Path; import java.util.Iterator; +import java.util.List; import java.util.Optional; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -81,8 +82,9 @@ public void start(final WorkerDestinationConfig destinationConfig, final Path jo writer = messageWriterFactory.createWriter(new BufferedWriter(new OutputStreamWriter(destinationProcess.getOutputStream(), Charsets.UTF_8))); + final List acceptedMessageTypes = List.of(Type.STATE, Type.TRACE, Type.CONTROL); messageIterator = streamFactory.create(IOs.newBufferedReader(destinationProcess.getInputStream())) - .filter(message -> message.getType() == Type.STATE || message.getType() == Type.TRACE) + .filter(message -> acceptedMessageTypes.contains(message.getType())) .iterator(); } diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/DefaultAirbyteSource.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/DefaultAirbyteSource.java index 2ebb4a022e33..ff8a0f8dfda8 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/DefaultAirbyteSource.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/DefaultAirbyteSource.java @@ -27,6 +27,7 @@ import java.time.Duration; import java.time.temporal.ChronoUnit; import java.util.Iterator; +import java.util.List; import java.util.Optional; import java.util.concurrent.TimeUnit; import org.slf4j.Logger; @@ -86,10 +87,10 @@ public void start(final WorkerSourceConfig sourceConfig, final Path jobRoot) thr logInitialStateAsJSON(sourceConfig); + final List acceptedMessageTypes = List.of(Type.RECORD, Type.STATE, Type.TRACE, Type.CONTROL); messageIterator = streamFactory.create(IOs.newBufferedReader(sourceProcess.getInputStream())) .peek(message -> heartbeatMonitor.beat()) - .filter(message -> message.getType() == Type.RECORD || message.getType() == Type.STATE || message.getType() == Type.TRACE - || message.getType() == Type.CONTROL) + .filter(message -> acceptedMessageTypes.contains(message.getType())) .iterator(); } From 289672d47fb5e1e9aecf78bfc7550705ee90eeef Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Mon, 28 Nov 2022 18:19:57 -0400 Subject: [PATCH 07/63] get api client from micronaut --- .../general/DefaultReplicationWorker.java | 16 ++++--- .../workers/helper/PersistConfigHelper.java | 42 ++++++------------- .../ReplicationJobOrchestrator.java | 15 ++++++- .../sync/ReplicationActivityImpl.java | 4 +- 4 files changed, 40 insertions(+), 37 deletions(-) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java index 51dfdb5104d2..9eb1170a5103 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java @@ -97,6 +97,7 @@ public class DefaultReplicationWorker implements ReplicationWorker { private final AtomicBoolean hasFailed; private final RecordSchemaValidator recordSchemaValidator; private final WorkerMetricReporter metricReporter; + private final PersistConfigHelper persistConfigHelper; public DefaultReplicationWorker(final String jobId, final int attempt, @@ -105,7 +106,8 @@ public DefaultReplicationWorker(final String jobId, final AirbyteDestination destination, final MessageTracker messageTracker, final RecordSchemaValidator recordSchemaValidator, - final WorkerMetricReporter metricReporter) { + final WorkerMetricReporter metricReporter, + final PersistConfigHelper persistConfigHelper) { this.jobId = jobId; this.attempt = attempt; this.source = source; @@ -115,6 +117,7 @@ public DefaultReplicationWorker(final String jobId, this.executors = Executors.newFixedThreadPool(2); this.recordSchemaValidator = recordSchemaValidator; this.metricReporter = metricReporter; + this.persistConfigHelper = persistConfigHelper; this.cancelled = new AtomicBoolean(false); this.hasFailed = new AtomicBoolean(false); @@ -187,7 +190,7 @@ private void replicate(final Path jobRoot, // note: `whenComplete` is used instead of `exceptionally` so that the original exception is still // thrown final CompletableFuture readFromDstThread = CompletableFuture.runAsync( - readFromDstRunnable(destination, cancelled, messageTracker, mdc, timeTracker, Long.valueOf(jobId)), + readFromDstRunnable(destination, cancelled, messageTracker, persistConfigHelper, mdc, timeTracker, Long.valueOf(jobId)), executors) .whenComplete((msg, ex) -> { if (ex != null) { @@ -201,7 +204,8 @@ private void replicate(final Path jobRoot, }); final CompletableFuture readSrcAndWriteDstThread = CompletableFuture.runAsync( - readFromSrcAndWriteToDstRunnable(source, destination, cancelled, mapper, messageTracker, mdc, recordSchemaValidator, metricReporter, + readFromSrcAndWriteToDstRunnable(source, destination, cancelled, mapper, messageTracker, persistConfigHelper, mdc, recordSchemaValidator, + metricReporter, timeTracker, Long.valueOf(jobId)), executors) .whenComplete((msg, ex) -> { @@ -239,6 +243,7 @@ private void replicate(final Path jobRoot, private static Runnable readFromDstRunnable(final AirbyteDestination destination, final AtomicBoolean cancelled, final MessageTracker messageTracker, + final PersistConfigHelper persistConfigHelper, final Map mdc, final ThreadedTimeTracker timeHolder, final Long jobId) { @@ -260,7 +265,7 @@ private static Runnable readFromDstRunnable(final AirbyteDestination destination try { if (message.getType() == Type.CONTROL && message.getControl().getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG) { - PersistConfigHelper.persistDestinationConfig(jobId, message.getControl().getConnectorConfig().getConfig()); + persistConfigHelper.persistDestinationConfig(jobId, message.getControl().getConnectorConfig().getConfig()); } } catch (final ApiException e) { // TODO this should probably throw rather than just log and continue @@ -297,6 +302,7 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou final AtomicBoolean cancelled, final AirbyteMapper mapper, final MessageTracker messageTracker, + final PersistConfigHelper persistConfigHelper, final Map mdc, final RecordSchemaValidator recordSchemaValidator, final WorkerMetricReporter metricReporter, @@ -325,7 +331,7 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou if (message.getType() == Type.CONTROL && message.getControl().getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG) { try { - PersistConfigHelper.persistSourceConfig(jobId, message.getControl().getConnectorConfig().getConfig()); + persistConfigHelper.persistSourceConfig(jobId, message.getControl().getConnectorConfig().getConfig()); } catch (final ApiException e) { // TODO this should probably throw rather than just log and continue LOGGER.error("Error trying to save updated source config", e); diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/PersistConfigHelper.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/PersistConfigHelper.java index 4a04bd7c8069..57bb99e4a5bf 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/PersistConfigHelper.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/PersistConfigHelper.java @@ -9,7 +9,6 @@ import io.airbyte.api.client.generated.DestinationApi; import io.airbyte.api.client.generated.JobsApi; import io.airbyte.api.client.generated.SourceApi; -import io.airbyte.api.client.invoker.generated.ApiClient; import io.airbyte.api.client.invoker.generated.ApiException; import io.airbyte.api.client.model.generated.ConnectionIdRequestBody; import io.airbyte.api.client.model.generated.ConnectionRead; @@ -22,9 +21,6 @@ import io.airbyte.api.client.model.generated.SourceRead; import io.airbyte.api.client.model.generated.SourceUpdate; import io.airbyte.commons.json.Jsons; -import io.airbyte.config.Configs; -import io.airbyte.config.Configs.WorkerEnvironment; -import io.airbyte.config.EnvConfigs; import io.airbyte.protocol.models.Config; import java.nio.charset.StandardCharsets; import java.util.UUID; @@ -39,40 +35,26 @@ public class PersistConfigHelper { private static final Logger LOGGER = LoggerFactory.getLogger(PersistConfigHelper.class); - private static final Configs CONFIGS = new EnvConfigs(); - private static final AirbyteApiClient CLIENT = getAirbyteApiClient(); - - // TODO inject this with micronaut, possibly making this class a Singleton - private static AirbyteApiClient getAirbyteApiClient() { - if (CONFIGS.getWorkerEnvironment() == WorkerEnvironment.DOCKER) { - return new AirbyteApiClient( - new ApiClient().setScheme("http") - .setHost(CONFIGS.getAirbyteApiHost()) - .setPort(CONFIGS.getAirbyteApiPort()) - .setBasePath("/api")); - } - - return new AirbyteApiClient( - new ApiClient().setScheme("http") - .setHost("airbyte-server-svc") - .setPort(8001) - .setBasePath("/api")); + private final AirbyteApiClient apiClient; + + public PersistConfigHelper(final AirbyteApiClient apiClient) { + this.apiClient = apiClient; } - private static UUID getConnectionIdFromJobId(final Long jobId) throws ApiException { - final JobsApi jobsApi = CLIENT.getJobsApi(); + private UUID getConnectionIdFromJobId(final Long jobId) throws ApiException { + final JobsApi jobsApi = apiClient.getJobsApi(); final JobIdRequestBody body = new JobIdRequestBody().id(jobId); final JobInfoLightRead jobInfo = jobsApi.getJobInfoLight(body); return UUID.fromString(jobInfo.getJob().getConfigId()); } - public static void persistSourceConfig(final Long jobId, final Config config) throws ApiException { + public void persistSourceConfig(final Long jobId, final Config config) throws ApiException { final UUID connectionId = getConnectionIdFromJobId(jobId); - final ConnectionRead connection = CLIENT.getConnectionApi().getConnection(new ConnectionIdRequestBody().connectionId(connectionId)); + final ConnectionRead connection = apiClient.getConnectionApi().getConnection(new ConnectionIdRequestBody().connectionId(connectionId)); final UUID sourceId = connection.getSourceId(); - final SourceApi sourceApi = CLIENT.getSourceApi(); + final SourceApi sourceApi = apiClient.getSourceApi(); final SourceRead source = sourceApi.getSource(new SourceIdRequestBody().sourceId(sourceId)); // TODO might need to strip out OAuth params @@ -88,13 +70,13 @@ public static void persistSourceConfig(final Long jobId, final Config config) th } - public static void persistDestinationConfig(final Long jobId, final Config config) throws ApiException { + public void persistDestinationConfig(final Long jobId, final Config config) throws ApiException { final UUID connectionId = getConnectionIdFromJobId(jobId); - final ConnectionRead connection = CLIENT.getConnectionApi().getConnection(new ConnectionIdRequestBody().connectionId(connectionId)); + final ConnectionRead connection = apiClient.getConnectionApi().getConnection(new ConnectionIdRequestBody().connectionId(connectionId)); final UUID destinationId = connection.getDestinationId(); - final DestinationApi destinationApi = CLIENT.getDestinationApi(); + final DestinationApi destinationApi = apiClient.getDestinationApi(); final DestinationRead destination = destinationApi.getDestination(new DestinationIdRequestBody().destinationId(destinationId)); // TODO might need to strip out OAuth params 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 11580c575faa..4fd40a299e4f 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 @@ -10,6 +10,8 @@ import static io.airbyte.metrics.lib.ApmTraceConstants.Tags.SOURCE_DOCKER_IMAGE_KEY; import datadog.trace.api.Trace; +import io.airbyte.api.client.AirbyteApiClient; +import io.airbyte.api.client.invoker.generated.ApiClient; import io.airbyte.commons.features.FeatureFlags; import io.airbyte.commons.json.Jsons; import io.airbyte.commons.logging.MdcScope; @@ -32,6 +34,7 @@ import io.airbyte.workers.WorkerUtils; import io.airbyte.workers.general.DefaultReplicationWorker; import io.airbyte.workers.general.ReplicationWorker; +import io.airbyte.workers.helper.PersistConfigHelper; import io.airbyte.workers.internal.AirbyteMessageTracker; import io.airbyte.workers.internal.AirbyteSource; import io.airbyte.workers.internal.AirbyteStreamFactory; @@ -139,7 +142,8 @@ public Optional runJob() throws Exception { new VersionedAirbyteMessageBufferedWriterFactory(serDeProvider, migratorFactory, destinationLauncherConfig.getProtocolVersion())), new AirbyteMessageTracker(), new RecordSchemaValidator(WorkerUtils.mapStreamNamesToSchemas(syncInput)), - metricReporter); + metricReporter, + new PersistConfigHelper(getAirbyteApiClient())); log.info("Running replication worker..."); final Path jobRoot = TemporalUtils.getJobRoot(configs.getWorkspaceRoot(), jobRunConfig.getJobId(), jobRunConfig.getAttemptId()); @@ -149,6 +153,15 @@ public Optional runJob() throws Exception { return Optional.of(Jsons.serialize(replicationOutput)); } + private AirbyteApiClient getAirbyteApiClient() { + // TODO make this actually functional + return new AirbyteApiClient( + new ApiClient().setScheme("http") + .setHost("airbyte-server-svc") + .setPort(8001) + .setBasePath("/api")); + } + private AirbyteStreamFactory getStreamFactory(final Version protocolVersion, final MdcScope.Builder mdcScope) { return protocolVersion != null ? new VersionedAirbyteStreamFactory(serDeProvider, migratorFactory, protocolVersion, mdcScope) diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java index b7104497e71a..c591f73e4021 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java @@ -51,6 +51,7 @@ import io.airbyte.workers.WorkerMetricReporter; import io.airbyte.workers.WorkerUtils; import io.airbyte.workers.general.DefaultReplicationWorker; +import io.airbyte.workers.helper.PersistConfigHelper; import io.airbyte.workers.internal.AirbyteMessageTracker; import io.airbyte.workers.internal.AirbyteSource; import io.airbyte.workers.internal.DefaultAirbyteDestination; @@ -293,7 +294,8 @@ private CheckedSupplier, Exception> new VersionedAirbyteMessageBufferedWriterFactory(serDeProvider, migratorFactory, destinationLauncherConfig.getProtocolVersion())), new AirbyteMessageTracker(), new RecordSchemaValidator(WorkerUtils.mapStreamNamesToSchemas(syncInput)), - metricReporter); + metricReporter, + new PersistConfigHelper(airbyteApiClient)); }; } From 9c2bded2e98fbc0d1b8747e6368114201513311c Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 30 Nov 2022 04:50:47 -0400 Subject: [PATCH 08/63] mask instance-wide oauth params when updating configs --- .../job/factory/OAuthConfigSupplier.java | 104 ++++++++++++++---- .../job/factory/OAuthConfigSupplierTest.java | 90 +++++++++++++++ .../java/io/airbyte/server/ServerApp.java | 6 +- .../server/handlers/DestinationHandler.java | 16 ++- .../server/handlers/SourceHandler.java | 16 ++- .../handlers/DestinationHandlerTest.java | 6 +- .../server/handlers/SourceHandlerTest.java | 6 +- 7 files changed, 210 insertions(+), 34 deletions(-) diff --git a/airbyte-persistence/job-persistence/src/main/java/io/airbyte/persistence/job/factory/OAuthConfigSupplier.java b/airbyte-persistence/job-persistence/src/main/java/io/airbyte/persistence/job/factory/OAuthConfigSupplier.java index 183622971416..8626628c61b5 100644 --- a/airbyte-persistence/job-persistence/src/main/java/io/airbyte/persistence/job/factory/OAuthConfigSupplier.java +++ b/airbyte-persistence/job-persistence/src/main/java/io/airbyte/persistence/job/factory/OAuthConfigSupplier.java @@ -26,6 +26,8 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.BiConsumer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,6 +49,36 @@ public static boolean hasOAuthConfigSpecification(final ConnectorSpecification s return spec != null && spec.getAdvancedAuth() != null && spec.getAdvancedAuth().getOauthConfigSpecification() != null; } + public JsonNode maskSourceOAuthParameters(final UUID sourceDefinitionId, final UUID workspaceId, final JsonNode sourceConnectorConfig) + throws IOException { + try { + final StandardSourceDefinition sourceDefinition = configRepository.getStandardSourceDefinition(sourceDefinitionId); + MoreOAuthParameters.getSourceOAuthParameter(configRepository.listSourceOAuthParam().stream(), workspaceId, sourceDefinitionId) + .ifPresent(sourceOAuthParameter -> { + maskOauthParameters(sourceDefinition.getName(), sourceDefinition.getSpec(), sourceConnectorConfig); + }); + return sourceConnectorConfig; + } catch (final JsonValidationException | ConfigNotFoundException e) { + throw new IOException(e); + } + } + + public JsonNode maskDestinationOAuthParameters(final UUID destinationDefinitionId, + final UUID workspaceId, + final JsonNode destinationConnectorConfig) + throws IOException { + try { + final StandardDestinationDefinition destinationDefinition = configRepository.getStandardDestinationDefinition(destinationDefinitionId); + MoreOAuthParameters.getDestinationOAuthParameter(configRepository.listDestinationOAuthParam().stream(), workspaceId, destinationDefinitionId) + .ifPresent(destinationOAuthParameter -> { + maskOauthParameters(destinationDefinition.getName(), destinationDefinition.getSpec(), destinationConnectorConfig); + }); + return destinationConnectorConfig; + } catch (final JsonValidationException | ConfigNotFoundException e) { + throw new IOException(e); + } + } + public JsonNode injectSourceOAuthParameters(final UUID sourceDefinitionId, final UUID workspaceId, final JsonNode sourceConnectorConfig) throws IOException { try { @@ -85,31 +117,21 @@ public JsonNode injectDestinationOAuthParameters(final UUID destinationDefinitio } } - private static boolean injectOAuthParameters(final String connectorName, - final ConnectorSpecification spec, - final JsonNode oAuthParameters, - final JsonNode connectorConfig) { - if (!hasOAuthConfigSpecification(spec)) { - // keep backward compatible behavior if connector does not declare an OAuth config spec - MoreOAuthParameters.mergeJsons((ObjectNode) connectorConfig, (ObjectNode) oAuthParameters); - return true; - } - if (!checkOAuthPredicate(spec.getAdvancedAuth().getPredicateKey(), spec.getAdvancedAuth().getPredicateValue(), connectorConfig)) { - // OAuth is not applicable in this connectorConfig due to the predicate not being verified - return false; - } - // TODO: if we write a migration to flatten persisted configs in db, we don't need to flatten - // here see https://github.com/airbytehq/airbyte/issues/7624 - boolean result = false; - final JsonNode flatOAuthParameters = MoreOAuthParameters.flattenOAuthConfig(oAuthParameters); + /** + * Gets the OAuth parameter paths as specified in the connector spec and traverses through them + */ + private static void traverseOAuthOutputPaths(final ConnectorSpecification spec, + final String connectorName, + final BiConsumer> consumer) { final JsonNode outputSpecTop = spec.getAdvancedAuth().getOauthConfigSpecification().getCompleteOauthServerOutputSpecification(); final JsonNode outputSpec; if (outputSpecTop.has(PROPERTIES)) { outputSpec = outputSpecTop.get(PROPERTIES); } else { LOGGER.error(String.format("In %s's advanced_auth spec, completeOAuthServerOutputSpecification does not declare properties.", connectorName)); - return false; + return; } + for (final String key : Jsons.keys(outputSpec)) { final JsonNode node = outputSpec.get(key); if (node.getNodeType() == OBJECT) { @@ -121,8 +143,7 @@ private static boolean injectOAuthParameters(final String connectorName, propertyPath.add(arrayNode.get(i).asText()); } if (!propertyPath.isEmpty()) { - Jsons.replaceNestedValue(connectorConfig, propertyPath, flatOAuthParameters.get(key)); - result = true; + consumer.accept(key, propertyPath); } else { LOGGER.error(String.format("In %s's advanced_auth spec, completeOAuthServerOutputSpecification includes an invalid empty %s for %s", connectorName, PATH_IN_CONNECTOR_CONFIG, key)); @@ -137,7 +158,48 @@ private static boolean injectOAuthParameters(final String connectorName, connectorName, key)); } } - return result; + } + + private static void maskOauthParameters(final String connectorName, final ConnectorSpecification spec, final JsonNode connectorConfig) { + if (!hasOAuthConfigSpecification(spec)) { + return; + } + if (!checkOAuthPredicate(spec.getAdvancedAuth().getPredicateKey(), spec.getAdvancedAuth().getPredicateValue(), connectorConfig)) { + // OAuth is not applicable in this connectorConfig due to the predicate not being verified + return; + } + + traverseOAuthOutputPaths(spec, connectorName, (_key, propertyPath) -> { + Jsons.replaceNestedValue(connectorConfig, propertyPath, Jsons.jsonNode(MoreOAuthParameters.SECRET_MASK)); + }); + + } + + private static boolean injectOAuthParameters(final String connectorName, + final ConnectorSpecification spec, + final JsonNode oAuthParameters, + final JsonNode connectorConfig) { + if (!hasOAuthConfigSpecification(spec)) { + // keep backward compatible behavior if connector does not declare an OAuth config spec + MoreOAuthParameters.mergeJsons((ObjectNode) connectorConfig, (ObjectNode) oAuthParameters); + return true; + } + if (!checkOAuthPredicate(spec.getAdvancedAuth().getPredicateKey(), spec.getAdvancedAuth().getPredicateValue(), connectorConfig)) { + // OAuth is not applicable in this connectorConfig due to the predicate not being verified + return false; + } + + // TODO: if we write a migration to flatten persisted configs in db, we don't need to flatten + // here see https://github.com/airbytehq/airbyte/issues/7624 + final JsonNode flatOAuthParameters = MoreOAuthParameters.flattenOAuthConfig(oAuthParameters); + + final AtomicBoolean result = new AtomicBoolean(false); + traverseOAuthOutputPaths(spec, connectorName, (key, propertyPath) -> { + Jsons.replaceNestedValue(connectorConfig, propertyPath, flatOAuthParameters.get(key)); + result.set(true); + }); + + return result.get(); } private static boolean checkOAuthPredicate(final List predicateKey, final String predicateValue, final JsonNode connectorConfig) { diff --git a/airbyte-persistence/job-persistence/src/test/java/io/airbyte/persistence/job/factory/OAuthConfigSupplierTest.java b/airbyte-persistence/job-persistence/src/test/java/io/airbyte/persistence/job/factory/OAuthConfigSupplierTest.java index 7812df9dee9f..b87a02c45f59 100644 --- a/airbyte-persistence/job-persistence/src/test/java/io/airbyte/persistence/job/factory/OAuthConfigSupplierTest.java +++ b/airbyte-persistence/job-persistence/src/test/java/io/airbyte/persistence/job/factory/OAuthConfigSupplierTest.java @@ -21,6 +21,7 @@ import io.airbyte.config.StandardSourceDefinition; import io.airbyte.config.persistence.ConfigNotFoundException; import io.airbyte.config.persistence.ConfigRepository; +import io.airbyte.oauth.MoreOAuthParameters; import io.airbyte.protocol.models.AdvancedAuth; import io.airbyte.protocol.models.AdvancedAuth.AuthFlowType; import io.airbyte.protocol.models.ConnectorSpecification; @@ -68,6 +69,14 @@ void testNoOAuthInjectionBecauseEmptyParams() throws IOException { assertNoTracking(); } + @Test + void testNoAuthMaskingBecauseEmptyParams() throws IOException { + final JsonNode config = generateJsonConfig(); + final UUID workspaceId = UUID.randomUUID(); + final JsonNode actualConfig = oAuthConfigSupplier.maskSourceOAuthParameters(sourceDefinitionId, workspaceId, Jsons.clone(config)); + assertEquals(config, actualConfig); + } + @Test void testNoOAuthInjectionBecauseMissingPredicateKey() throws IOException, JsonValidationException, ConfigNotFoundException { setupStandardDefinitionMock(createAdvancedAuth() @@ -94,6 +103,18 @@ void testNoOAuthInjectionBecauseWrongPredicateValue() throws IOException, JsonVa assertNoTracking(); } + @Test + void testNoOAuthMaskingBecauseWrongPredicateValue() throws IOException, JsonValidationException, ConfigNotFoundException { + setupStandardDefinitionMock(createAdvancedAuth() + .withPredicateKey(List.of(CREDENTIALS, AUTH_TYPE)) + .withPredicateValue("wrong_auth_type")); + final JsonNode config = generateJsonConfig(); + final UUID workspaceId = UUID.randomUUID(); + setupOAuthParamMocks(generateOAuthParameters()); + final JsonNode actualConfig = oAuthConfigSupplier.maskSourceOAuthParameters(sourceDefinitionId, workspaceId, Jsons.clone(config)); + assertEquals(config, actualConfig); + } + @Test void testOAuthInjection() throws JsonValidationException, IOException { final JsonNode config = generateJsonConfig(); @@ -106,6 +127,17 @@ void testOAuthInjection() throws JsonValidationException, IOException { assertTracking(workspaceId); } + @Test + void testOAuthMasking() throws JsonValidationException, IOException { + final JsonNode config = generateJsonConfig(); + final UUID workspaceId = UUID.randomUUID(); + final Map oauthParameters = generateOAuthParameters(); + setupOAuthParamMocks(oauthParameters); + final JsonNode actualConfig = oAuthConfigSupplier.maskSourceOAuthParameters(sourceDefinitionId, workspaceId, Jsons.clone(config)); + final JsonNode expectedConfig = getExpectedNode(MoreOAuthParameters.SECRET_MASK); + assertEquals(expectedConfig, actualConfig); + } + @Test void testOAuthInjectionWithoutPredicate() throws JsonValidationException, IOException, ConfigNotFoundException { setupStandardDefinitionMock(createAdvancedAuth() @@ -121,6 +153,20 @@ void testOAuthInjectionWithoutPredicate() throws JsonValidationException, IOExce assertTracking(workspaceId); } + @Test + void testOAuthMaskingWithoutPredicate() throws JsonValidationException, IOException, ConfigNotFoundException { + setupStandardDefinitionMock(createAdvancedAuth() + .withPredicateKey(null) + .withPredicateValue(null)); + final JsonNode config = generateJsonConfig(); + final UUID workspaceId = UUID.randomUUID(); + final Map oauthParameters = generateOAuthParameters(); + setupOAuthParamMocks(oauthParameters); + final JsonNode actualConfig = oAuthConfigSupplier.maskSourceOAuthParameters(sourceDefinitionId, workspaceId, Jsons.clone(config)); + final JsonNode expectedConfig = getExpectedNode(MoreOAuthParameters.SECRET_MASK); + assertEquals(expectedConfig, actualConfig); + } + @Test void testOAuthInjectionWithoutPredicateValue() throws JsonValidationException, IOException, ConfigNotFoundException { setupStandardDefinitionMock(createAdvancedAuth() @@ -136,6 +182,20 @@ void testOAuthInjectionWithoutPredicateValue() throws JsonValidationException, I assertTracking(workspaceId); } + @Test + void testOAuthMaskingWithoutPredicateValue() throws JsonValidationException, IOException, ConfigNotFoundException { + setupStandardDefinitionMock(createAdvancedAuth() + .withPredicateKey(List.of(CREDENTIALS, AUTH_TYPE)) + .withPredicateValue("")); + final JsonNode config = generateJsonConfig(); + final UUID workspaceId = UUID.randomUUID(); + final Map oauthParameters = generateOAuthParameters(); + setupOAuthParamMocks(oauthParameters); + final JsonNode actualConfig = oAuthConfigSupplier.maskSourceOAuthParameters(sourceDefinitionId, workspaceId, Jsons.clone(config)); + final JsonNode expectedConfig = getExpectedNode(MoreOAuthParameters.SECRET_MASK); + assertEquals(expectedConfig, actualConfig); + } + @Test void testOAuthFullInjectionBecauseNoOAuthSpec() throws JsonValidationException, IOException, ConfigNotFoundException { final JsonNode config = generateJsonConfig(); @@ -158,6 +218,23 @@ void testOAuthFullInjectionBecauseNoOAuthSpec() throws JsonValidationException, assertTracking(workspaceId); } + @Test + void testOAuthNoMaskingBecauseNoOAuthSpec() throws JsonValidationException, IOException, ConfigNotFoundException { + final JsonNode config = generateJsonConfig(); + final UUID workspaceId = UUID.randomUUID(); + final Map oauthParameters = generateOAuthParameters(); + when(configRepository.getStandardSourceDefinition(any())) + .thenReturn(new StandardSourceDefinition() + .withSourceDefinitionId(sourceDefinitionId) + .withName("test") + .withDockerRepository("test/test") + .withDockerImageTag("dev") + .withSpec(null)); + setupOAuthParamMocks(oauthParameters); + final JsonNode actualConfig = oAuthConfigSupplier.maskSourceOAuthParameters(sourceDefinitionId, workspaceId, Jsons.clone(config)); + assertEquals(config, actualConfig); + } + @Test void testOAuthInjectionScopedToWorkspace() throws JsonValidationException, IOException { final JsonNode config = generateJsonConfig(); @@ -213,6 +290,19 @@ void testOAuthInjectionNestedParameters() throws JsonValidationException, IOExce assertTracking(workspaceId); } + @Test + void testOAuthMaskingNestedParameters() throws JsonValidationException, IOException { + // Until https://github.com/airbytehq/airbyte/issues/7624 is solved, we need to handle nested oauth + // parameters + final JsonNode config = generateJsonConfig(); + final UUID workspaceId = UUID.randomUUID(); + final Map oauthParameters = generateNestedOAuthParameters(); + setupOAuthParamMocks(oauthParameters); + final JsonNode actualConfig = oAuthConfigSupplier.maskSourceOAuthParameters(sourceDefinitionId, workspaceId, Jsons.clone(config)); + final JsonNode expectedConfig = getExpectedNode(MoreOAuthParameters.SECRET_MASK); + assertEquals(expectedConfig, actualConfig); + } + private static AdvancedAuth createAdvancedAuth() { return new AdvancedAuth() .withAuthFlowType(AuthFlowType.OAUTH_2_0) 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 e74c91094418..06a3dc898f06 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/ServerApp.java +++ b/airbyte-server/src/main/java/io/airbyte/server/ServerApp.java @@ -273,7 +273,8 @@ public static ServerRunnable getServer(final ServerFactory apiFactory, secretsRepositoryReader, secretsRepositoryWriter, schemaValidator, - connectionsHandler); + connectionsHandler, + oAuthConfigSupplier); final OperationsHandler operationsHandler = new OperationsHandler(configRepository); @@ -302,7 +303,8 @@ public static ServerRunnable getServer(final ServerFactory apiFactory, secretsRepositoryReader, secretsRepositoryWriter, schemaValidator, - connectionsHandler); + connectionsHandler, + oAuthConfigSupplier); final SourceDefinitionsHandler sourceDefinitionsHandler = new SourceDefinitionsHandler(configRepository, syncSchedulerClient, sourceHandler); diff --git a/airbyte-server/src/main/java/io/airbyte/server/handlers/DestinationHandler.java b/airbyte-server/src/main/java/io/airbyte/server/handlers/DestinationHandler.java index a828448c77a9..92c30e0ee228 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/handlers/DestinationHandler.java +++ b/airbyte-server/src/main/java/io/airbyte/server/handlers/DestinationHandler.java @@ -26,6 +26,7 @@ import io.airbyte.config.persistence.SecretsRepositoryReader; import io.airbyte.config.persistence.SecretsRepositoryWriter; import io.airbyte.config.persistence.split_secrets.JsonSecretsProcessor; +import io.airbyte.persistence.job.factory.OAuthConfigSupplier; import io.airbyte.protocol.models.ConnectorSpecification; import io.airbyte.server.converters.ConfigurationUpdate; import io.airbyte.validation.json.JsonSchemaValidator; @@ -45,6 +46,7 @@ public class DestinationHandler { private final JsonSchemaValidator validator; private final ConfigurationUpdate configurationUpdate; private final JsonSecretsProcessor secretsProcessor; + private final OAuthConfigSupplier oAuthConfigSupplier; @VisibleForTesting DestinationHandler(final ConfigRepository configRepository, @@ -54,7 +56,8 @@ public class DestinationHandler { final ConnectionsHandler connectionsHandler, final Supplier uuidGenerator, final JsonSecretsProcessor secretsProcessor, - final ConfigurationUpdate configurationUpdate) { + final ConfigurationUpdate configurationUpdate, + final OAuthConfigSupplier oAuthConfigSupplier) { this.configRepository = configRepository; this.secretsRepositoryReader = secretsRepositoryReader; this.secretsRepositoryWriter = secretsRepositoryWriter; @@ -63,13 +66,15 @@ public class DestinationHandler { this.uuidGenerator = uuidGenerator; this.configurationUpdate = configurationUpdate; this.secretsProcessor = secretsProcessor; + this.oAuthConfigSupplier = oAuthConfigSupplier; } public DestinationHandler(final ConfigRepository configRepository, final SecretsRepositoryReader secretsRepositoryReader, final SecretsRepositoryWriter secretsRepositoryWriter, final JsonSchemaValidator integrationSchemaValidation, - final ConnectionsHandler connectionsHandler) { + final ConnectionsHandler connectionsHandler, + final OAuthConfigSupplier oAuthConfigSupplier) { this( configRepository, secretsRepositoryReader, @@ -80,7 +85,8 @@ public DestinationHandler(final ConfigRepository configRepository, JsonSecretsProcessor.builder() .copySecrets(true) .build(), - new ConfigurationUpdate(configRepository, secretsRepositoryReader)); + new ConfigurationUpdate(configRepository, secretsRepositoryReader), + oAuthConfigSupplier); } public DestinationRead createDestination(final DestinationCreate destinationCreate) @@ -248,12 +254,14 @@ private void persistDestinationConnection(final String name, final JsonNode configurationJson, final boolean tombstone) throws JsonValidationException, IOException, ConfigNotFoundException { + final JsonNode oAuthMaskedConfigurationJson = + oAuthConfigSupplier.maskDestinationOAuthParameters(destinationDefinitionId, workspaceId, configurationJson); final DestinationConnection destinationConnection = new DestinationConnection() .withName(name) .withDestinationDefinitionId(destinationDefinitionId) .withWorkspaceId(workspaceId) .withDestinationId(destinationId) - .withConfiguration(configurationJson) + .withConfiguration(oAuthMaskedConfigurationJson) .withTombstone(tombstone); secretsRepositoryWriter.writeDestinationConnection(destinationConnection, getSpec(destinationDefinitionId)); } diff --git a/airbyte-server/src/main/java/io/airbyte/server/handlers/SourceHandler.java b/airbyte-server/src/main/java/io/airbyte/server/handlers/SourceHandler.java index ab1e1092d235..b38a877248cf 100644 --- a/airbyte-server/src/main/java/io/airbyte/server/handlers/SourceHandler.java +++ b/airbyte-server/src/main/java/io/airbyte/server/handlers/SourceHandler.java @@ -24,6 +24,7 @@ import io.airbyte.config.persistence.SecretsRepositoryReader; import io.airbyte.config.persistence.SecretsRepositoryWriter; import io.airbyte.config.persistence.split_secrets.JsonSecretsProcessor; +import io.airbyte.persistence.job.factory.OAuthConfigSupplier; import io.airbyte.protocol.models.ConnectorSpecification; import io.airbyte.server.converters.ConfigurationUpdate; import io.airbyte.validation.json.JsonSchemaValidator; @@ -43,6 +44,7 @@ public class SourceHandler { private final ConnectionsHandler connectionsHandler; private final ConfigurationUpdate configurationUpdate; private final JsonSecretsProcessor secretsProcessor; + private final OAuthConfigSupplier oAuthConfigSupplier; SourceHandler(final ConfigRepository configRepository, final SecretsRepositoryReader secretsRepositoryReader, @@ -51,7 +53,8 @@ public class SourceHandler { final ConnectionsHandler connectionsHandler, final Supplier uuidGenerator, final JsonSecretsProcessor secretsProcessor, - final ConfigurationUpdate configurationUpdate) { + final ConfigurationUpdate configurationUpdate, + final OAuthConfigSupplier oAuthConfigSupplier) { this.configRepository = configRepository; this.secretsRepositoryReader = secretsRepositoryReader; this.secretsRepositoryWriter = secretsRepositoryWriter; @@ -60,13 +63,15 @@ public class SourceHandler { this.uuidGenerator = uuidGenerator; this.configurationUpdate = configurationUpdate; this.secretsProcessor = secretsProcessor; + this.oAuthConfigSupplier = oAuthConfigSupplier; } public SourceHandler(final ConfigRepository configRepository, final SecretsRepositoryReader secretsRepositoryReader, final SecretsRepositoryWriter secretsRepositoryWriter, final JsonSchemaValidator integrationSchemaValidation, - final ConnectionsHandler connectionsHandler) { + final ConnectionsHandler connectionsHandler, + final OAuthConfigSupplier oAuthConfigSupplier) { this( configRepository, secretsRepositoryReader, @@ -77,7 +82,8 @@ public SourceHandler(final ConfigRepository configRepository, JsonSecretsProcessor.builder() .copySecrets(true) .build(), - new ConfigurationUpdate(configRepository, secretsRepositoryReader)); + new ConfigurationUpdate(configRepository, secretsRepositoryReader), + oAuthConfigSupplier); } public SourceRead createSource(final SourceCreate sourceCreate) @@ -295,14 +301,14 @@ private void persistSourceConnection(final String name, final JsonNode configurationJson, final ConnectorSpecification spec) throws JsonValidationException, IOException { + final JsonNode oAuthMaskedConfigurationJson = oAuthConfigSupplier.maskSourceOAuthParameters(sourceDefinitionId, workspaceId, configurationJson); final SourceConnection sourceConnection = new SourceConnection() .withName(name) .withSourceDefinitionId(sourceDefinitionId) .withWorkspaceId(workspaceId) .withSourceId(sourceId) .withTombstone(tombstone) - .withConfiguration(configurationJson); - + .withConfiguration(oAuthMaskedConfigurationJson); secretsRepositoryWriter.writeSourceConnection(sourceConnection, spec); } diff --git a/airbyte-server/src/test/java/io/airbyte/server/handlers/DestinationHandlerTest.java b/airbyte-server/src/test/java/io/airbyte/server/handlers/DestinationHandlerTest.java index 28fb3868bb2b..d9bbeef6ed8d 100644 --- a/airbyte-server/src/test/java/io/airbyte/server/handlers/DestinationHandlerTest.java +++ b/airbyte-server/src/test/java/io/airbyte/server/handlers/DestinationHandlerTest.java @@ -31,6 +31,7 @@ import io.airbyte.config.persistence.SecretsRepositoryReader; import io.airbyte.config.persistence.SecretsRepositoryWriter; import io.airbyte.config.persistence.split_secrets.JsonSecretsProcessor; +import io.airbyte.persistence.job.factory.OAuthConfigSupplier; import io.airbyte.protocol.models.ConnectorSpecification; import io.airbyte.server.converters.ConfigurationUpdate; import io.airbyte.server.helpers.ConnectorSpecificationHelpers; @@ -58,6 +59,7 @@ class DestinationHandlerTest { private Supplier uuidGenerator; private JsonSecretsProcessor secretsProcessor; private ConnectorSpecification connectorSpecification; + private OAuthConfigSupplier oAuthConfigSupplier; // needs to match name of file in src/test/resources/icons private static final String ICON = "test-destination.svg"; @@ -74,6 +76,7 @@ void setUp() throws IOException { connectionsHandler = mock(ConnectionsHandler.class); configurationUpdate = mock(ConfigurationUpdate.class); secretsProcessor = mock(JsonSecretsProcessor.class); + oAuthConfigSupplier = mock(OAuthConfigSupplier.class); connectorSpecification = ConnectorSpecificationHelpers.generateConnectorSpecification(); @@ -103,7 +106,8 @@ void setUp() throws IOException { connectionsHandler, uuidGenerator, secretsProcessor, - configurationUpdate); + configurationUpdate, + oAuthConfigSupplier); } @Test diff --git a/airbyte-server/src/test/java/io/airbyte/server/handlers/SourceHandlerTest.java b/airbyte-server/src/test/java/io/airbyte/server/handlers/SourceHandlerTest.java index 9da285912971..75ac06d43129 100644 --- a/airbyte-server/src/test/java/io/airbyte/server/handlers/SourceHandlerTest.java +++ b/airbyte-server/src/test/java/io/airbyte/server/handlers/SourceHandlerTest.java @@ -35,6 +35,7 @@ import io.airbyte.config.persistence.SecretsRepositoryReader; import io.airbyte.config.persistence.SecretsRepositoryWriter; import io.airbyte.config.persistence.split_secrets.JsonSecretsProcessor; +import io.airbyte.persistence.job.factory.OAuthConfigSupplier; import io.airbyte.protocol.models.ConnectorSpecification; import io.airbyte.server.converters.ConfigurationUpdate; import io.airbyte.server.helpers.ConnectionHelpers; @@ -64,6 +65,7 @@ class SourceHandlerTest { private Supplier uuidGenerator; private JsonSecretsProcessor secretsProcessor; private ConnectorSpecification connectorSpecification; + private OAuthConfigSupplier oAuthConfigSupplier; // needs to match name of file in src/test/resources/icons private static final String ICON = "test-source.svg"; @@ -79,6 +81,7 @@ void setUp() throws IOException { configurationUpdate = mock(ConfigurationUpdate.class); uuidGenerator = mock(Supplier.class); secretsProcessor = mock(JsonSecretsProcessor.class); + oAuthConfigSupplier = mock(OAuthConfigSupplier.class); connectorSpecification = ConnectorSpecificationHelpers.generateConnectorSpecification(); @@ -105,7 +108,8 @@ void setUp() throws IOException { connectionsHandler, uuidGenerator, secretsProcessor, - configurationUpdate); + configurationUpdate, + oAuthConfigSupplier); } @Test From 1ebed551fbeee9784bb718dd20bcaf04e333d390 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 30 Nov 2022 13:24:38 -0400 Subject: [PATCH 09/63] defaultreplicationworker tests --- .../general/DefaultReplicationWorker.java | 4 +- .../test_utils/AirbyteMessageUtils.java | 13 ++ .../general/DefaultReplicationWorkerTest.java | 169 ++++++++++++++++-- 3 files changed, 165 insertions(+), 21 deletions(-) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java index 9eb1170a5103..37bea8d50c0e 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java @@ -268,8 +268,8 @@ private static Runnable readFromDstRunnable(final AirbyteDestination destination persistConfigHelper.persistDestinationConfig(jobId, message.getControl().getConnectorConfig().getConfig()); } } catch (final ApiException e) { - // TODO this should probably throw rather than just log and continue LOGGER.error("Error trying to save updated destination config", e); + throw new RuntimeException(e); } } @@ -333,8 +333,8 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou try { persistConfigHelper.persistSourceConfig(jobId, message.getControl().getConnectorConfig().getConfig()); } catch (final ApiException e) { - // TODO this should probably throw rather than just log and continue LOGGER.error("Error trying to save updated source config", e); + throw new RuntimeException(e); } } diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/test_utils/AirbyteMessageUtils.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/test_utils/AirbyteMessageUtils.java index 2aede7159739..bffe290be9d2 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/test_utils/AirbyteMessageUtils.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/test_utils/AirbyteMessageUtils.java @@ -7,6 +7,8 @@ import com.fasterxml.jackson.databind.JsonNode; import com.google.common.collect.ImmutableMap; import io.airbyte.commons.json.Jsons; +import io.airbyte.protocol.models.AirbyteControlConnectorConfigMessage; +import io.airbyte.protocol.models.AirbyteControlMessage; import io.airbyte.protocol.models.AirbyteErrorTraceMessage; import io.airbyte.protocol.models.AirbyteGlobalState; import io.airbyte.protocol.models.AirbyteLogMessage; @@ -17,6 +19,7 @@ import io.airbyte.protocol.models.AirbyteStateMessage.AirbyteStateType; import io.airbyte.protocol.models.AirbyteStreamState; import io.airbyte.protocol.models.AirbyteTraceMessage; +import io.airbyte.protocol.models.Config; import io.airbyte.protocol.models.StreamDescriptor; import java.time.Instant; import java.util.ArrayList; @@ -127,4 +130,14 @@ public static AirbyteMessage createTraceMessage(final String message, final Doub .withError(new AirbyteErrorTraceMessage().withMessage(message))); } + public static AirbyteMessage createConfigControlMessage(final Config config, final Double emittedAt) { + return new AirbyteMessage() + .withType(Type.CONTROL) + .withControl(new AirbyteControlMessage() + .withEmittedAt(emittedAt) + .withType(AirbyteControlMessage.Type.CONNECTOR_CONFIG) + .withConnectorConfig(new AirbyteControlConnectorConfigMessage() + .withConfig(config))); + } + } diff --git a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java index 7b915ed4943c..8b9a220bf1ac 100644 --- a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java +++ b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java @@ -20,6 +20,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.google.common.collect.ImmutableMap; +import io.airbyte.api.client.invoker.generated.ApiException; import io.airbyte.commons.io.IOs; import io.airbyte.commons.json.Jsons; import io.airbyte.commons.string.Strings; @@ -45,10 +46,12 @@ import io.airbyte.protocol.models.AirbyteMessage; import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair; import io.airbyte.protocol.models.AirbyteTraceMessage; +import io.airbyte.protocol.models.Config; import io.airbyte.validation.json.JsonSchemaValidator; import io.airbyte.workers.*; import io.airbyte.workers.exception.WorkerException; import io.airbyte.workers.helper.FailureHelper; +import io.airbyte.workers.helper.PersistConfigHelper; import io.airbyte.workers.internal.AirbyteDestination; import io.airbyte.workers.internal.AirbyteMessageTracker; import io.airbyte.workers.internal.AirbyteSource; @@ -106,6 +109,7 @@ class DefaultReplicationWorkerTest { private RecordSchemaValidator recordSchemaValidator; private MetricClient metricClient; private WorkerMetricReporter workerMetricReporter; + private PersistConfigHelper persistConfigHelper; @SuppressWarnings("unchecked") @BeforeEach @@ -125,6 +129,7 @@ void setup() throws Exception { destination = mock(AirbyteDestination.class); messageTracker = mock(AirbyteMessageTracker.class); recordSchemaValidator = mock(RecordSchemaValidator.class); + persistConfigHelper = mock(PersistConfigHelper.class); metricClient = MetricClientFactory.getMetricClient(); workerMetricReporter = new WorkerMetricReporter(metricClient, "docker_image:v1.0.0"); @@ -153,7 +158,8 @@ void test() throws Exception { destination, messageTracker, recordSchemaValidator, - workerMetricReporter); + workerMetricReporter, + persistConfigHelper); worker.run(syncInput, jobRoot); @@ -181,7 +187,8 @@ void testInvalidSchema() throws Exception { destination, messageTracker, recordSchemaValidator, - workerMetricReporter); + workerMetricReporter, + persistConfigHelper); worker.run(syncInput, jobRoot); @@ -211,7 +218,8 @@ void testSourceNonZeroExitValue() throws Exception { destination, messageTracker, recordSchemaValidator, - workerMetricReporter); + workerMetricReporter, + persistConfigHelper); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); assertTrue(output.getFailures().stream().anyMatch(f -> f.getFailureOrigin().equals(FailureOrigin.SOURCE))); @@ -231,7 +239,8 @@ void testReplicationRunnableSourceFailure() throws Exception { destination, messageTracker, recordSchemaValidator, - workerMetricReporter); + workerMetricReporter, + persistConfigHelper); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); @@ -239,6 +248,113 @@ void testReplicationRunnableSourceFailure() throws Exception { .anyMatch(f -> f.getFailureOrigin().equals(FailureOrigin.SOURCE) && f.getStacktrace().contains(SOURCE_ERROR_MESSAGE))); } + @Test + void testReplicationRunnableSourceUpdateConfig() throws Exception { + final Config connectorConfig = new Config().withAdditionalProperty("my_key", "my_new_value"); + final AirbyteMessage configMsg = AirbyteMessageUtils.createConfigControlMessage(connectorConfig, 1D); + when(source.attemptRead()).thenReturn(Optional.of(RECORD_MESSAGE1), Optional.of(configMsg), Optional.empty()); + when(mapper.mapMessage(configMsg)).thenReturn(configMsg); + + final ReplicationWorker worker = new DefaultReplicationWorker( + JOB_ID, + JOB_ATTEMPT, + source, + mapper, + destination, + messageTracker, + recordSchemaValidator, + workerMetricReporter, + persistConfigHelper); + + final ReplicationOutput output = worker.run(syncInput, jobRoot); + assertEquals(ReplicationStatus.COMPLETED, output.getReplicationAttemptSummary().getStatus()); + + verify(persistConfigHelper).persistSourceConfig(Long.valueOf(JOB_ID), connectorConfig); + } + + @Test + void testReplicationFailureOnSourceConfigPersistError() throws Exception { + final Config connectorConfig = new Config().withAdditionalProperty("my_key", "my_new_value"); + final AirbyteMessage configMsg = AirbyteMessageUtils.createConfigControlMessage(connectorConfig, 1D); + when(source.attemptRead()).thenReturn(Optional.of(configMsg)); + when(source.isFinished()).thenReturn(false, true); + when(mapper.mapMessage(configMsg)).thenReturn(configMsg); + + final String PERSIST_ERROR_MESSAGE = "there was a problem persisting the new config"; + doThrow(new ApiException(PERSIST_ERROR_MESSAGE)).when(persistConfigHelper).persistSourceConfig(Mockito.any(), Mockito.any()); + + final ReplicationWorker worker = new DefaultReplicationWorker( + JOB_ID, + JOB_ATTEMPT, + source, + mapper, + destination, + messageTracker, + recordSchemaValidator, + workerMetricReporter, + persistConfigHelper); + + final ReplicationOutput output = worker.run(syncInput, jobRoot); + assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); + assertTrue(output.getFailures().stream() + .anyMatch(f -> f.getFailureOrigin().equals(FailureOrigin.REPLICATION) && f.getStacktrace().contains(PERSIST_ERROR_MESSAGE))); + + verify(persistConfigHelper).persistSourceConfig(Long.valueOf(JOB_ID), connectorConfig); + } + + @Test + void testReplicationRunnableDestinationUpdateConfig() throws Exception { + final Config connectorConfig = new Config().withAdditionalProperty("my_key", "my_new_value"); + final AirbyteMessage configMsg = AirbyteMessageUtils.createConfigControlMessage(connectorConfig, 1D); + when(destination.attemptRead()).thenReturn(Optional.of(STATE_MESSAGE), Optional.of(configMsg)); + when(destination.isFinished()).thenReturn(false, false, true); + + final ReplicationWorker worker = new DefaultReplicationWorker( + JOB_ID, + JOB_ATTEMPT, + source, + mapper, + destination, + messageTracker, + recordSchemaValidator, + workerMetricReporter, + persistConfigHelper); + + final ReplicationOutput output = worker.run(syncInput, jobRoot); + assertEquals(ReplicationStatus.COMPLETED, output.getReplicationAttemptSummary().getStatus()); + + verify(persistConfigHelper).persistDestinationConfig(Long.valueOf(JOB_ID), connectorConfig); + } + + @Test + void testReplicationFailureOnDestinationConfigPersistError() throws Exception { + final Config connectorConfig = new Config().withAdditionalProperty("my_key", "my_new_value"); + final AirbyteMessage configMsg = AirbyteMessageUtils.createConfigControlMessage(connectorConfig, 1D); + when(destination.attemptRead()).thenReturn(Optional.of(configMsg)); + when(destination.isFinished()).thenReturn(false, true); + + final String PERSIST_ERROR_MESSAGE = "there was a problem persisting the new config"; + doThrow(new ApiException(PERSIST_ERROR_MESSAGE)).when(persistConfigHelper).persistDestinationConfig(Mockito.any(), Mockito.any()); + + final ReplicationWorker worker = new DefaultReplicationWorker( + JOB_ID, + JOB_ATTEMPT, + source, + mapper, + destination, + messageTracker, + recordSchemaValidator, + workerMetricReporter, + persistConfigHelper); + + final ReplicationOutput output = worker.run(syncInput, jobRoot); + assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); + assertTrue(output.getFailures().stream() + .anyMatch(f -> f.getFailureOrigin().equals(FailureOrigin.REPLICATION) && f.getStacktrace().contains(PERSIST_ERROR_MESSAGE))); + + verify(persistConfigHelper).persistDestinationConfig(Long.valueOf(JOB_ID), connectorConfig); + } + @Test void testReplicationRunnableDestinationFailure() throws Exception { final String DESTINATION_ERROR_MESSAGE = "the destination had a failure"; @@ -253,7 +369,8 @@ void testReplicationRunnableDestinationFailure() throws Exception { destination, messageTracker, recordSchemaValidator, - workerMetricReporter); + workerMetricReporter, + persistConfigHelper); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); @@ -274,7 +391,8 @@ void testReplicationRunnableDestinationFailureViaTraceMessage() throws Exception destination, messageTracker, recordSchemaValidator, - workerMetricReporter); + workerMetricReporter, + persistConfigHelper); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertTrue(output.getFailures().stream() @@ -296,7 +414,8 @@ void testReplicationRunnableWorkerFailure() throws Exception { destination, messageTracker, recordSchemaValidator, - workerMetricReporter); + workerMetricReporter, + persistConfigHelper); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); @@ -322,7 +441,8 @@ void testOnlyStateAndRecordMessagesDeliveredToDestination() throws Exception { destination, messageTracker, recordSchemaValidator, - workerMetricReporter); + workerMetricReporter, + persistConfigHelper); worker.run(syncInput, jobRoot); @@ -346,7 +466,8 @@ void testDestinationNonZeroExitValue() throws Exception { destination, messageTracker, recordSchemaValidator, - workerMetricReporter); + workerMetricReporter, + persistConfigHelper); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); @@ -367,7 +488,8 @@ void testDestinationRunnableDestinationFailure() throws Exception { destination, messageTracker, recordSchemaValidator, - workerMetricReporter); + workerMetricReporter, + persistConfigHelper); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); @@ -389,7 +511,8 @@ void testDestinationRunnableWorkerFailure() throws Exception { destination, messageTracker, recordSchemaValidator, - workerMetricReporter); + workerMetricReporter, + persistConfigHelper); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); @@ -412,7 +535,8 @@ void testLoggingInThreads() throws IOException, WorkerException { destination, messageTracker, recordSchemaValidator, - workerMetricReporter); + workerMetricReporter, + persistConfigHelper); worker.run(syncInput, jobRoot); @@ -453,7 +577,8 @@ void testCancellation() throws InterruptedException { destination, messageTracker, recordSchemaValidator, - workerMetricReporter); + workerMetricReporter, + persistConfigHelper); final Thread workerThread = new Thread(() -> { try { @@ -501,7 +626,8 @@ void testPopulatesOutputOnSuccess() throws WorkerException { destination, messageTracker, recordSchemaValidator, - workerMetricReporter); + workerMetricReporter, + persistConfigHelper); final ReplicationOutput actual = worker.run(syncInput, jobRoot); final ReplicationOutput replicationOutput = new ReplicationOutput() @@ -568,7 +694,8 @@ void testPopulatesStateOnFailureIfAvailable() throws Exception { destination, messageTracker, recordSchemaValidator, - workerMetricReporter); + workerMetricReporter, + persistConfigHelper); final ReplicationOutput actual = worker.run(syncInput, jobRoot); assertNotNull(actual); @@ -587,7 +714,8 @@ void testRetainsStateOnFailureIfNewStateNotAvailable() throws Exception { destination, messageTracker, recordSchemaValidator, - workerMetricReporter); + workerMetricReporter, + persistConfigHelper); final ReplicationOutput actual = worker.run(syncInput, jobRoot); @@ -621,7 +749,8 @@ void testPopulatesStatsOnFailureIfAvailable() throws Exception { destination, messageTracker, recordSchemaValidator, - workerMetricReporter); + workerMetricReporter, + persistConfigHelper); final ReplicationOutput actual = worker.run(syncInput, jobRoot); final SyncStats expectedTotalStats = new SyncStats() @@ -667,7 +796,8 @@ void testDoesNotPopulatesStateOnFailureIfNotAvailable() throws Exception { destination, messageTracker, recordSchemaValidator, - workerMetricReporter); + workerMetricReporter, + persistConfigHelper); final ReplicationOutput actual = worker.run(syncInputWithoutState, jobRoot); @@ -687,7 +817,8 @@ void testDoesNotPopulateOnIrrecoverableFailure() { destination, messageTracker, recordSchemaValidator, - workerMetricReporter); + workerMetricReporter, + persistConfigHelper); assertThrows(WorkerException.class, () -> worker.run(syncInput, jobRoot)); } From 6ec3db520de89ec8f4f57aaca30fa523e39bc80b Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 30 Nov 2022 13:25:00 -0400 Subject: [PATCH 10/63] formatting --- .../airbyte/workers/general/DefaultReplicationWorkerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java index 8b9a220bf1ac..03035b605b75 100644 --- a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java +++ b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java @@ -252,8 +252,8 @@ void testReplicationRunnableSourceFailure() throws Exception { void testReplicationRunnableSourceUpdateConfig() throws Exception { final Config connectorConfig = new Config().withAdditionalProperty("my_key", "my_new_value"); final AirbyteMessage configMsg = AirbyteMessageUtils.createConfigControlMessage(connectorConfig, 1D); - when(source.attemptRead()).thenReturn(Optional.of(RECORD_MESSAGE1), Optional.of(configMsg), Optional.empty()); - when(mapper.mapMessage(configMsg)).thenReturn(configMsg); + when(source.attemptRead()).thenReturn(Optional.of(RECORD_MESSAGE1), Optional.of(configMsg), Optional.empty()); + when(mapper.mapMessage(configMsg)).thenReturn(configMsg); final ReplicationWorker worker = new DefaultReplicationWorker( JOB_ID, From 1bad8e9a60a2f6eef08687a3df20b09f490ad9d7 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 30 Nov 2022 13:38:44 -0400 Subject: [PATCH 11/63] tests for source/destination handlers --- .../server/handlers/DestinationHandlerTest.java | 10 ++++++++++ .../io/airbyte/server/handlers/SourceHandlerTest.java | 10 ++++++++++ 2 files changed, 20 insertions(+) diff --git a/airbyte-server/src/test/java/io/airbyte/server/handlers/DestinationHandlerTest.java b/airbyte-server/src/test/java/io/airbyte/server/handlers/DestinationHandlerTest.java index d9bbeef6ed8d..19d263fdbc94 100644 --- a/airbyte-server/src/test/java/io/airbyte/server/handlers/DestinationHandlerTest.java +++ b/airbyte-server/src/test/java/io/airbyte/server/handlers/DestinationHandlerTest.java @@ -118,6 +118,9 @@ void testCreateDestination() throws JsonValidationException, ConfigNotFoundExcep .thenReturn(destinationConnection); when(configRepository.getStandardDestinationDefinition(standardDestinationDefinition.getDestinationDefinitionId())) .thenReturn(standardDestinationDefinition); + when(oAuthConfigSupplier.maskDestinationOAuthParameters(destinationDefinitionSpecificationRead.getDestinationDefinitionId(), + destinationConnection.getWorkspaceId(), + destinationConnection.getConfiguration())).thenReturn(destinationConnection.getConfiguration()); when(secretsProcessor.prepareSecretsForOutput(destinationConnection.getConfiguration(), destinationDefinitionSpecificationRead.getConnectionSpecification())) .thenReturn(destinationConnection.getConfiguration()); @@ -144,6 +147,8 @@ void testCreateDestination() throws JsonValidationException, ConfigNotFoundExcep verify(validator).ensure(destinationDefinitionSpecificationRead.getConnectionSpecification(), destinationConnection.getConfiguration()); verify(secretsRepositoryWriter).writeDestinationConnection(destinationConnection, connectorSpecification); + verify(oAuthConfigSupplier).maskDestinationOAuthParameters(destinationDefinitionSpecificationRead.getDestinationDefinitionId(), + destinationConnection.getWorkspaceId(), destinationConnection.getConfiguration()); verify(secretsProcessor) .prepareSecretsForOutput(destinationConnection.getConfiguration(), destinationDefinitionSpecificationRead.getConnectionSpecification()); } @@ -169,6 +174,9 @@ void testUpdateDestination() throws JsonValidationException, ConfigNotFoundExcep .thenReturn(newConfiguration); when(secretsProcessor.prepareSecretsForOutput(newConfiguration, destinationDefinitionSpecificationRead.getConnectionSpecification())) .thenReturn(newConfiguration); + when(oAuthConfigSupplier.maskDestinationOAuthParameters(destinationDefinitionSpecificationRead.getDestinationDefinitionId(), + destinationConnection.getWorkspaceId(), + newConfiguration)).thenReturn(newConfiguration); when(configRepository.getStandardDestinationDefinition(standardDestinationDefinition.getDestinationDefinitionId())) .thenReturn(standardDestinationDefinition); when(configRepository.getDestinationDefinitionFromDestination(destinationConnection.getDestinationId())) @@ -187,6 +195,8 @@ void testUpdateDestination() throws JsonValidationException, ConfigNotFoundExcep verify(secretsProcessor).prepareSecretsForOutput(newConfiguration, destinationDefinitionSpecificationRead.getConnectionSpecification()); verify(secretsRepositoryWriter).writeDestinationConnection(expectedDestinationConnection, connectorSpecification); + verify(oAuthConfigSupplier).maskDestinationOAuthParameters(destinationDefinitionSpecificationRead.getDestinationDefinitionId(), + destinationConnection.getWorkspaceId(), newConfiguration); verify(validator).ensure(destinationDefinitionSpecificationRead.getConnectionSpecification(), newConfiguration); } diff --git a/airbyte-server/src/test/java/io/airbyte/server/handlers/SourceHandlerTest.java b/airbyte-server/src/test/java/io/airbyte/server/handlers/SourceHandlerTest.java index 75ac06d43129..758bad37f909 100644 --- a/airbyte-server/src/test/java/io/airbyte/server/handlers/SourceHandlerTest.java +++ b/airbyte-server/src/test/java/io/airbyte/server/handlers/SourceHandlerTest.java @@ -124,6 +124,8 @@ void testCreateSource() throws JsonValidationException, ConfigNotFoundException, when(configRepository.getSourceConnection(sourceConnection.getSourceId())).thenReturn(sourceConnection); when(configRepository.getStandardSourceDefinition(sourceDefinitionSpecificationRead.getSourceDefinitionId())) .thenReturn(standardSourceDefinition); + when(oAuthConfigSupplier.maskSourceOAuthParameters(sourceDefinitionSpecificationRead.getSourceDefinitionId(), sourceConnection.getWorkspaceId(), + sourceCreate.getConnectionConfiguration())).thenReturn(sourceCreate.getConnectionConfiguration()); when(secretsProcessor.prepareSecretsForOutput(sourceCreate.getConnectionConfiguration(), sourceDefinitionSpecificationRead.getConnectionSpecification())) .thenReturn(sourceCreate.getConnectionConfiguration()); @@ -137,6 +139,8 @@ void testCreateSource() throws JsonValidationException, ConfigNotFoundException, verify(secretsProcessor).prepareSecretsForOutput(sourceCreate.getConnectionConfiguration(), sourceDefinitionSpecificationRead.getConnectionSpecification()); + verify(oAuthConfigSupplier).maskSourceOAuthParameters(sourceDefinitionSpecificationRead.getSourceDefinitionId(), + sourceConnection.getWorkspaceId(), sourceCreate.getConnectionConfiguration()); verify(secretsRepositoryWriter).writeSourceConnection(sourceConnection, connectorSpecification); verify(validator).ensure(sourceDefinitionSpecificationRead.getConnectionSpecification(), sourceConnection.getConfiguration()); } @@ -162,6 +166,8 @@ void testUpdateSource() throws JsonValidationException, ConfigNotFoundException, .thenReturn(newConfiguration); when(secretsProcessor.prepareSecretsForOutput(newConfiguration, sourceDefinitionSpecificationRead.getConnectionSpecification())) .thenReturn(newConfiguration); + when(oAuthConfigSupplier.maskSourceOAuthParameters(sourceDefinitionSpecificationRead.getSourceDefinitionId(), sourceConnection.getWorkspaceId(), + newConfiguration)).thenReturn(newConfiguration); when(configRepository.getStandardSourceDefinition(sourceDefinitionSpecificationRead.getSourceDefinitionId())) .thenReturn(standardSourceDefinition); when(configRepository.getSourceDefinitionFromSource(sourceConnection.getSourceId())) @@ -179,6 +185,8 @@ void testUpdateSource() throws JsonValidationException, ConfigNotFoundException, assertEquals(expectedSourceRead, actualSourceRead); verify(secretsProcessor).prepareSecretsForOutput(newConfiguration, sourceDefinitionSpecificationRead.getConnectionSpecification()); + verify(oAuthConfigSupplier).maskSourceOAuthParameters(sourceDefinitionSpecificationRead.getSourceDefinitionId(), + sourceConnection.getWorkspaceId(), newConfiguration); verify(secretsRepositoryWriter).writeSourceConnection(expectedSourceConnection, connectorSpecification); verify(validator).ensure(sourceDefinitionSpecificationRead.getConnectionSpecification(), newConfiguration); } @@ -344,6 +352,8 @@ void testDeleteSource() throws JsonValidationException, ConfigNotFoundException, when(secretsRepositoryReader.getSourceConnectionWithSecrets(sourceConnection.getSourceId())) .thenReturn(sourceConnection) .thenReturn(expectedSourceConnection); + when(oAuthConfigSupplier.maskSourceOAuthParameters(sourceDefinitionSpecificationRead.getSourceDefinitionId(), sourceConnection.getWorkspaceId(), + newConfiguration)).thenReturn(newConfiguration); when(configRepository.getStandardSourceDefinition(sourceDefinitionSpecificationRead.getSourceDefinitionId())) .thenReturn(standardSourceDefinition); when(configRepository.getSourceDefinitionFromSource(sourceConnection.getSourceId())).thenReturn(standardSourceDefinition); From 753dafe1908c6f1de190b2823266e41bdb363250 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 30 Nov 2022 13:45:27 -0400 Subject: [PATCH 12/63] rm todo --- .../java/io/airbyte/workers/helper/PersistConfigHelper.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/PersistConfigHelper.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/PersistConfigHelper.java index 57bb99e4a5bf..e5f854ee0f6f 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/PersistConfigHelper.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/PersistConfigHelper.java @@ -57,8 +57,6 @@ public void persistSourceConfig(final Long jobId, final Config config) throws Ap final SourceApi sourceApi = apiClient.getSourceApi(); final SourceRead source = sourceApi.getSource(new SourceIdRequestBody().sourceId(sourceId)); - // TODO might need to strip out OAuth params - final SourceRead updatedSource = sourceApi .updateSource(new SourceUpdate() .sourceId(sourceId) @@ -79,8 +77,6 @@ public void persistDestinationConfig(final Long jobId, final Config config) thro final DestinationApi destinationApi = apiClient.getDestinationApi(); final DestinationRead destination = destinationApi.getDestination(new DestinationIdRequestBody().destinationId(destinationId)); - // TODO might need to strip out OAuth params - final DestinationRead updatedDestination = destinationApi .updateDestination(new DestinationUpdate() .destinationId(destinationId) From ab8e2fc6b1a351024e12f22fa75aff16b727d8b7 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 30 Nov 2022 15:29:17 -0400 Subject: [PATCH 13/63] refactor test a bit to fix pmd --- .../general/DefaultReplicationWorkerTest.java | 29 +++++++------------ 1 file changed, 11 insertions(+), 18 deletions(-) diff --git a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java index 03035b605b75..35ad10226d4e 100644 --- a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java +++ b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java @@ -93,6 +93,8 @@ class DefaultReplicationWorkerTest { private static final AirbyteMessage STATE_MESSAGE = AirbyteMessageUtils.createStateMessage("checkpoint", "1"); private static final AirbyteTraceMessage ERROR_TRACE_MESSAGE = AirbyteMessageUtils.createErrorTraceMessage("a connector error occurred", Double.valueOf(123)); + final Config CONNECTOR_CONFIG = new Config().withAdditionalProperty("my_key", "my_new_value"); + final AirbyteMessage CONFIG_MESSAGE = AirbyteMessageUtils.createConfigControlMessage(CONNECTOR_CONFIG, 1D); private static final String STREAM1 = "stream1"; private static final String NAMESPACE = "namespace"; @@ -141,6 +143,7 @@ void setup() throws Exception { when(mapper.mapMessage(RECORD_MESSAGE1)).thenReturn(RECORD_MESSAGE1); when(mapper.mapMessage(RECORD_MESSAGE2)).thenReturn(RECORD_MESSAGE2); when(mapper.mapMessage(RECORD_MESSAGE3)).thenReturn(RECORD_MESSAGE3); + when(mapper.mapMessage(CONFIG_MESSAGE)).thenReturn(CONFIG_MESSAGE); } @AfterEach @@ -250,10 +253,7 @@ void testReplicationRunnableSourceFailure() throws Exception { @Test void testReplicationRunnableSourceUpdateConfig() throws Exception { - final Config connectorConfig = new Config().withAdditionalProperty("my_key", "my_new_value"); - final AirbyteMessage configMsg = AirbyteMessageUtils.createConfigControlMessage(connectorConfig, 1D); - when(source.attemptRead()).thenReturn(Optional.of(RECORD_MESSAGE1), Optional.of(configMsg), Optional.empty()); - when(mapper.mapMessage(configMsg)).thenReturn(configMsg); + when(source.attemptRead()).thenReturn(Optional.of(RECORD_MESSAGE1), Optional.of(CONFIG_MESSAGE), Optional.empty()); final ReplicationWorker worker = new DefaultReplicationWorker( JOB_ID, @@ -269,16 +269,13 @@ void testReplicationRunnableSourceUpdateConfig() throws Exception { final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.COMPLETED, output.getReplicationAttemptSummary().getStatus()); - verify(persistConfigHelper).persistSourceConfig(Long.valueOf(JOB_ID), connectorConfig); + verify(persistConfigHelper).persistSourceConfig(Long.valueOf(JOB_ID), CONNECTOR_CONFIG); } @Test void testReplicationFailureOnSourceConfigPersistError() throws Exception { - final Config connectorConfig = new Config().withAdditionalProperty("my_key", "my_new_value"); - final AirbyteMessage configMsg = AirbyteMessageUtils.createConfigControlMessage(connectorConfig, 1D); - when(source.attemptRead()).thenReturn(Optional.of(configMsg)); + when(source.attemptRead()).thenReturn(Optional.of(CONFIG_MESSAGE)); when(source.isFinished()).thenReturn(false, true); - when(mapper.mapMessage(configMsg)).thenReturn(configMsg); final String PERSIST_ERROR_MESSAGE = "there was a problem persisting the new config"; doThrow(new ApiException(PERSIST_ERROR_MESSAGE)).when(persistConfigHelper).persistSourceConfig(Mockito.any(), Mockito.any()); @@ -299,14 +296,12 @@ void testReplicationFailureOnSourceConfigPersistError() throws Exception { assertTrue(output.getFailures().stream() .anyMatch(f -> f.getFailureOrigin().equals(FailureOrigin.REPLICATION) && f.getStacktrace().contains(PERSIST_ERROR_MESSAGE))); - verify(persistConfigHelper).persistSourceConfig(Long.valueOf(JOB_ID), connectorConfig); + verify(persistConfigHelper).persistSourceConfig(Long.valueOf(JOB_ID), CONNECTOR_CONFIG); } @Test void testReplicationRunnableDestinationUpdateConfig() throws Exception { - final Config connectorConfig = new Config().withAdditionalProperty("my_key", "my_new_value"); - final AirbyteMessage configMsg = AirbyteMessageUtils.createConfigControlMessage(connectorConfig, 1D); - when(destination.attemptRead()).thenReturn(Optional.of(STATE_MESSAGE), Optional.of(configMsg)); + when(destination.attemptRead()).thenReturn(Optional.of(STATE_MESSAGE), Optional.of(CONFIG_MESSAGE)); when(destination.isFinished()).thenReturn(false, false, true); final ReplicationWorker worker = new DefaultReplicationWorker( @@ -323,14 +318,12 @@ void testReplicationRunnableDestinationUpdateConfig() throws Exception { final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.COMPLETED, output.getReplicationAttemptSummary().getStatus()); - verify(persistConfigHelper).persistDestinationConfig(Long.valueOf(JOB_ID), connectorConfig); + verify(persistConfigHelper).persistDestinationConfig(Long.valueOf(JOB_ID), CONNECTOR_CONFIG); } @Test void testReplicationFailureOnDestinationConfigPersistError() throws Exception { - final Config connectorConfig = new Config().withAdditionalProperty("my_key", "my_new_value"); - final AirbyteMessage configMsg = AirbyteMessageUtils.createConfigControlMessage(connectorConfig, 1D); - when(destination.attemptRead()).thenReturn(Optional.of(configMsg)); + when(destination.attemptRead()).thenReturn(Optional.of(CONFIG_MESSAGE)); when(destination.isFinished()).thenReturn(false, true); final String PERSIST_ERROR_MESSAGE = "there was a problem persisting the new config"; @@ -352,7 +345,7 @@ void testReplicationFailureOnDestinationConfigPersistError() throws Exception { assertTrue(output.getFailures().stream() .anyMatch(f -> f.getFailureOrigin().equals(FailureOrigin.REPLICATION) && f.getStacktrace().contains(PERSIST_ERROR_MESSAGE))); - verify(persistConfigHelper).persistDestinationConfig(Long.valueOf(JOB_ID), connectorConfig); + verify(persistConfigHelper).persistDestinationConfig(Long.valueOf(JOB_ID), CONNECTOR_CONFIG); } @Test From de0d638ee72728709f0b9f8d08d6ebedfb503573 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 30 Nov 2022 15:55:17 -0400 Subject: [PATCH 14/63] fix pmd --- .../job/factory/OAuthConfigSupplierTest.java | 28 ++++++++----------- 1 file changed, 11 insertions(+), 17 deletions(-) diff --git a/airbyte-persistence/job-persistence/src/test/java/io/airbyte/persistence/job/factory/OAuthConfigSupplierTest.java b/airbyte-persistence/job-persistence/src/test/java/io/airbyte/persistence/job/factory/OAuthConfigSupplierTest.java index b87a02c45f59..89d8c55df6ad 100644 --- a/airbyte-persistence/job-persistence/src/test/java/io/airbyte/persistence/job/factory/OAuthConfigSupplierTest.java +++ b/airbyte-persistence/job-persistence/src/test/java/io/airbyte/persistence/job/factory/OAuthConfigSupplierTest.java @@ -48,6 +48,7 @@ class OAuthConfigSupplierTest { private TrackingClient trackingClient; private OAuthConfigSupplier oAuthConfigSupplier; private UUID sourceDefinitionId; + private StandardSourceDefinition testSourceDefinition; @BeforeEach void setup() throws JsonValidationException, ConfigNotFoundException, IOException { @@ -55,6 +56,13 @@ void setup() throws JsonValidationException, ConfigNotFoundException, IOExceptio trackingClient = mock(TrackingClient.class); oAuthConfigSupplier = new OAuthConfigSupplier(configRepository, trackingClient); sourceDefinitionId = UUID.randomUUID(); + testSourceDefinition = new StandardSourceDefinition() + .withSourceDefinitionId(sourceDefinitionId) + .withName("test") + .withDockerRepository("test/test") + .withDockerImageTag("dev") + .withSpec(null); + setupStandardDefinitionMock(createAdvancedAuth() .withPredicateKey(List.of(CREDENTIALS, AUTH_TYPE)) .withPredicateValue(OAUTH)); @@ -202,12 +210,7 @@ void testOAuthFullInjectionBecauseNoOAuthSpec() throws JsonValidationException, final UUID workspaceId = UUID.randomUUID(); final Map oauthParameters = generateOAuthParameters(); when(configRepository.getStandardSourceDefinition(any())) - .thenReturn(new StandardSourceDefinition() - .withSourceDefinitionId(sourceDefinitionId) - .withName("test") - .withDockerRepository("test/test") - .withDockerImageTag("dev") - .withSpec(null)); + .thenReturn(testSourceDefinition); setupOAuthParamMocks(oauthParameters); final JsonNode actualConfig = oAuthConfigSupplier.injectSourceOAuthParameters(sourceDefinitionId, workspaceId, Jsons.clone(config)); final ObjectNode expectedConfig = ((ObjectNode) Jsons.clone(config)); @@ -224,12 +227,7 @@ void testOAuthNoMaskingBecauseNoOAuthSpec() throws JsonValidationException, IOEx final UUID workspaceId = UUID.randomUUID(); final Map oauthParameters = generateOAuthParameters(); when(configRepository.getStandardSourceDefinition(any())) - .thenReturn(new StandardSourceDefinition() - .withSourceDefinitionId(sourceDefinitionId) - .withName("test") - .withDockerRepository("test/test") - .withDockerImageTag("dev") - .withSpec(null)); + .thenReturn(testSourceDefinition); setupOAuthParamMocks(oauthParameters); final JsonNode actualConfig = oAuthConfigSupplier.maskSourceOAuthParameters(sourceDefinitionId, workspaceId, Jsons.clone(config)); assertEquals(config, actualConfig); @@ -314,11 +312,7 @@ private static AdvancedAuth createAdvancedAuth() { } private void setupStandardDefinitionMock(final AdvancedAuth advancedAuth) throws JsonValidationException, ConfigNotFoundException, IOException { - when(configRepository.getStandardSourceDefinition(any())).thenReturn(new StandardSourceDefinition() - .withSourceDefinitionId(sourceDefinitionId) - .withName("test") - .withDockerRepository("test/test") - .withDockerImageTag("dev") + when(configRepository.getStandardSourceDefinition(any())).thenReturn(testSourceDefinition .withSpec(new ConnectorSpecification().withAdvancedAuth(advancedAuth))); } From d159366d9163d1611d3fa144a970f6a7605cf33c Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 30 Nov 2022 16:13:58 -0400 Subject: [PATCH 15/63] fix test --- .../persistence/job/factory/OAuthConfigSupplierTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airbyte-persistence/job-persistence/src/test/java/io/airbyte/persistence/job/factory/OAuthConfigSupplierTest.java b/airbyte-persistence/job-persistence/src/test/java/io/airbyte/persistence/job/factory/OAuthConfigSupplierTest.java index 89d8c55df6ad..5d85505ef8a6 100644 --- a/airbyte-persistence/job-persistence/src/test/java/io/airbyte/persistence/job/factory/OAuthConfigSupplierTest.java +++ b/airbyte-persistence/job-persistence/src/test/java/io/airbyte/persistence/job/factory/OAuthConfigSupplierTest.java @@ -210,7 +210,7 @@ void testOAuthFullInjectionBecauseNoOAuthSpec() throws JsonValidationException, final UUID workspaceId = UUID.randomUUID(); final Map oauthParameters = generateOAuthParameters(); when(configRepository.getStandardSourceDefinition(any())) - .thenReturn(testSourceDefinition); + .thenReturn(testSourceDefinition.withSpec(null)); setupOAuthParamMocks(oauthParameters); final JsonNode actualConfig = oAuthConfigSupplier.injectSourceOAuthParameters(sourceDefinitionId, workspaceId, Jsons.clone(config)); final ObjectNode expectedConfig = ((ObjectNode) Jsons.clone(config)); @@ -227,7 +227,7 @@ void testOAuthNoMaskingBecauseNoOAuthSpec() throws JsonValidationException, IOEx final UUID workspaceId = UUID.randomUUID(); final Map oauthParameters = generateOAuthParameters(); when(configRepository.getStandardSourceDefinition(any())) - .thenReturn(testSourceDefinition); + .thenReturn(testSourceDefinition.withSpec(null)); setupOAuthParamMocks(oauthParameters); final JsonNode actualConfig = oAuthConfigSupplier.maskSourceOAuthParameters(sourceDefinitionId, workspaceId, Jsons.clone(config)); assertEquals(config, actualConfig); From 0f1c122ab120734c51d9a430ff427a8d6093b0bb Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 30 Nov 2022 18:24:38 -0400 Subject: [PATCH 16/63] add PersistConfigHelperTest --- .../helper/PersistConfigHelperTest.java | 115 ++++++++++++++++++ .../sync/PersistStateActivityImpl.java | 1 - 2 files changed, 115 insertions(+), 1 deletion(-) create mode 100644 airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/PersistConfigHelperTest.java diff --git a/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/PersistConfigHelperTest.java b/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/PersistConfigHelperTest.java new file mode 100644 index 000000000000..f81f21ea7d85 --- /dev/null +++ b/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/PersistConfigHelperTest.java @@ -0,0 +1,115 @@ +/* + * Copyright (c) 2022 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.workers.helper; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.fasterxml.jackson.databind.JsonNode; +import io.airbyte.api.client.AirbyteApiClient; +import io.airbyte.api.client.generated.ConnectionApi; +import io.airbyte.api.client.generated.DestinationApi; +import io.airbyte.api.client.generated.JobsApi; +import io.airbyte.api.client.generated.SourceApi; +import io.airbyte.api.client.invoker.generated.ApiException; +import io.airbyte.api.client.model.generated.ConnectionIdRequestBody; +import io.airbyte.api.client.model.generated.ConnectionRead; +import io.airbyte.api.client.model.generated.DestinationIdRequestBody; +import io.airbyte.api.client.model.generated.DestinationRead; +import io.airbyte.api.client.model.generated.DestinationUpdate; +import io.airbyte.api.client.model.generated.JobIdRequestBody; +import io.airbyte.api.client.model.generated.JobInfoLightRead; +import io.airbyte.api.client.model.generated.JobRead; +import io.airbyte.api.client.model.generated.SourceIdRequestBody; +import io.airbyte.api.client.model.generated.SourceRead; +import io.airbyte.api.client.model.generated.SourceUpdate; +import io.airbyte.commons.json.Jsons; +import io.airbyte.protocol.models.Config; +import java.util.UUID; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +public class PersistConfigHelperTest { + + private static final Long JOB_ID = 123L; + private static final UUID CONNECTION_ID = UUID.randomUUID(); + private static final UUID SOURCE_ID = UUID.randomUUID(); + private static final String SOURCE_NAME = "source-stripe"; + private static final UUID DESTINATION_ID = UUID.randomUUID(); + private static final String DESTINATION_NAME = "destination-google-sheets"; + + private final AirbyteApiClient airbyteApiClient = mock(AirbyteApiClient.class); + private final JobsApi mJobsApi = mock(JobsApi.class); + private final ConnectionApi mConnectionApi = mock(ConnectionApi.class); + private final SourceApi mSourceApi = mock(SourceApi.class); + private final DestinationApi mDestinationApi = mock(DestinationApi.class); + + private PersistConfigHelper persistConfigHelper; + + @BeforeEach + void setUp() throws ApiException { + when(airbyteApiClient.getJobsApi()).thenReturn(mJobsApi); + when(airbyteApiClient.getSourceApi()).thenReturn(mSourceApi); + when(airbyteApiClient.getDestinationApi()).thenReturn(mDestinationApi); + when(airbyteApiClient.getConnectionApi()).thenReturn(mConnectionApi); + + when(mJobsApi.getJobInfoLight(new JobIdRequestBody() + .id(JOB_ID))).thenReturn(new JobInfoLightRead() + .job(new JobRead() + .configId(CONNECTION_ID.toString()))); + + when(mConnectionApi.getConnection(new ConnectionIdRequestBody() + .connectionId(CONNECTION_ID))).thenReturn(new ConnectionRead() + .sourceId(SOURCE_ID) + .destinationId(DESTINATION_ID)); + + when(mSourceApi.getSource(new SourceIdRequestBody() + .sourceId(SOURCE_ID))).thenReturn(new SourceRead() + .sourceId(SOURCE_ID) + .name(SOURCE_NAME)); + + when(mDestinationApi.getDestination(new DestinationIdRequestBody() + .destinationId(DESTINATION_ID))).thenReturn(new DestinationRead() + .destinationId(DESTINATION_ID) + .name(DESTINATION_NAME)); + + persistConfigHelper = new PersistConfigHelper(airbyteApiClient); + } + + @Test + void testPersistSourceConfig() throws ApiException { + final Config newConfiguration = new Config().withAdditionalProperty("key", "new_value"); + final JsonNode configJson = Jsons.jsonNode(newConfiguration.getAdditionalProperties()); + + final SourceUpdate expectedSourceUpdate = new SourceUpdate() + .sourceId(SOURCE_ID) + .name(SOURCE_NAME) + .connectionConfiguration(configJson); + + when(mSourceApi.updateSource(Mockito.any())).thenReturn(new SourceRead().connectionConfiguration(configJson)); + + persistConfigHelper.persistSourceConfig(JOB_ID, newConfiguration); + verify(mSourceApi).updateSource(expectedSourceUpdate); + } + + @Test + void testPersistDestinationConfig() throws ApiException { + final Config newConfiguration = new Config().withAdditionalProperty("key", "new_value"); + final JsonNode configJson = Jsons.jsonNode(newConfiguration.getAdditionalProperties()); + + final DestinationUpdate expectedDestinationUpdate = new DestinationUpdate() + .destinationId(DESTINATION_ID) + .name(DESTINATION_NAME) + .connectionConfiguration(configJson); + + when(mDestinationApi.updateDestination(Mockito.any())).thenReturn(new DestinationRead().connectionConfiguration(configJson)); + + persistConfigHelper.persistDestinationConfig(JOB_ID, newConfiguration); + verify(mDestinationApi).updateDestination(expectedDestinationUpdate); + } + +} diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/PersistStateActivityImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/PersistStateActivityImpl.java index 00d3959aad65..f9ed3057066c 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/PersistStateActivityImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/PersistStateActivityImpl.java @@ -48,7 +48,6 @@ public PersistStateActivityImpl(final AirbyteApiClient airbyteApiClient, final F @Override public boolean persist(final UUID connectionId, final StandardSyncOutput syncOutput, final ConfiguredAirbyteCatalog configuredCatalog) { ApmTraceUtils.addTagsToTrace(Map.of(CONNECTION_ID_KEY, connectionId.toString())); - final State state = syncOutput.getState(); if (state != null) { // todo: these validation logic should happen on server side. From 519b9200581eac765314897ee755e94b0381fabb Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 30 Nov 2022 18:30:08 -0400 Subject: [PATCH 17/63] update message tracker comment --- .../io/airbyte/workers/internal/AirbyteMessageTracker.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/AirbyteMessageTracker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/AirbyteMessageTracker.java index 6c9e865612a9..e042ed6420dc 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/AirbyteMessageTracker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/AirbyteMessageTracker.java @@ -238,8 +238,8 @@ private void handleEmittedOrchestratorMessage(final AirbyteControlMessage contro @SuppressWarnings("PMD") // until method is implemented private void handleEmittedOrchestratorConnectorConfig(final AirbyteControlConnectorConfigMessage configMessage, final ConnectorType connectorType) { - // Currently being persisted as part of the DefaultReplicationWorker - // TODO decide if it should be persisted from here instead + // Config updates are being persisted as part of the DefaultNormalizationWorker. + // In the future, we could add tracking of these kinds of messages here. Nothing to do for now. } /** From 4ef8b916b1b069ddf9bbe51cdbd4c637e02c8d32 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 30 Nov 2022 20:18:08 -0400 Subject: [PATCH 18/63] fix pmd --- .../java/io/airbyte/workers/internal/AirbyteMessageTracker.java | 2 +- .../java/io/airbyte/workers/helper/PersistConfigHelperTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/AirbyteMessageTracker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/AirbyteMessageTracker.java index e042ed6420dc..f2c32f6eb24f 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/AirbyteMessageTracker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/internal/AirbyteMessageTracker.java @@ -238,7 +238,7 @@ private void handleEmittedOrchestratorMessage(final AirbyteControlMessage contro @SuppressWarnings("PMD") // until method is implemented private void handleEmittedOrchestratorConnectorConfig(final AirbyteControlConnectorConfigMessage configMessage, final ConnectorType connectorType) { - // Config updates are being persisted as part of the DefaultNormalizationWorker. + // Config updates are being persisted as part of the DefaultReplicationWorker. // In the future, we could add tracking of these kinds of messages here. Nothing to do for now. } diff --git a/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/PersistConfigHelperTest.java b/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/PersistConfigHelperTest.java index f81f21ea7d85..e2d77ba8e06a 100644 --- a/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/PersistConfigHelperTest.java +++ b/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/PersistConfigHelperTest.java @@ -33,7 +33,7 @@ import org.junit.jupiter.api.Test; import org.mockito.Mockito; -public class PersistConfigHelperTest { +class PersistConfigHelperTest { private static final Long JOB_ID = 123L; private static final UUID CONNECTION_ID = UUID.randomUUID(); From dda5861daab3bed202356aa0040f6b06b667b000 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 30 Nov 2022 20:26:01 -0400 Subject: [PATCH 19/63] format --- .../orchestrator/ReplicationJobOrchestrator.java | 1 - 1 file changed, 1 deletion(-) diff --git a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java index 44a360f36c94..75178179b611 100644 --- a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java +++ b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java @@ -32,7 +32,6 @@ import io.airbyte.workers.WorkerMetricReporter; import io.airbyte.workers.WorkerUtils; import io.airbyte.workers.general.DefaultReplicationWorker; -import io.airbyte.workers.general.ReplicationWorker; import io.airbyte.workers.helper.PersistConfigHelper; import io.airbyte.workers.internal.AirbyteMessageTracker; import io.airbyte.workers.internal.AirbyteStreamFactory; From c30e8f64002d4042121b3b9b6284d75cfc054ea1 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 30 Nov 2022 21:46:17 -0400 Subject: [PATCH 20/63] move ApiClientBeanFactory to commons-worker, use in container-orchestrator --- airbyte-commons-worker/build.gradle | 1 + .../workers/config/ApiClientBeanFactory.java | 0 .../config/ContainerOrchestratorFactory.java | 6 ++++-- .../ReplicationJobOrchestrator.java | 17 +++++------------ .../src/main/resources/application.yml | 16 +++++++++++++++- .../ContainerOrchestratorFactoryTest.java | 14 +++++++++----- .../src/test/resources/application.yml | 16 +++++++++++++++- 7 files changed, 49 insertions(+), 21 deletions(-) rename {airbyte-workers => airbyte-commons-worker}/src/main/java/io/airbyte/workers/config/ApiClientBeanFactory.java (100%) diff --git a/airbyte-commons-worker/build.gradle b/airbyte-commons-worker/build.gradle index 8051e8038d0d..13bae0878a66 100644 --- a/airbyte-commons-worker/build.gradle +++ b/airbyte-commons-worker/build.gradle @@ -10,6 +10,7 @@ dependencies { implementation libs.bundles.micronaut implementation 'io.fabric8:kubernetes-client:5.12.2' + implementation 'com.auth0:java-jwt:3.19.2' implementation libs.guava implementation (libs.temporal.sdk) { exclude module: 'guava' diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/config/ApiClientBeanFactory.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/config/ApiClientBeanFactory.java similarity index 100% rename from airbyte-workers/src/main/java/io/airbyte/workers/config/ApiClientBeanFactory.java rename to airbyte-commons-worker/src/main/java/io/airbyte/workers/config/ApiClientBeanFactory.java diff --git a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/config/ContainerOrchestratorFactory.java b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/config/ContainerOrchestratorFactory.java index 69d8cd74b836..e266670de9e5 100644 --- a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/config/ContainerOrchestratorFactory.java +++ b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/config/ContainerOrchestratorFactory.java @@ -4,6 +4,7 @@ package io.airbyte.container_orchestrator.config; +import io.airbyte.api.client.AirbyteApiClient; import io.airbyte.commons.features.EnvVariableFeatureFlags; import io.airbyte.commons.features.FeatureFlags; import io.airbyte.commons.protocol.AirbyteMessageSerDeProvider; @@ -100,10 +101,11 @@ JobOrchestrator jobOrchestrator( final WorkerConfigs workerConfigs, final AirbyteMessageSerDeProvider serdeProvider, final AirbyteMessageVersionedMigratorFactory migratorFactory, - final JobRunConfig jobRunConfig) { + final JobRunConfig jobRunConfig, + final AirbyteApiClient airbyteApiClient) { return switch (application) { case ReplicationLauncherWorker.REPLICATION -> new ReplicationJobOrchestrator(envConfigs, processFactory, featureFlags, serdeProvider, - migratorFactory, jobRunConfig); + migratorFactory, jobRunConfig, airbyteApiClient); case NormalizationLauncherWorker.NORMALIZATION -> new NormalizationJobOrchestrator(envConfigs, processFactory, jobRunConfig); case DbtLauncherWorker.DBT -> new DbtJobOrchestrator(envConfigs, workerConfigs, processFactory, jobRunConfig); case AsyncOrchestratorPodProcess.NO_OP -> new NoOpOrchestrator(); diff --git a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java index 75178179b611..63edd9176b9e 100644 --- a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java +++ b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java @@ -11,7 +11,6 @@ import datadog.trace.api.Trace; import io.airbyte.api.client.AirbyteApiClient; -import io.airbyte.api.client.invoker.generated.ApiClient; import io.airbyte.commons.features.FeatureFlags; import io.airbyte.commons.json.Jsons; import io.airbyte.commons.logging.MdcScope; @@ -62,19 +61,22 @@ public class ReplicationJobOrchestrator implements JobOrchestrator runJob() throws Exception { new AirbyteMessageTracker(), new RecordSchemaValidator(WorkerUtils.mapStreamNamesToSchemas(syncInput)), metricReporter, - new PersistConfigHelper(getAirbyteApiClient())); + new PersistConfigHelper(airbyteApiClient)); log.info("Running replication worker..."); final var jobRoot = TemporalUtils.getJobRoot(configs.getWorkspaceRoot(), @@ -157,15 +159,6 @@ public Optional runJob() throws Exception { return Optional.of(Jsons.serialize(replicationOutput)); } - private AirbyteApiClient getAirbyteApiClient() { - // TODO make this actually functional - return new AirbyteApiClient( - new ApiClient().setScheme("http") - .setHost("airbyte-server-svc") - .setPort(8001) - .setBasePath("/api")); - } - private AirbyteStreamFactory getStreamFactory(final Version protocolVersion, final MdcScope.Builder mdcScope) { return protocolVersion != null ? new VersionedAirbyteStreamFactory(serDeProvider, migratorFactory, protocolVersion, mdcScope) diff --git a/airbyte-container-orchestrator/src/main/resources/application.yml b/airbyte-container-orchestrator/src/main/resources/application.yml index e13a964cc13e..42263e9a7382 100644 --- a/airbyte-container-orchestrator/src/main/resources/application.yml +++ b/airbyte-container-orchestrator/src/main/resources/application.yml @@ -3,4 +3,18 @@ micronaut: port: 9000 airbyte: - config-dir: /config \ No newline at end of file + config-dir: /config + control: + plane: + auth-endpoint: ${CONTROL_PLANE_AUTH_ENDPOINT:} + data: + plane: + service-account: + credentials-path: ${DATA_PLANE_SERVICE_ACCOUNT_CREDENTIALS_PATH:} + email: ${DATA_PLANE_SERVICE_ACCOUNT_EMAIL:} + internal: + api: + auth-header: + name: ${AIRBYTE_API_AUTH_HEADER_NAME:} + value: ${AIRBYTE_API_AUTH_HEADER_VALUE:} + host: ${INTERNAL_API_HOST} \ No newline at end of file diff --git a/airbyte-container-orchestrator/src/test/java/io/airbyte/container_orchestrator/config/ContainerOrchestratorFactoryTest.java b/airbyte-container-orchestrator/src/test/java/io/airbyte/container_orchestrator/config/ContainerOrchestratorFactoryTest.java index d99a521fae99..64b468f85206 100644 --- a/airbyte-container-orchestrator/src/test/java/io/airbyte/container_orchestrator/config/ContainerOrchestratorFactoryTest.java +++ b/airbyte-container-orchestrator/src/test/java/io/airbyte/container_orchestrator/config/ContainerOrchestratorFactoryTest.java @@ -9,6 +9,7 @@ import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; +import io.airbyte.api.client.AirbyteApiClient; import io.airbyte.commons.features.FeatureFlags; import io.airbyte.commons.protocol.AirbyteMessageSerDeProvider; import io.airbyte.commons.protocol.AirbyteMessageVersionedMigratorFactory; @@ -49,6 +50,9 @@ class ContainerOrchestratorFactoryTest { @Inject JobRunConfig jobRunConfig; + @Inject + AirbyteApiClient airbyteApiClient; + // Tests will fail if this is uncommented, due to how the implementation of the DocumentStoreClient // is being created // @Inject @@ -87,29 +91,29 @@ void jobOrchestrator() { final var repl = factory.jobOrchestrator( ReplicationLauncherWorker.REPLICATION, envConfigs, processFactory, featureFlags, workerConfigs, - airbyteMessageSerDeProvider, airbyteMessageVersionedMigratorFactory, jobRunConfig); + airbyteMessageSerDeProvider, airbyteMessageVersionedMigratorFactory, jobRunConfig, airbyteApiClient); assertEquals("Replication", repl.getOrchestratorName()); final var norm = factory.jobOrchestrator( NormalizationLauncherWorker.NORMALIZATION, envConfigs, processFactory, featureFlags, workerConfigs, - airbyteMessageSerDeProvider, airbyteMessageVersionedMigratorFactory, jobRunConfig); + airbyteMessageSerDeProvider, airbyteMessageVersionedMigratorFactory, jobRunConfig, airbyteApiClient); assertEquals("Normalization", norm.getOrchestratorName()); final var dbt = factory.jobOrchestrator( DbtLauncherWorker.DBT, envConfigs, processFactory, featureFlags, workerConfigs, - airbyteMessageSerDeProvider, airbyteMessageVersionedMigratorFactory, jobRunConfig); + airbyteMessageSerDeProvider, airbyteMessageVersionedMigratorFactory, jobRunConfig, airbyteApiClient); assertEquals("DBT Transformation", dbt.getOrchestratorName()); final var noop = factory.jobOrchestrator( AsyncOrchestratorPodProcess.NO_OP, envConfigs, processFactory, featureFlags, workerConfigs, - airbyteMessageSerDeProvider, airbyteMessageVersionedMigratorFactory, jobRunConfig); + airbyteMessageSerDeProvider, airbyteMessageVersionedMigratorFactory, jobRunConfig, airbyteApiClient); assertEquals("NO_OP", noop.getOrchestratorName()); var caught = false; try { factory.jobOrchestrator( "does not exist", envConfigs, processFactory, featureFlags, workerConfigs, - airbyteMessageSerDeProvider, airbyteMessageVersionedMigratorFactory, jobRunConfig); + airbyteMessageSerDeProvider, airbyteMessageVersionedMigratorFactory, jobRunConfig, airbyteApiClient); } catch (final Exception e) { caught = true; } diff --git a/airbyte-container-orchestrator/src/test/resources/application.yml b/airbyte-container-orchestrator/src/test/resources/application.yml index a534edb590c6..8856e9fab93c 100644 --- a/airbyte-container-orchestrator/src/test/resources/application.yml +++ b/airbyte-container-orchestrator/src/test/resources/application.yml @@ -3,4 +3,18 @@ micronaut: port: 9000 airbyte: - config-dir: src/test/resources/files \ No newline at end of file + config-dir: src/test/resources/files + control: + plane: + auth-endpoint: "" + data: + plane: + service-account: + credentials-path: "" + email: "" + internal: + api: + auth-header: + name: "" + value: "" + host: airbyte-server-svc:8001 \ No newline at end of file From 51f19765184523e0ae88d8ffceaaad6e39a772e1 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Tue, 6 Dec 2022 01:06:01 -0400 Subject: [PATCH 21/63] pull out config updating to separate methods --- .../general/DefaultReplicationWorker.java | 47 +++++++++++-------- 1 file changed, 28 insertions(+), 19 deletions(-) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java index 37bea8d50c0e..0aa0a7f7fd20 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java @@ -262,16 +262,7 @@ private static Runnable readFromDstRunnable(final AirbyteDestination destination final AirbyteMessage message = messageOptional.get(); LOGGER.info("State in DefaultReplicationWorker from destination: {}", message); messageTracker.acceptFromDestination(message); - - try { - if (message.getType() == Type.CONTROL && message.getControl().getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG) { - persistConfigHelper.persistDestinationConfig(jobId, message.getControl().getConnectorConfig().getConfig()); - } - } catch (final ApiException e) { - LOGGER.error("Error trying to save updated destination config", e); - throw new RuntimeException(e); - } - + acceptAndUpdateDstConfig(jobId, message, persistConfigHelper); } } timeHolder.trackDestinationWriteEndTime(); @@ -328,15 +319,7 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou final AirbyteMessage message = mapper.mapMessage(airbyteMessage); messageTracker.acceptFromSource(message); - - if (message.getType() == Type.CONTROL && message.getControl().getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG) { - try { - persistConfigHelper.persistSourceConfig(jobId, message.getControl().getConnectorConfig().getConfig()); - } catch (final ApiException e) { - LOGGER.error("Error trying to save updated source config", e); - throw new RuntimeException(e); - } - } + acceptAndUpdateSrcConfig(jobId, message, persistConfigHelper); try { if (message.getType() == Type.RECORD || message.getType() == Type.STATE) { @@ -396,6 +379,32 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou }; } + private static boolean isControlConfigMsg(final AirbyteMessage message) { + return message.getType() == Type.CONTROL && message.getControl().getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG; + } + + private static void acceptAndUpdateSrcConfig(final Long jobId, final AirbyteMessage message, final PersistConfigHelper persistConfigHelper) { + if (isControlConfigMsg(message)) { + try { + persistConfigHelper.persistSourceConfig(jobId, message.getControl().getConnectorConfig().getConfig()); + } catch (final ApiException e) { + LOGGER.error("Error trying to save updated source config", e); + throw new RuntimeException(e); + } + } + } + + private static void acceptAndUpdateDstConfig(final Long jobId, final AirbyteMessage message, final PersistConfigHelper persistConfigHelper) { + try { + if (isControlConfigMsg(message)) { + persistConfigHelper.persistDestinationConfig(jobId, message.getControl().getConnectorConfig().getConfig()); + } + } catch (final ApiException e) { + LOGGER.error("Error trying to save updated destination config", e); + throw new RuntimeException(e); + } + } + private ReplicationOutput getReplicationOutput(final StandardSyncInput syncInput, final WorkerDestinationConfig destinationConfig, final AtomicReference replicationRunnableFailureRef, From 8203432df13dd43adf060dc7704ce68de0da27ef Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Tue, 6 Dec 2022 01:23:02 -0400 Subject: [PATCH 22/63] add jitter --- .../general/DefaultReplicationWorker.java | 5 +- .../workers/helper/PersistConfigHelper.java | 55 ++++++++++++------- 2 files changed, 36 insertions(+), 24 deletions(-) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java index 0aa0a7f7fd20..ec5f310acc2c 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java @@ -12,7 +12,6 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import datadog.trace.api.Trace; -import io.airbyte.api.client.invoker.generated.ApiException; import io.airbyte.commons.io.LineGobbler; import io.airbyte.config.FailureReason; import io.airbyte.config.ReplicationAttemptSummary; @@ -387,7 +386,7 @@ private static void acceptAndUpdateSrcConfig(final Long jobId, final AirbyteMess if (isControlConfigMsg(message)) { try { persistConfigHelper.persistSourceConfig(jobId, message.getControl().getConnectorConfig().getConfig()); - } catch (final ApiException e) { + } catch (final Exception e) { LOGGER.error("Error trying to save updated source config", e); throw new RuntimeException(e); } @@ -399,7 +398,7 @@ private static void acceptAndUpdateDstConfig(final Long jobId, final AirbyteMess if (isControlConfigMsg(message)) { persistConfigHelper.persistDestinationConfig(jobId, message.getControl().getConnectorConfig().getConfig()); } - } catch (final ApiException e) { + } catch (final Exception e) { LOGGER.error("Error trying to save updated destination config", e); throw new RuntimeException(e); } diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/PersistConfigHelper.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/PersistConfigHelper.java index e5f854ee0f6f..316d15cf2a31 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/PersistConfigHelper.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/PersistConfigHelper.java @@ -9,7 +9,6 @@ import io.airbyte.api.client.generated.DestinationApi; import io.airbyte.api.client.generated.JobsApi; import io.airbyte.api.client.generated.SourceApi; -import io.airbyte.api.client.invoker.generated.ApiException; import io.airbyte.api.client.model.generated.ConnectionIdRequestBody; import io.airbyte.api.client.model.generated.ConnectionRead; import io.airbyte.api.client.model.generated.DestinationIdRequestBody; @@ -41,47 +40,61 @@ public PersistConfigHelper(final AirbyteApiClient apiClient) { this.apiClient = apiClient; } - private UUID getConnectionIdFromJobId(final Long jobId) throws ApiException { + private UUID getConnectionIdFromJobId(final Long jobId) { final JobsApi jobsApi = apiClient.getJobsApi(); final JobIdRequestBody body = new JobIdRequestBody().id(jobId); - final JobInfoLightRead jobInfo = jobsApi.getJobInfoLight(body); + final JobInfoLightRead jobInfo = AirbyteApiClient.retryWithJitter( + () -> jobsApi.getJobInfoLight(body), + "get job info"); return UUID.fromString(jobInfo.getJob().getConfigId()); } - public void persistSourceConfig(final Long jobId, final Config config) throws ApiException { + public void persistSourceConfig(final Long jobId, final Config config) { final UUID connectionId = getConnectionIdFromJobId(jobId); - final ConnectionRead connection = apiClient.getConnectionApi().getConnection(new ConnectionIdRequestBody().connectionId(connectionId)); + final ConnectionRead connection = AirbyteApiClient.retryWithJitter( + () -> apiClient.getConnectionApi().getConnection(new ConnectionIdRequestBody().connectionId(connectionId)), + "get connection"); final UUID sourceId = connection.getSourceId(); final SourceApi sourceApi = apiClient.getSourceApi(); - final SourceRead source = sourceApi.getSource(new SourceIdRequestBody().sourceId(sourceId)); - - final SourceRead updatedSource = sourceApi - .updateSource(new SourceUpdate() - .sourceId(sourceId) - .name(source.getName()) - .connectionConfiguration(Jsons.jsonNode(config.getAdditionalProperties()))); + final SourceRead source = AirbyteApiClient.retryWithJitter( + () -> sourceApi.getSource(new SourceIdRequestBody().sourceId(sourceId)), + "get source"); + + final SourceRead updatedSource = AirbyteApiClient.retryWithJitter( + () -> sourceApi + .updateSource(new SourceUpdate() + .sourceId(sourceId) + .name(source.getName()) + .connectionConfiguration(Jsons.jsonNode(config.getAdditionalProperties()))), + "update source"); LOGGER.info("Persisted updated configuration for source {}. New config hash: {}.", sourceId, Hashing.sha256().hashString(updatedSource.getConnectionConfiguration().asText(), StandardCharsets.UTF_8)); } - public void persistDestinationConfig(final Long jobId, final Config config) throws ApiException { + public void persistDestinationConfig(final Long jobId, final Config config) { final UUID connectionId = getConnectionIdFromJobId(jobId); - final ConnectionRead connection = apiClient.getConnectionApi().getConnection(new ConnectionIdRequestBody().connectionId(connectionId)); + final ConnectionRead connection = AirbyteApiClient.retryWithJitter( + () -> apiClient.getConnectionApi().getConnection(new ConnectionIdRequestBody().connectionId(connectionId)), + "get connection"); final UUID destinationId = connection.getDestinationId(); final DestinationApi destinationApi = apiClient.getDestinationApi(); - final DestinationRead destination = destinationApi.getDestination(new DestinationIdRequestBody().destinationId(destinationId)); - - final DestinationRead updatedDestination = destinationApi - .updateDestination(new DestinationUpdate() - .destinationId(destinationId) - .name(destination.getName()) - .connectionConfiguration(Jsons.jsonNode(config.getAdditionalProperties()))); + final DestinationRead destination = AirbyteApiClient.retryWithJitter( + () -> destinationApi.getDestination(new DestinationIdRequestBody().destinationId(destinationId)), + "get destination"); + + final DestinationRead updatedDestination = AirbyteApiClient.retryWithJitter( + () -> destinationApi + .updateDestination(new DestinationUpdate() + .destinationId(destinationId) + .name(destination.getName()) + .connectionConfiguration(Jsons.jsonNode(config.getAdditionalProperties()))), + "update destination"); LOGGER.info("Persisted updated configuration for destination {}. New config hash: {}.", destinationId, Hashing.sha256().hashString(updatedDestination.getConnectionConfiguration().asText(), StandardCharsets.UTF_8)); From bff1d44900f3e682c37eb4adc6f9df6db77b2248 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Tue, 6 Dec 2022 01:35:37 -0400 Subject: [PATCH 23/63] rename PersistConfigHelper -> UpdateConnectorConfigHelper, docs --- .../general/DefaultReplicationWorker.java | 33 ++++++---- ....java => UpdateConnectorConfigHelper.java} | 22 +++++-- .../general/DefaultReplicationWorkerTest.java | 64 +++++++++---------- ...a => UpdateConnectorConfigHelperTest.java} | 10 +-- .../ReplicationJobOrchestrator.java | 4 +- .../sync/ReplicationActivityImpl.java | 4 +- 6 files changed, 77 insertions(+), 60 deletions(-) rename airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/{PersistConfigHelper.java => UpdateConnectorConfigHelper.java} (82%) rename airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/{PersistConfigHelperTest.java => UpdateConnectorConfigHelperTest.java} (93%) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java index ec5f310acc2c..e0bbedada829 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java @@ -35,8 +35,8 @@ import io.airbyte.workers.exception.RecordSchemaValidationException; import io.airbyte.workers.exception.WorkerException; import io.airbyte.workers.helper.FailureHelper; -import io.airbyte.workers.helper.PersistConfigHelper; import io.airbyte.workers.helper.ThreadedTimeTracker; +import io.airbyte.workers.helper.UpdateConnectorConfigHelper; import io.airbyte.workers.internal.AirbyteDestination; import io.airbyte.workers.internal.AirbyteMapper; import io.airbyte.workers.internal.AirbyteSource; @@ -96,7 +96,7 @@ public class DefaultReplicationWorker implements ReplicationWorker { private final AtomicBoolean hasFailed; private final RecordSchemaValidator recordSchemaValidator; private final WorkerMetricReporter metricReporter; - private final PersistConfigHelper persistConfigHelper; + private final UpdateConnectorConfigHelper updateConnectorConfigHelper; public DefaultReplicationWorker(final String jobId, final int attempt, @@ -106,7 +106,7 @@ public DefaultReplicationWorker(final String jobId, final MessageTracker messageTracker, final RecordSchemaValidator recordSchemaValidator, final WorkerMetricReporter metricReporter, - final PersistConfigHelper persistConfigHelper) { + final UpdateConnectorConfigHelper updateConnectorConfigHelper) { this.jobId = jobId; this.attempt = attempt; this.source = source; @@ -116,7 +116,7 @@ public DefaultReplicationWorker(final String jobId, this.executors = Executors.newFixedThreadPool(2); this.recordSchemaValidator = recordSchemaValidator; this.metricReporter = metricReporter; - this.persistConfigHelper = persistConfigHelper; + this.updateConnectorConfigHelper = updateConnectorConfigHelper; this.cancelled = new AtomicBoolean(false); this.hasFailed = new AtomicBoolean(false); @@ -189,7 +189,7 @@ private void replicate(final Path jobRoot, // note: `whenComplete` is used instead of `exceptionally` so that the original exception is still // thrown final CompletableFuture readFromDstThread = CompletableFuture.runAsync( - readFromDstRunnable(destination, cancelled, messageTracker, persistConfigHelper, mdc, timeTracker, Long.valueOf(jobId)), + readFromDstRunnable(destination, cancelled, messageTracker, updateConnectorConfigHelper, mdc, timeTracker, Long.valueOf(jobId)), executors) .whenComplete((msg, ex) -> { if (ex != null) { @@ -203,7 +203,8 @@ private void replicate(final Path jobRoot, }); final CompletableFuture readSrcAndWriteDstThread = CompletableFuture.runAsync( - readFromSrcAndWriteToDstRunnable(source, destination, cancelled, mapper, messageTracker, persistConfigHelper, mdc, recordSchemaValidator, + readFromSrcAndWriteToDstRunnable(source, destination, cancelled, mapper, messageTracker, updateConnectorConfigHelper, mdc, + recordSchemaValidator, metricReporter, timeTracker, Long.valueOf(jobId)), executors) @@ -242,7 +243,7 @@ private void replicate(final Path jobRoot, private static Runnable readFromDstRunnable(final AirbyteDestination destination, final AtomicBoolean cancelled, final MessageTracker messageTracker, - final PersistConfigHelper persistConfigHelper, + final UpdateConnectorConfigHelper updateConnectorConfigHelper, final Map mdc, final ThreadedTimeTracker timeHolder, final Long jobId) { @@ -261,7 +262,7 @@ private static Runnable readFromDstRunnable(final AirbyteDestination destination final AirbyteMessage message = messageOptional.get(); LOGGER.info("State in DefaultReplicationWorker from destination: {}", message); messageTracker.acceptFromDestination(message); - acceptAndUpdateDstConfig(jobId, message, persistConfigHelper); + acceptAndUpdateDstConfig(jobId, message, updateConnectorConfigHelper); } } timeHolder.trackDestinationWriteEndTime(); @@ -292,7 +293,7 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou final AtomicBoolean cancelled, final AirbyteMapper mapper, final MessageTracker messageTracker, - final PersistConfigHelper persistConfigHelper, + final UpdateConnectorConfigHelper updateConnectorConfigHelper, final Map mdc, final RecordSchemaValidator recordSchemaValidator, final WorkerMetricReporter metricReporter, @@ -318,7 +319,7 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou final AirbyteMessage message = mapper.mapMessage(airbyteMessage); messageTracker.acceptFromSource(message); - acceptAndUpdateSrcConfig(jobId, message, persistConfigHelper); + acceptAndUpdateSrcConfig(jobId, message, updateConnectorConfigHelper); try { if (message.getType() == Type.RECORD || message.getType() == Type.STATE) { @@ -382,10 +383,12 @@ private static boolean isControlConfigMsg(final AirbyteMessage message) { return message.getType() == Type.CONTROL && message.getControl().getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG; } - private static void acceptAndUpdateSrcConfig(final Long jobId, final AirbyteMessage message, final PersistConfigHelper persistConfigHelper) { + private static void acceptAndUpdateSrcConfig(final Long jobId, + final AirbyteMessage message, + final UpdateConnectorConfigHelper updateConnectorConfigHelper) { if (isControlConfigMsg(message)) { try { - persistConfigHelper.persistSourceConfig(jobId, message.getControl().getConnectorConfig().getConfig()); + updateConnectorConfigHelper.updateSource(jobId, message.getControl().getConnectorConfig().getConfig()); } catch (final Exception e) { LOGGER.error("Error trying to save updated source config", e); throw new RuntimeException(e); @@ -393,10 +396,12 @@ private static void acceptAndUpdateSrcConfig(final Long jobId, final AirbyteMess } } - private static void acceptAndUpdateDstConfig(final Long jobId, final AirbyteMessage message, final PersistConfigHelper persistConfigHelper) { + private static void acceptAndUpdateDstConfig(final Long jobId, + final AirbyteMessage message, + final UpdateConnectorConfigHelper updateConnectorConfigHelper) { try { if (isControlConfigMsg(message)) { - persistConfigHelper.persistDestinationConfig(jobId, message.getControl().getConnectorConfig().getConfig()); + updateConnectorConfigHelper.updateDestination(jobId, message.getControl().getConnectorConfig().getConfig()); } } catch (final Exception e) { LOGGER.error("Error trying to save updated destination config", e); diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/PersistConfigHelper.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/UpdateConnectorConfigHelper.java similarity index 82% rename from airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/PersistConfigHelper.java rename to airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/UpdateConnectorConfigHelper.java index 316d15cf2a31..bff3684a3595 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/PersistConfigHelper.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/UpdateConnectorConfigHelper.java @@ -29,14 +29,18 @@ /** * Helper class for workers to persist updates to Source/Destination configs emitted from * AirbyteControlMessages. + * + * This is in order to support connectors updating configs when running commands, which is specially + * useful for migrating configuration to a new version or for enabling connectors that require + * single-use or short-lived OAuth tokens. */ -public class PersistConfigHelper { +public class UpdateConnectorConfigHelper { - private static final Logger LOGGER = LoggerFactory.getLogger(PersistConfigHelper.class); + private static final Logger LOGGER = LoggerFactory.getLogger(UpdateConnectorConfigHelper.class); private final AirbyteApiClient apiClient; - public PersistConfigHelper(final AirbyteApiClient apiClient) { + public UpdateConnectorConfigHelper(final AirbyteApiClient apiClient) { this.apiClient = apiClient; } @@ -49,7 +53,11 @@ private UUID getConnectionIdFromJobId(final Long jobId) { return UUID.fromString(jobInfo.getJob().getConfigId()); } - public void persistSourceConfig(final Long jobId, final Config config) { + /** + * Updates the Source from a sync job ID with the provided Configuration. Secrets and OAuth + * parameters will be masked when saving. + */ + public void updateSource(final Long jobId, final Config config) { final UUID connectionId = getConnectionIdFromJobId(jobId); final ConnectionRead connection = AirbyteApiClient.retryWithJitter( @@ -75,7 +83,11 @@ public void persistSourceConfig(final Long jobId, final Config config) { } - public void persistDestinationConfig(final Long jobId, final Config config) { + /** + * Updates the Destination from a sync job ID with the provided Configuration. Secrets and OAuth + * parameters will be masked when saving. + */ + public void updateDestination(final Long jobId, final Config config) { final UUID connectionId = getConnectionIdFromJobId(jobId); final ConnectionRead connection = AirbyteApiClient.retryWithJitter( diff --git a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java index 3a59bc3362fe..be960f2b8a1d 100644 --- a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java +++ b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java @@ -51,7 +51,7 @@ import io.airbyte.workers.*; import io.airbyte.workers.exception.WorkerException; import io.airbyte.workers.helper.FailureHelper; -import io.airbyte.workers.helper.PersistConfigHelper; +import io.airbyte.workers.helper.UpdateConnectorConfigHelper; import io.airbyte.workers.internal.AirbyteDestination; import io.airbyte.workers.internal.AirbyteMessageTracker; import io.airbyte.workers.internal.AirbyteSource; @@ -111,7 +111,7 @@ class DefaultReplicationWorkerTest { private RecordSchemaValidator recordSchemaValidator; private MetricClient metricClient; private WorkerMetricReporter workerMetricReporter; - private PersistConfigHelper persistConfigHelper; + private UpdateConnectorConfigHelper updateConnectorConfigHelper; @SuppressWarnings("unchecked") @BeforeEach @@ -131,7 +131,7 @@ void setup() throws Exception { destination = mock(AirbyteDestination.class); messageTracker = mock(AirbyteMessageTracker.class); recordSchemaValidator = mock(RecordSchemaValidator.class); - persistConfigHelper = mock(PersistConfigHelper.class); + updateConnectorConfigHelper = mock(UpdateConnectorConfigHelper.class); metricClient = MetricClientFactory.getMetricClient(); workerMetricReporter = new WorkerMetricReporter(metricClient, "docker_image:v1.0.0"); @@ -162,7 +162,7 @@ void test() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); worker.run(syncInput, jobRoot); @@ -191,7 +191,7 @@ void testInvalidSchema() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); worker.run(syncInput, jobRoot); @@ -222,7 +222,7 @@ void testSourceNonZeroExitValue() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); assertTrue(output.getFailures().stream().anyMatch(f -> f.getFailureOrigin().equals(FailureOrigin.SOURCE))); @@ -243,7 +243,7 @@ void testReplicationRunnableSourceFailure() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); @@ -264,12 +264,12 @@ void testReplicationRunnableSourceUpdateConfig() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.COMPLETED, output.getReplicationAttemptSummary().getStatus()); - verify(persistConfigHelper).persistSourceConfig(Long.valueOf(JOB_ID), CONNECTOR_CONFIG); + verify(updateConnectorConfigHelper).updateSource(Long.valueOf(JOB_ID), CONNECTOR_CONFIG); } @Test @@ -278,7 +278,7 @@ void testReplicationFailureOnSourceConfigPersistError() throws Exception { when(source.isFinished()).thenReturn(false, true); final String PERSIST_ERROR_MESSAGE = "there was a problem persisting the new config"; - doThrow(new ApiException(PERSIST_ERROR_MESSAGE)).when(persistConfigHelper).persistSourceConfig(Mockito.any(), Mockito.any()); + doThrow(new ApiException(PERSIST_ERROR_MESSAGE)).when(updateConnectorConfigHelper).updateSource(Mockito.any(), Mockito.any()); final ReplicationWorker worker = new DefaultReplicationWorker( JOB_ID, @@ -289,14 +289,14 @@ void testReplicationFailureOnSourceConfigPersistError() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); assertTrue(output.getFailures().stream() .anyMatch(f -> f.getFailureOrigin().equals(FailureOrigin.REPLICATION) && f.getStacktrace().contains(PERSIST_ERROR_MESSAGE))); - verify(persistConfigHelper).persistSourceConfig(Long.valueOf(JOB_ID), CONNECTOR_CONFIG); + verify(updateConnectorConfigHelper).updateSource(Long.valueOf(JOB_ID), CONNECTOR_CONFIG); } @Test @@ -313,12 +313,12 @@ void testReplicationRunnableDestinationUpdateConfig() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.COMPLETED, output.getReplicationAttemptSummary().getStatus()); - verify(persistConfigHelper).persistDestinationConfig(Long.valueOf(JOB_ID), CONNECTOR_CONFIG); + verify(updateConnectorConfigHelper).updateDestination(Long.valueOf(JOB_ID), CONNECTOR_CONFIG); } @Test @@ -327,7 +327,7 @@ void testReplicationFailureOnDestinationConfigPersistError() throws Exception { when(destination.isFinished()).thenReturn(false, true); final String PERSIST_ERROR_MESSAGE = "there was a problem persisting the new config"; - doThrow(new ApiException(PERSIST_ERROR_MESSAGE)).when(persistConfigHelper).persistDestinationConfig(Mockito.any(), Mockito.any()); + doThrow(new ApiException(PERSIST_ERROR_MESSAGE)).when(updateConnectorConfigHelper).updateDestination(Mockito.any(), Mockito.any()); final ReplicationWorker worker = new DefaultReplicationWorker( JOB_ID, @@ -338,14 +338,14 @@ void testReplicationFailureOnDestinationConfigPersistError() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); assertTrue(output.getFailures().stream() .anyMatch(f -> f.getFailureOrigin().equals(FailureOrigin.REPLICATION) && f.getStacktrace().contains(PERSIST_ERROR_MESSAGE))); - verify(persistConfigHelper).persistDestinationConfig(Long.valueOf(JOB_ID), CONNECTOR_CONFIG); + verify(updateConnectorConfigHelper).updateDestination(Long.valueOf(JOB_ID), CONNECTOR_CONFIG); } @Test @@ -363,7 +363,7 @@ void testReplicationRunnableDestinationFailure() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); @@ -385,7 +385,7 @@ void testReplicationRunnableDestinationFailureViaTraceMessage() throws Exception messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertTrue(output.getFailures().stream() @@ -408,7 +408,7 @@ void testReplicationRunnableWorkerFailure() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); @@ -435,7 +435,7 @@ void testOnlyStateAndRecordMessagesDeliveredToDestination() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); worker.run(syncInput, jobRoot); @@ -460,7 +460,7 @@ void testDestinationNonZeroExitValue() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); @@ -482,7 +482,7 @@ void testDestinationRunnableDestinationFailure() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); @@ -505,7 +505,7 @@ void testDestinationRunnableWorkerFailure() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); @@ -529,7 +529,7 @@ void testLoggingInThreads() throws IOException, WorkerException { messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); worker.run(syncInput, jobRoot); @@ -571,7 +571,7 @@ void testCancellation() throws InterruptedException { messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); final Thread workerThread = new Thread(() -> { try { @@ -620,7 +620,7 @@ void testPopulatesOutputOnSuccess() throws WorkerException { messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); final ReplicationOutput actual = worker.run(syncInput, jobRoot); final ReplicationOutput replicationOutput = new ReplicationOutput() @@ -688,7 +688,7 @@ void testPopulatesStateOnFailureIfAvailable() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); final ReplicationOutput actual = worker.run(syncInput, jobRoot); assertNotNull(actual); @@ -708,7 +708,7 @@ void testRetainsStateOnFailureIfNewStateNotAvailable() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); final ReplicationOutput actual = worker.run(syncInput, jobRoot); @@ -743,7 +743,7 @@ void testPopulatesStatsOnFailureIfAvailable() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); final ReplicationOutput actual = worker.run(syncInput, jobRoot); final SyncStats expectedTotalStats = new SyncStats() @@ -790,7 +790,7 @@ void testDoesNotPopulatesStateOnFailureIfNotAvailable() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); final ReplicationOutput actual = worker.run(syncInputWithoutState, jobRoot); @@ -811,7 +811,7 @@ void testDoesNotPopulateOnIrrecoverableFailure() { messageTracker, recordSchemaValidator, workerMetricReporter, - persistConfigHelper); + updateConnectorConfigHelper); assertThrows(WorkerException.class, () -> worker.run(syncInput, jobRoot)); } diff --git a/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/PersistConfigHelperTest.java b/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/UpdateConnectorConfigHelperTest.java similarity index 93% rename from airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/PersistConfigHelperTest.java rename to airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/UpdateConnectorConfigHelperTest.java index e2d77ba8e06a..b275a2b4207f 100644 --- a/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/PersistConfigHelperTest.java +++ b/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/UpdateConnectorConfigHelperTest.java @@ -33,7 +33,7 @@ import org.junit.jupiter.api.Test; import org.mockito.Mockito; -class PersistConfigHelperTest { +class UpdateConnectorConfigHelperTest { private static final Long JOB_ID = 123L; private static final UUID CONNECTION_ID = UUID.randomUUID(); @@ -48,7 +48,7 @@ class PersistConfigHelperTest { private final SourceApi mSourceApi = mock(SourceApi.class); private final DestinationApi mDestinationApi = mock(DestinationApi.class); - private PersistConfigHelper persistConfigHelper; + private UpdateConnectorConfigHelper updateConnectorConfigHelper; @BeforeEach void setUp() throws ApiException { @@ -77,7 +77,7 @@ void setUp() throws ApiException { .destinationId(DESTINATION_ID) .name(DESTINATION_NAME)); - persistConfigHelper = new PersistConfigHelper(airbyteApiClient); + updateConnectorConfigHelper = new UpdateConnectorConfigHelper(airbyteApiClient); } @Test @@ -92,7 +92,7 @@ void testPersistSourceConfig() throws ApiException { when(mSourceApi.updateSource(Mockito.any())).thenReturn(new SourceRead().connectionConfiguration(configJson)); - persistConfigHelper.persistSourceConfig(JOB_ID, newConfiguration); + updateConnectorConfigHelper.updateSource(JOB_ID, newConfiguration); verify(mSourceApi).updateSource(expectedSourceUpdate); } @@ -108,7 +108,7 @@ void testPersistDestinationConfig() throws ApiException { when(mDestinationApi.updateDestination(Mockito.any())).thenReturn(new DestinationRead().connectionConfiguration(configJson)); - persistConfigHelper.persistDestinationConfig(JOB_ID, newConfiguration); + updateConnectorConfigHelper.updateDestination(JOB_ID, newConfiguration); verify(mDestinationApi).updateDestination(expectedDestinationUpdate); } diff --git a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java index 63edd9176b9e..bfbe900a094d 100644 --- a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java +++ b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java @@ -31,7 +31,7 @@ import io.airbyte.workers.WorkerMetricReporter; import io.airbyte.workers.WorkerUtils; import io.airbyte.workers.general.DefaultReplicationWorker; -import io.airbyte.workers.helper.PersistConfigHelper; +import io.airbyte.workers.helper.UpdateConnectorConfigHelper; import io.airbyte.workers.internal.AirbyteMessageTracker; import io.airbyte.workers.internal.AirbyteStreamFactory; import io.airbyte.workers.internal.DefaultAirbyteDestination; @@ -148,7 +148,7 @@ public Optional runJob() throws Exception { new AirbyteMessageTracker(), new RecordSchemaValidator(WorkerUtils.mapStreamNamesToSchemas(syncInput)), metricReporter, - new PersistConfigHelper(airbyteApiClient)); + new UpdateConnectorConfigHelper(airbyteApiClient)); log.info("Running replication worker..."); final var jobRoot = TemporalUtils.getJobRoot(configs.getWorkspaceRoot(), diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java index 71e2cdc11ceb..9a9ebcb3bb4b 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java @@ -51,7 +51,7 @@ import io.airbyte.workers.WorkerMetricReporter; import io.airbyte.workers.WorkerUtils; import io.airbyte.workers.general.DefaultReplicationWorker; -import io.airbyte.workers.helper.PersistConfigHelper; +import io.airbyte.workers.helper.UpdateConnectorConfigHelper; import io.airbyte.workers.internal.AirbyteMessageTracker; import io.airbyte.workers.internal.AirbyteSource; import io.airbyte.workers.internal.DefaultAirbyteDestination; @@ -295,7 +295,7 @@ private CheckedSupplier, Exception> new AirbyteMessageTracker(), new RecordSchemaValidator(WorkerUtils.mapStreamNamesToSchemas(syncInput)), metricReporter, - new PersistConfigHelper(airbyteApiClient)); + new UpdateConnectorConfigHelper(airbyteApiClient)); }; } From 44be06dccb77cf4a79e2f30234f268fbb0e4e2b7 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Tue, 6 Dec 2022 09:21:49 -0400 Subject: [PATCH 24/63] fix exception type --- .../airbyte/workers/general/DefaultReplicationWorkerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java index b4df4ef5aa7c..6de37bdbd46c 100644 --- a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java +++ b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java @@ -278,7 +278,7 @@ void testReplicationFailureOnSourceConfigPersistError() throws Exception { when(source.isFinished()).thenReturn(false, true); final String PERSIST_ERROR_MESSAGE = "there was a problem persisting the new config"; - doThrow(new ApiException(PERSIST_ERROR_MESSAGE)).when(updateConnectorConfigHelper).updateSource(Mockito.any(), Mockito.any()); + doThrow(new RuntimeException(PERSIST_ERROR_MESSAGE)).when(updateConnectorConfigHelper).updateSource(Mockito.any(), Mockito.any()); final ReplicationWorker worker = new DefaultReplicationWorker( JOB_ID, @@ -327,7 +327,7 @@ void testReplicationFailureOnDestinationConfigPersistError() throws Exception { when(destination.isFinished()).thenReturn(false, true); final String PERSIST_ERROR_MESSAGE = "there was a problem persisting the new config"; - doThrow(new ApiException(PERSIST_ERROR_MESSAGE)).when(updateConnectorConfigHelper).updateDestination(Mockito.any(), Mockito.any()); + doThrow(new RuntimeException(PERSIST_ERROR_MESSAGE)).when(updateConnectorConfigHelper).updateDestination(Mockito.any(), Mockito.any()); final ReplicationWorker worker = new DefaultReplicationWorker( JOB_ID, From 5227c730328f01176371c848c3fd434e6634079e Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Tue, 6 Dec 2022 10:03:21 -0400 Subject: [PATCH 25/63] fmt --- .../io/airbyte/workers/general/DefaultReplicationWorkerTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java index 6de37bdbd46c..2492fbf2e2a6 100644 --- a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java +++ b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java @@ -20,7 +20,6 @@ import com.fasterxml.jackson.databind.JsonNode; import com.google.common.collect.ImmutableMap; -import io.airbyte.api.client.invoker.generated.ApiException; import io.airbyte.commons.io.IOs; import io.airbyte.commons.json.Jsons; import io.airbyte.commons.string.Strings; From f2ec8ddd31f4b94f39f11d954501d24707227613 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Tue, 6 Dec 2022 17:31:27 -0400 Subject: [PATCH 26/63] move message type check into runnable --- .../general/DefaultReplicationWorker.java | 41 ++++++++----------- 1 file changed, 17 insertions(+), 24 deletions(-) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java index 03e7d294196d..15c257836927 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java @@ -261,8 +261,12 @@ private static Runnable readFromDstRunnable(final AirbyteDestination destination if (messageOptional.isPresent()) { final AirbyteMessage message = messageOptional.get(); LOGGER.info("State in DefaultReplicationWorker from destination: {}", message); + messageTracker.acceptFromDestination(message); - acceptAndUpdateDstConfig(jobId, message, updateConnectorConfigHelper); + + if(message.getType() == Type.CONTROL) { + acceptDstControlMessage(jobId, message.getControl(), updateConnectorConfigHelper); + } } } timeHolder.trackDestinationWriteEndTime(); @@ -319,7 +323,10 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou final AirbyteMessage message = mapper.mapMessage(airbyteMessage); messageTracker.acceptFromSource(message); - acceptAndUpdateSrcConfig(jobId, message, updateConnectorConfigHelper); + + if(message.getType() == Type.CONTROL) { + acceptSrcControlMessage(jobId, message.getControl(), updateConnectorConfigHelper); + } try { if (message.getType() == Type.RECORD || message.getType() == Type.STATE) { @@ -379,33 +386,19 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou }; } - private static boolean isControlConfigMsg(final AirbyteMessage message) { - return message.getType() == Type.CONTROL && message.getControl().getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG; - } - - private static void acceptAndUpdateSrcConfig(final Long jobId, - final AirbyteMessage message, + private static void acceptSrcControlMessage(final Long jobId, + final AirbyteControlMessage controlMessage, final UpdateConnectorConfigHelper updateConnectorConfigHelper) { - if (isControlConfigMsg(message)) { - try { - updateConnectorConfigHelper.updateSource(jobId, message.getControl().getConnectorConfig().getConfig()); - } catch (final Exception e) { - LOGGER.error("Error trying to save updated source config", e); - throw new RuntimeException(e); - } + if (controlMessage.getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG) { + updateConnectorConfigHelper.updateSource(jobId, controlMessage.getConnectorConfig().getConfig()); } } - private static void acceptAndUpdateDstConfig(final Long jobId, - final AirbyteMessage message, + private static void acceptDstControlMessage(final Long jobId, + final AirbyteControlMessage controlMessage, final UpdateConnectorConfigHelper updateConnectorConfigHelper) { - try { - if (isControlConfigMsg(message)) { - updateConnectorConfigHelper.updateDestination(jobId, message.getControl().getConnectorConfig().getConfig()); - } - } catch (final Exception e) { - LOGGER.error("Error trying to save updated destination config", e); - throw new RuntimeException(e); + if (controlMessage.getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG) { + updateConnectorConfigHelper.updateDestination(jobId, controlMessage.getConnectorConfig().getConfig()); } } From 329f64143e288dad9aaaf814937ecc6c8872ff53 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Tue, 6 Dec 2022 17:32:25 -0400 Subject: [PATCH 27/63] formatting --- .../workers/general/DefaultReplicationWorker.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java index 15c257836927..6f05a7a7cf36 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java @@ -264,7 +264,7 @@ private static Runnable readFromDstRunnable(final AirbyteDestination destination messageTracker.acceptFromDestination(message); - if(message.getType() == Type.CONTROL) { + if (message.getType() == Type.CONTROL) { acceptDstControlMessage(jobId, message.getControl(), updateConnectorConfigHelper); } } @@ -324,7 +324,7 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou messageTracker.acceptFromSource(message); - if(message.getType() == Type.CONTROL) { + if (message.getType() == Type.CONTROL) { acceptSrcControlMessage(jobId, message.getControl(), updateConnectorConfigHelper); } @@ -387,16 +387,16 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou } private static void acceptSrcControlMessage(final Long jobId, - final AirbyteControlMessage controlMessage, - final UpdateConnectorConfigHelper updateConnectorConfigHelper) { + final AirbyteControlMessage controlMessage, + final UpdateConnectorConfigHelper updateConnectorConfigHelper) { if (controlMessage.getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG) { updateConnectorConfigHelper.updateSource(jobId, controlMessage.getConnectorConfig().getConfig()); } } private static void acceptDstControlMessage(final Long jobId, - final AirbyteControlMessage controlMessage, - final UpdateConnectorConfigHelper updateConnectorConfigHelper) { + final AirbyteControlMessage controlMessage, + final UpdateConnectorConfigHelper updateConnectorConfigHelper) { if (controlMessage.getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG) { updateConnectorConfigHelper.updateDestination(jobId, controlMessage.getConnectorConfig().getConfig()); } From b94a8f8b04dcf7d6939e06e4db40c8fdd62a063f Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Thu, 8 Dec 2022 01:36:10 -0400 Subject: [PATCH 28/63] pass api client env vars to container orchestrator --- ...ontainerOrchestratorConfigBeanFactory.java | 22 ++++++++++++++++++- 1 file changed, 21 insertions(+), 1 deletion(-) diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/config/ContainerOrchestratorConfigBeanFactory.java b/airbyte-workers/src/main/java/io/airbyte/workers/config/ContainerOrchestratorConfigBeanFactory.java index d9259005b3ce..64d740dc5b73 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/config/ContainerOrchestratorConfigBeanFactory.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/config/ContainerOrchestratorConfigBeanFactory.java @@ -37,6 +37,12 @@ public class ContainerOrchestratorConfigBeanFactory { private static final String DD_VERSION_ENV_VAR = "DD_VERSION"; private static final String JAVA_OPTS_ENV_VAR = "JAVA_OPTS"; private static final String PUBLISH_METRICS_ENV_VAR = "PUBLISH_METRICS"; + private static final String CONTROL_PLANE_AUTH_ENDPOINT_ENV_VAR = "CONTROL_PLANE_AUTH_ENDPOINT"; + private static final String DATA_PLANE_SERVICE_ACCOUNT_CREDENTIALS_PATH_ENV_VAR = "DATA_PLANE_SERVICE_ACCOUNT_CREDENTIALS_PATH"; + private static final String DATA_PLANE_SERVICE_ACCOUNT_EMAIL_ENV_VAR = "DATA_PLANE_SERVICE_ACCOUNT_EMAIL"; + private static final String AIRBYTE_API_AUTH_HEADER_NAME_ENV_VAR = "AIRBYTE_API_AUTH_HEADER_NAME"; + private static final String AIRBYTE_API_AUTH_HEADER_VALUE_ENV_VAR = "AIRBYTE_API_AUTH_HEADER_VALUE"; + private static final String INTERNAL_API_HOST_ENV_VAR = "INTERNAL_API_HOST"; // IMPORTANT: Changing the storage location will orphan already existing kube pods when the new // version is deployed! @@ -61,7 +67,13 @@ public ContainerOrchestratorConfig kubernetesContainerOrchestratorConfig( @Value("${airbyte.metric.should-publish}") final String shouldPublishMetrics, final FeatureFlags featureFlags, @Value("${airbyte.container.orchestrator.java-opts}") final String containerOrchestratorJavaOpts, - final WorkerEnvironment workerEnvironment) { + final WorkerEnvironment workerEnvironment, + @Value("${airbyte.internal.api.host}") final String airbyteApiHost, + @Value("${airbyte.internal.api.auth-header.name}") final String airbyteApiAuthHeaderName, + @Value("${airbyte.internal.api.auth-header.value}") final String airbyteApiAuthHeaderValue, + @Value("${airbyte.control.plane.auth-endpoint}") final String controlPlaneAuthEndpoint, + @Value("${airbyte.data.plane.service-account.email}") final String dataPlaneServiceAccountEmail, + @Value("${airbyte.data.plane.service-account.credentials-path}") final String dataPlaneServiceAccountCredentialsPath) { final var kubernetesClient = new DefaultKubernetesClient(); final DocumentStoreClient documentStoreClient = StateClients.create( @@ -87,6 +99,14 @@ public ContainerOrchestratorConfig kubernetesContainerOrchestratorConfig( environmentVariables.put(DD_VERSION_ENV_VAR, System.getenv(DD_VERSION_ENV_VAR)); } + // Environment variables for ApiClientBeanFactory + environmentVariables.put(CONTROL_PLANE_AUTH_ENDPOINT_ENV_VAR, controlPlaneAuthEndpoint); + environmentVariables.put(DATA_PLANE_SERVICE_ACCOUNT_CREDENTIALS_PATH_ENV_VAR, dataPlaneServiceAccountCredentialsPath); + environmentVariables.put(DATA_PLANE_SERVICE_ACCOUNT_EMAIL_ENV_VAR, dataPlaneServiceAccountEmail); + environmentVariables.put(AIRBYTE_API_AUTH_HEADER_NAME_ENV_VAR, airbyteApiAuthHeaderName); + environmentVariables.put(AIRBYTE_API_AUTH_HEADER_VALUE_ENV_VAR, airbyteApiAuthHeaderValue); + environmentVariables.put(INTERNAL_API_HOST_ENV_VAR, airbyteApiHost); + return new ContainerOrchestratorConfig( namespace, documentStoreClient, From 6fbbfbed8f71f6bd960d238d24e8890174e31e43 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Thu, 8 Dec 2022 14:39:39 -0400 Subject: [PATCH 29/63] pass micronaut envs to container orchestrator --- .../config/ContainerOrchestratorConfigBeanFactory.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/config/ContainerOrchestratorConfigBeanFactory.java b/airbyte-workers/src/main/java/io/airbyte/workers/config/ContainerOrchestratorConfigBeanFactory.java index 64d740dc5b73..53906a238e88 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/config/ContainerOrchestratorConfigBeanFactory.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/config/ContainerOrchestratorConfigBeanFactory.java @@ -15,6 +15,7 @@ import io.micronaut.context.annotation.Factory; import io.micronaut.context.annotation.Requires; import io.micronaut.context.annotation.Value; +import io.micronaut.context.env.Environment; import io.micronaut.core.util.StringUtils; import jakarta.inject.Named; import jakarta.inject.Singleton; @@ -107,6 +108,10 @@ public ContainerOrchestratorConfig kubernetesContainerOrchestratorConfig( environmentVariables.put(AIRBYTE_API_AUTH_HEADER_VALUE_ENV_VAR, airbyteApiAuthHeaderValue); environmentVariables.put(INTERNAL_API_HOST_ENV_VAR, airbyteApiHost); + if (System.getenv(Environment.ENVIRONMENTS_ENV) != null) { + environmentVariables.put(Environment.ENVIRONMENTS_ENV, System.getenv(Environment.ENVIRONMENTS_ENV)); + } + return new ContainerOrchestratorConfig( namespace, documentStoreClient, From b486f6d2174cf603ede953f16b3b1598583af5d7 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Thu, 8 Dec 2022 15:27:54 -0400 Subject: [PATCH 30/63] print stacktrace for debugging --- .../src/main/java/io/airbyte/api/client/AirbyteApiClient.java | 1 + 1 file changed, 1 insertion(+) diff --git a/airbyte-api/src/main/java/io/airbyte/api/client/AirbyteApiClient.java b/airbyte-api/src/main/java/io/airbyte/api/client/AirbyteApiClient.java index 7a2efa51b722..76967d4cdc9d 100644 --- a/airbyte-api/src/main/java/io/airbyte/api/client/AirbyteApiClient.java +++ b/airbyte-api/src/main/java/io/airbyte/api/client/AirbyteApiClient.java @@ -182,6 +182,7 @@ public static T retryWithJitter(final Callable call, keepTrying = false; } catch (final Exception e) { LOGGER.info("Attempt {} to {} error: {}", currRetries, desc, e); + LOGGER.error("Error", e); currRetries++; // Sleep anywhere from 1 to jitterMaxIntervalSecs seconds. From adabd35c365b18a8ac5afea12cb91c4956611c2f Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Thu, 15 Dec 2022 18:48:34 -0400 Subject: [PATCH 31/63] different api host for container orchestrator --- .../ContainerOrchestratorConfigBeanFactory.java | 12 ++++++------ airbyte-workers/src/main/resources/application.yml | 5 +++++ 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/config/ContainerOrchestratorConfigBeanFactory.java b/airbyte-workers/src/main/java/io/airbyte/workers/config/ContainerOrchestratorConfigBeanFactory.java index 53906a238e88..e852b1839f87 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/config/ContainerOrchestratorConfigBeanFactory.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/config/ContainerOrchestratorConfigBeanFactory.java @@ -69,9 +69,9 @@ public ContainerOrchestratorConfig kubernetesContainerOrchestratorConfig( final FeatureFlags featureFlags, @Value("${airbyte.container.orchestrator.java-opts}") final String containerOrchestratorJavaOpts, final WorkerEnvironment workerEnvironment, - @Value("${airbyte.internal.api.host}") final String airbyteApiHost, - @Value("${airbyte.internal.api.auth-header.name}") final String airbyteApiAuthHeaderName, - @Value("${airbyte.internal.api.auth-header.value}") final String airbyteApiAuthHeaderValue, + @Value("${airbyte.container.orchestrator.api.host}") final String containerOrchestratorApiHost, + @Value("${airbyte.container.orchestrator.api.auth-header.name}") final String containerOrchestratorApiAuthHeaderName, + @Value("${airbyte.container.orchestrator.api.auth-header.value}") final String containerOrchestratorApiAuthHeaderValue, @Value("${airbyte.control.plane.auth-endpoint}") final String controlPlaneAuthEndpoint, @Value("${airbyte.data.plane.service-account.email}") final String dataPlaneServiceAccountEmail, @Value("${airbyte.data.plane.service-account.credentials-path}") final String dataPlaneServiceAccountCredentialsPath) { @@ -104,9 +104,9 @@ public ContainerOrchestratorConfig kubernetesContainerOrchestratorConfig( environmentVariables.put(CONTROL_PLANE_AUTH_ENDPOINT_ENV_VAR, controlPlaneAuthEndpoint); environmentVariables.put(DATA_PLANE_SERVICE_ACCOUNT_CREDENTIALS_PATH_ENV_VAR, dataPlaneServiceAccountCredentialsPath); environmentVariables.put(DATA_PLANE_SERVICE_ACCOUNT_EMAIL_ENV_VAR, dataPlaneServiceAccountEmail); - environmentVariables.put(AIRBYTE_API_AUTH_HEADER_NAME_ENV_VAR, airbyteApiAuthHeaderName); - environmentVariables.put(AIRBYTE_API_AUTH_HEADER_VALUE_ENV_VAR, airbyteApiAuthHeaderValue); - environmentVariables.put(INTERNAL_API_HOST_ENV_VAR, airbyteApiHost); + environmentVariables.put(AIRBYTE_API_AUTH_HEADER_NAME_ENV_VAR, containerOrchestratorApiAuthHeaderName); + environmentVariables.put(AIRBYTE_API_AUTH_HEADER_VALUE_ENV_VAR, containerOrchestratorApiAuthHeaderValue); + environmentVariables.put(INTERNAL_API_HOST_ENV_VAR, containerOrchestratorApiHost); if (System.getenv(Environment.ENVIRONMENTS_ENV) != null) { environmentVariables.put(Environment.ENVIRONMENTS_ENV, System.getenv(Environment.ENVIRONMENTS_ENV)); diff --git a/airbyte-workers/src/main/resources/application.yml b/airbyte-workers/src/main/resources/application.yml index 733ae744ebdb..e24a21be28f4 100644 --- a/airbyte-workers/src/main/resources/application.yml +++ b/airbyte-workers/src/main/resources/application.yml @@ -52,6 +52,11 @@ airbyte: specific-resource-defaults-enabled: ${CONNECTOR_SPECIFIC_RESOURCE_DEFAULTS_ENABLED:false} container: orchestrator: + api: + auth-header: + name: ${AIRBYTE_API_AUTH_HEADER_NAME:} + value: ${AIRBYTE_API_AUTH_HEADER_VALUE:} + host: ${CONTAINER_ORCHESTRATOR_API_HOST:airbyte-server-svc.ab:80} enabled: ${CONTAINER_ORCHESTRATOR_ENABLED:false} image: ${CONTAINER_ORCHESTRATOR_IMAGE:} java-opts: ${CONTAINER_ORCHESTRATOR_JAVA_OPTS:} From 27ebbfbdc0c78049205d53d051c208c83a84c515 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Fri, 16 Dec 2022 01:32:38 -0400 Subject: [PATCH 32/63] fix default env var --- airbyte-workers/src/main/resources/application.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airbyte-workers/src/main/resources/application.yml b/airbyte-workers/src/main/resources/application.yml index e24a21be28f4..592fdb524f5a 100644 --- a/airbyte-workers/src/main/resources/application.yml +++ b/airbyte-workers/src/main/resources/application.yml @@ -56,7 +56,7 @@ airbyte: auth-header: name: ${AIRBYTE_API_AUTH_HEADER_NAME:} value: ${AIRBYTE_API_AUTH_HEADER_VALUE:} - host: ${CONTAINER_ORCHESTRATOR_API_HOST:airbyte-server-svc.ab:80} + host: ${CONTAINER_ORCHESTRATOR_API_HOST:`airbyte-server-svc.ab:80`} enabled: ${CONTAINER_ORCHESTRATOR_ENABLED:false} image: ${CONTAINER_ORCHESTRATOR_IMAGE:} java-opts: ${CONTAINER_ORCHESTRATOR_JAVA_OPTS:} From d88f12f52d8a665fbeca29a85596327ff2a3fcc0 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Fri, 16 Dec 2022 16:18:56 -0400 Subject: [PATCH 33/63] format --- .../io/airbyte/workers/general/DefaultReplicationWorker.java | 4 ++-- .../airbyte/workers/general/DefaultReplicationWorkerTest.java | 4 ++-- .../orchestrator/ReplicationJobOrchestrator.java | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java index 70312e4566ab..64e76ec3eb9e 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java @@ -111,7 +111,7 @@ public DefaultReplicationWorker(final String jobId, final MessageTracker messageTracker, final RecordSchemaValidator recordSchemaValidator, final WorkerMetricReporter metricReporter, - final UpdateConnectorConfigHelper updateConnectorConfigHelper) { + final UpdateConnectorConfigHelper updateConnectorConfigHelper, final boolean fieldSelectionEnabled) { this.jobId = jobId; this.attempt = attempt; @@ -221,7 +221,7 @@ private void replicate(final Path jobRoot, recordSchemaValidator, metricReporter, timeTracker, - Long.valueOf(jobId) + Long.valueOf(jobId), fieldSelectionEnabled), executors) .whenComplete((msg, ex) -> { diff --git a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java index 12bad7535a32..83a91dcb80ea 100644 --- a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java +++ b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java @@ -163,7 +163,7 @@ void test() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper); + updateConnectorConfigHelper, workerMetricReporter, false); worker.run(syncInput, jobRoot); @@ -193,7 +193,7 @@ void testInvalidSchema() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, + updateConnectorConfigHelper, false); worker.run(syncInput, jobRoot); diff --git a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java index 7c9b342fa40f..f00a944adbc0 100644 --- a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java +++ b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java @@ -154,7 +154,7 @@ public Optional runJob() throws Exception { new AirbyteMessageTracker(), new RecordSchemaValidator(WorkerUtils.mapStreamNamesToSchemas(syncInput)), metricReporter, - new UpdateConnectorConfigHelper(airbyteApiClient), + new UpdateConnectorConfigHelper(airbyteApiClient), featureFlags.applyFieldSelection()); log.info("Running replication worker..."); From ef656dc334cc6105bd7ae6600c9f5542b1d72a29 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Fri, 16 Dec 2022 16:32:39 -0400 Subject: [PATCH 34/63] fix errors after merge --- .../workers/general/DefaultReplicationWorker.java | 1 + .../general/DefaultReplicationWorkerTest.java | 15 +++++++-------- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java index 64e76ec3eb9e..afff1c30b336 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java @@ -217,6 +217,7 @@ private void replicate(final Path jobRoot, cancelled, mapper, messageTracker, + updateConnectorConfigHelper, mdc, recordSchemaValidator, metricReporter, diff --git a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java index 83a91dcb80ea..a66010ba7ef6 100644 --- a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java +++ b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java @@ -163,8 +163,7 @@ void test() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, - workerMetricReporter, false); + updateConnectorConfigHelper, false); worker.run(syncInput, jobRoot); @@ -267,7 +266,7 @@ void testReplicationRunnableSourceUpdateConfig() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper); + updateConnectorConfigHelper, false); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.COMPLETED, output.getReplicationAttemptSummary().getStatus()); @@ -292,7 +291,7 @@ void testReplicationFailureOnSourceConfigPersistError() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper); + updateConnectorConfigHelper, false); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); @@ -316,7 +315,7 @@ void testReplicationRunnableDestinationUpdateConfig() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper); + updateConnectorConfigHelper, false); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.COMPLETED, output.getReplicationAttemptSummary().getStatus()); @@ -341,7 +340,7 @@ void testReplicationFailureOnDestinationConfigPersistError() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper); + updateConnectorConfigHelper, false); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); @@ -472,7 +471,7 @@ void testOnlySelectedFieldsDeliveredToDestinationWithFieldSelectionEnabled() thr messageTracker, recordSchemaValidator, workerMetricReporter, - true); + updateConnectorConfigHelper, true); worker.run(syncInput, jobRoot); @@ -503,7 +502,7 @@ void testAllFieldsDeliveredWithFieldSelectionDisabled() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - false); + updateConnectorConfigHelper, false); worker.run(syncInput, jobRoot); From 8a67e1302f0527fe39901ce430435bc76730a6c2 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Mon, 19 Dec 2022 19:19:54 -0400 Subject: [PATCH 35/63] set source and destination actor id as part of the sync input --- .../java/io/airbyte/workers/WorkerUtils.java | 2 + .../general/DefaultReplicationWorker.java | 20 +++++----- .../helper/UpdateConnectorConfigHelper.java | 32 +-------------- .../workers/test_utils/TestConfigHelpers.java | 2 + .../general/DefaultReplicationWorkerTest.java | 8 ++-- .../UpdateConnectorConfigHelperTest.java | 27 +------------ .../resources/types/StandardSyncInput.yaml | 10 +++++ .../types/WorkerDestinationConfig.yaml | 4 ++ .../resources/types/WorkerSourceConfig.yaml | 4 ++ .../activities/GenerateInputActivityImpl.java | 39 ++++++++----------- 10 files changed, 57 insertions(+), 91 deletions(-) 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 b704ade5f6a4..993ef17b754f 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 @@ -91,6 +91,7 @@ public static void cancelProcess(final Process process) { */ public static WorkerSourceConfig syncToWorkerSourceConfig(final StandardSyncInput sync) { return new WorkerSourceConfig() + .withActorId(sync.getSourceActorId()) .withSourceConnectionConfiguration(sync.getSourceConfiguration()) .withCatalog(sync.getCatalog()) .withState(sync.getState()); @@ -102,6 +103,7 @@ public static WorkerSourceConfig syncToWorkerSourceConfig(final StandardSyncInpu */ public static WorkerDestinationConfig syncToWorkerDestinationConfig(final StandardSyncInput sync) { return new WorkerDestinationConfig() + .withActorId(sync.getDestinationActorId()) .withDestinationConnectionConfiguration(sync.getDestinationConfiguration()) .withCatalog(sync.getCatalog()) .withState(sync.getState()); diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java index afff1c30b336..f97220079a37 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java @@ -196,7 +196,7 @@ private void replicate(final Path jobRoot, // note: `whenComplete` is used instead of `exceptionally` so that the original exception is still // thrown final CompletableFuture readFromDstThread = CompletableFuture.runAsync( - readFromDstRunnable(destination, cancelled, messageTracker, updateConnectorConfigHelper, mdc, timeTracker, Long.valueOf(jobId)), + readFromDstRunnable(destination, cancelled, messageTracker, updateConnectorConfigHelper, mdc, timeTracker, destinationConfig), executors) .whenComplete((msg, ex) -> { if (ex != null) { @@ -222,7 +222,7 @@ private void replicate(final Path jobRoot, recordSchemaValidator, metricReporter, timeTracker, - Long.valueOf(jobId), + sourceConfig, fieldSelectionEnabled), executors) .whenComplete((msg, ex) -> { @@ -263,7 +263,7 @@ private static Runnable readFromDstRunnable(final AirbyteDestination destination final UpdateConnectorConfigHelper updateConnectorConfigHelper, final Map mdc, final ThreadedTimeTracker timeHolder, - final Long jobId) { + final WorkerDestinationConfig destinationConfig) { return () -> { MDC.setContextMap(mdc); LOGGER.info("Destination output thread started."); @@ -282,7 +282,7 @@ private static Runnable readFromDstRunnable(final AirbyteDestination destination messageTracker.acceptFromDestination(message); if (message.getType() == Type.CONTROL) { - acceptDstControlMessage(jobId, message.getControl(), updateConnectorConfigHelper); + acceptDstControlMessage(destinationConfig, message.getControl(), updateConnectorConfigHelper); } } } @@ -320,7 +320,7 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou final RecordSchemaValidator recordSchemaValidator, final WorkerMetricReporter metricReporter, final ThreadedTimeTracker timeHolder, - final Long jobId, + final WorkerSourceConfig sourceConfig, final boolean fieldSelectionEnabled) { return () -> { MDC.setContextMap(mdc); @@ -351,7 +351,7 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou messageTracker.acceptFromSource(message); if (message.getType() == Type.CONTROL) { - acceptSrcControlMessage(jobId, message.getControl(), updateConnectorConfigHelper); + acceptSrcControlMessage(sourceConfig, message.getControl(), updateConnectorConfigHelper); } try { @@ -412,19 +412,19 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou }; } - private static void acceptSrcControlMessage(final Long jobId, + private static void acceptSrcControlMessage(final WorkerSourceConfig sourceConfig, final AirbyteControlMessage controlMessage, final UpdateConnectorConfigHelper updateConnectorConfigHelper) { if (controlMessage.getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG) { - updateConnectorConfigHelper.updateSource(jobId, controlMessage.getConnectorConfig().getConfig()); + updateConnectorConfigHelper.updateSource(sourceConfig.getActorId(), controlMessage.getConnectorConfig().getConfig()); } } - private static void acceptDstControlMessage(final Long jobId, + private static void acceptDstControlMessage(final WorkerDestinationConfig destinationConfig, final AirbyteControlMessage controlMessage, final UpdateConnectorConfigHelper updateConnectorConfigHelper) { if (controlMessage.getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG) { - updateConnectorConfigHelper.updateDestination(jobId, controlMessage.getConnectorConfig().getConfig()); + updateConnectorConfigHelper.updateDestination(destinationConfig.getActorId(), controlMessage.getConnectorConfig().getConfig()); } } diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/UpdateConnectorConfigHelper.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/UpdateConnectorConfigHelper.java index bff3684a3595..22b06217f077 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/UpdateConnectorConfigHelper.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/UpdateConnectorConfigHelper.java @@ -7,15 +7,10 @@ import com.google.common.hash.Hashing; import io.airbyte.api.client.AirbyteApiClient; import io.airbyte.api.client.generated.DestinationApi; -import io.airbyte.api.client.generated.JobsApi; import io.airbyte.api.client.generated.SourceApi; -import io.airbyte.api.client.model.generated.ConnectionIdRequestBody; -import io.airbyte.api.client.model.generated.ConnectionRead; import io.airbyte.api.client.model.generated.DestinationIdRequestBody; import io.airbyte.api.client.model.generated.DestinationRead; import io.airbyte.api.client.model.generated.DestinationUpdate; -import io.airbyte.api.client.model.generated.JobIdRequestBody; -import io.airbyte.api.client.model.generated.JobInfoLightRead; import io.airbyte.api.client.model.generated.SourceIdRequestBody; import io.airbyte.api.client.model.generated.SourceRead; import io.airbyte.api.client.model.generated.SourceUpdate; @@ -44,27 +39,11 @@ public UpdateConnectorConfigHelper(final AirbyteApiClient apiClient) { this.apiClient = apiClient; } - private UUID getConnectionIdFromJobId(final Long jobId) { - final JobsApi jobsApi = apiClient.getJobsApi(); - final JobIdRequestBody body = new JobIdRequestBody().id(jobId); - final JobInfoLightRead jobInfo = AirbyteApiClient.retryWithJitter( - () -> jobsApi.getJobInfoLight(body), - "get job info"); - return UUID.fromString(jobInfo.getJob().getConfigId()); - } - /** * Updates the Source from a sync job ID with the provided Configuration. Secrets and OAuth * parameters will be masked when saving. */ - public void updateSource(final Long jobId, final Config config) { - final UUID connectionId = getConnectionIdFromJobId(jobId); - - final ConnectionRead connection = AirbyteApiClient.retryWithJitter( - () -> apiClient.getConnectionApi().getConnection(new ConnectionIdRequestBody().connectionId(connectionId)), - "get connection"); - final UUID sourceId = connection.getSourceId(); - + public void updateSource(final UUID sourceId, final Config config) { final SourceApi sourceApi = apiClient.getSourceApi(); final SourceRead source = AirbyteApiClient.retryWithJitter( () -> sourceApi.getSource(new SourceIdRequestBody().sourceId(sourceId)), @@ -87,14 +66,7 @@ public void updateSource(final Long jobId, final Config config) { * Updates the Destination from a sync job ID with the provided Configuration. Secrets and OAuth * parameters will be masked when saving. */ - public void updateDestination(final Long jobId, final Config config) { - final UUID connectionId = getConnectionIdFromJobId(jobId); - - final ConnectionRead connection = AirbyteApiClient.retryWithJitter( - () -> apiClient.getConnectionApi().getConnection(new ConnectionIdRequestBody().connectionId(connectionId)), - "get connection"); - final UUID destinationId = connection.getDestinationId(); - + public void updateDestination(final UUID destinationId, final Config config) { final DestinationApi destinationApi = apiClient.getDestinationApi(); final DestinationRead destination = AirbyteApiClient.retryWithJitter( () -> destinationApi.getDestination(new DestinationIdRequestBody().destinationId(destinationId)), diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/test_utils/TestConfigHelpers.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/test_utils/TestConfigHelpers.java index 30e055982e79..aa82889da75a 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/test_utils/TestConfigHelpers.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/test_utils/TestConfigHelpers.java @@ -128,6 +128,8 @@ public static ImmutablePair createSyncConfig(fi final StandardSyncInput syncInput = new StandardSyncInput() .withNamespaceDefinition(standardSync.getNamespaceDefinition()) .withPrefix(standardSync.getPrefix()) + .withSourceActorId(sourceId) + .withDestinationActorId(destinationId) .withDestinationConfiguration(destinationConnectionConfig.getConfiguration()) .withCatalog(standardSync.getCatalog()) .withSourceConfiguration(sourceConnectionConfig.getConfiguration()) diff --git a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java index a66010ba7ef6..1d8524b3e9d7 100644 --- a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java +++ b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java @@ -271,7 +271,7 @@ void testReplicationRunnableSourceUpdateConfig() throws Exception { final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.COMPLETED, output.getReplicationAttemptSummary().getStatus()); - verify(updateConnectorConfigHelper).updateSource(Long.valueOf(JOB_ID), CONNECTOR_CONFIG); + verify(updateConnectorConfigHelper).updateSource(syncInput.getSourceActorId(), CONNECTOR_CONFIG); } @Test @@ -298,7 +298,7 @@ void testReplicationFailureOnSourceConfigPersistError() throws Exception { assertTrue(output.getFailures().stream() .anyMatch(f -> f.getFailureOrigin().equals(FailureOrigin.REPLICATION) && f.getStacktrace().contains(PERSIST_ERROR_MESSAGE))); - verify(updateConnectorConfigHelper).updateSource(Long.valueOf(JOB_ID), CONNECTOR_CONFIG); + verify(updateConnectorConfigHelper).updateSource(syncInput.getSourceActorId(), CONNECTOR_CONFIG); } @Test @@ -320,7 +320,7 @@ void testReplicationRunnableDestinationUpdateConfig() throws Exception { final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.COMPLETED, output.getReplicationAttemptSummary().getStatus()); - verify(updateConnectorConfigHelper).updateDestination(Long.valueOf(JOB_ID), CONNECTOR_CONFIG); + verify(updateConnectorConfigHelper).updateDestination(syncInput.getDestinationActorId(), CONNECTOR_CONFIG); } @Test @@ -347,7 +347,7 @@ void testReplicationFailureOnDestinationConfigPersistError() throws Exception { assertTrue(output.getFailures().stream() .anyMatch(f -> f.getFailureOrigin().equals(FailureOrigin.REPLICATION) && f.getStacktrace().contains(PERSIST_ERROR_MESSAGE))); - verify(updateConnectorConfigHelper).updateDestination(Long.valueOf(JOB_ID), CONNECTOR_CONFIG); + verify(updateConnectorConfigHelper).updateDestination(syncInput.getDestinationActorId(), CONNECTOR_CONFIG); } @Test diff --git a/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/UpdateConnectorConfigHelperTest.java b/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/UpdateConnectorConfigHelperTest.java index b275a2b4207f..cbe9bf596a7d 100644 --- a/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/UpdateConnectorConfigHelperTest.java +++ b/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/UpdateConnectorConfigHelperTest.java @@ -10,19 +10,12 @@ import com.fasterxml.jackson.databind.JsonNode; import io.airbyte.api.client.AirbyteApiClient; -import io.airbyte.api.client.generated.ConnectionApi; import io.airbyte.api.client.generated.DestinationApi; -import io.airbyte.api.client.generated.JobsApi; import io.airbyte.api.client.generated.SourceApi; import io.airbyte.api.client.invoker.generated.ApiException; -import io.airbyte.api.client.model.generated.ConnectionIdRequestBody; -import io.airbyte.api.client.model.generated.ConnectionRead; import io.airbyte.api.client.model.generated.DestinationIdRequestBody; import io.airbyte.api.client.model.generated.DestinationRead; import io.airbyte.api.client.model.generated.DestinationUpdate; -import io.airbyte.api.client.model.generated.JobIdRequestBody; -import io.airbyte.api.client.model.generated.JobInfoLightRead; -import io.airbyte.api.client.model.generated.JobRead; import io.airbyte.api.client.model.generated.SourceIdRequestBody; import io.airbyte.api.client.model.generated.SourceRead; import io.airbyte.api.client.model.generated.SourceUpdate; @@ -35,16 +28,12 @@ class UpdateConnectorConfigHelperTest { - private static final Long JOB_ID = 123L; - private static final UUID CONNECTION_ID = UUID.randomUUID(); private static final UUID SOURCE_ID = UUID.randomUUID(); private static final String SOURCE_NAME = "source-stripe"; private static final UUID DESTINATION_ID = UUID.randomUUID(); private static final String DESTINATION_NAME = "destination-google-sheets"; private final AirbyteApiClient airbyteApiClient = mock(AirbyteApiClient.class); - private final JobsApi mJobsApi = mock(JobsApi.class); - private final ConnectionApi mConnectionApi = mock(ConnectionApi.class); private final SourceApi mSourceApi = mock(SourceApi.class); private final DestinationApi mDestinationApi = mock(DestinationApi.class); @@ -52,20 +41,8 @@ class UpdateConnectorConfigHelperTest { @BeforeEach void setUp() throws ApiException { - when(airbyteApiClient.getJobsApi()).thenReturn(mJobsApi); when(airbyteApiClient.getSourceApi()).thenReturn(mSourceApi); when(airbyteApiClient.getDestinationApi()).thenReturn(mDestinationApi); - when(airbyteApiClient.getConnectionApi()).thenReturn(mConnectionApi); - - when(mJobsApi.getJobInfoLight(new JobIdRequestBody() - .id(JOB_ID))).thenReturn(new JobInfoLightRead() - .job(new JobRead() - .configId(CONNECTION_ID.toString()))); - - when(mConnectionApi.getConnection(new ConnectionIdRequestBody() - .connectionId(CONNECTION_ID))).thenReturn(new ConnectionRead() - .sourceId(SOURCE_ID) - .destinationId(DESTINATION_ID)); when(mSourceApi.getSource(new SourceIdRequestBody() .sourceId(SOURCE_ID))).thenReturn(new SourceRead() @@ -92,7 +69,7 @@ void testPersistSourceConfig() throws ApiException { when(mSourceApi.updateSource(Mockito.any())).thenReturn(new SourceRead().connectionConfiguration(configJson)); - updateConnectorConfigHelper.updateSource(JOB_ID, newConfiguration); + updateConnectorConfigHelper.updateSource(SOURCE_ID, newConfiguration); verify(mSourceApi).updateSource(expectedSourceUpdate); } @@ -108,7 +85,7 @@ void testPersistDestinationConfig() throws ApiException { when(mDestinationApi.updateDestination(Mockito.any())).thenReturn(new DestinationRead().connectionConfiguration(configJson)); - updateConnectorConfigHelper.updateDestination(JOB_ID, newConfiguration); + updateConnectorConfigHelper.updateDestination(DESTINATION_ID, newConfiguration); verify(mDestinationApi).updateDestination(expectedDestinationUpdate); } diff --git a/airbyte-config/config-models/src/main/resources/types/StandardSyncInput.yaml b/airbyte-config/config-models/src/main/resources/types/StandardSyncInput.yaml index ea84cecce3d4..75940f7a7886 100644 --- a/airbyte-config/config-models/src/main/resources/types/StandardSyncInput.yaml +++ b/airbyte-config/config-models/src/main/resources/types/StandardSyncInput.yaml @@ -6,7 +6,9 @@ description: job sync config type: object additionalProperties: false required: + - sourceActorId - sourceConfiguration + - destinationActorId - destinationConfiguration - catalog properties: @@ -19,6 +21,14 @@ properties: prefix: description: Prefix that will be prepended to the name of each stream when it is written to the destination. type: string + sourceActorId: + description: Actor ID for the source used in the sync - this is used to update the actor configuration when requested. + type: string + format: uuid + destinationActorId: + description: Actor ID for the destination used in the sync - this is used to update the actor configuration when requested. + type: string + format: uuid sourceConfiguration: description: Integration specific blob. Must be a valid JSON string. type: object diff --git a/airbyte-config/config-models/src/main/resources/types/WorkerDestinationConfig.yaml b/airbyte-config/config-models/src/main/resources/types/WorkerDestinationConfig.yaml index cc093a719712..d2afeb4e290c 100644 --- a/airbyte-config/config-models/src/main/resources/types/WorkerDestinationConfig.yaml +++ b/airbyte-config/config-models/src/main/resources/types/WorkerDestinationConfig.yaml @@ -6,10 +6,14 @@ description: WorkerDestinationConfig type: object additionalProperties: false required: + - actorId - destinationConnectionConfiguration - catalog - connectionId properties: + actorId: + type: string + format: uuid destinationConnectionConfiguration: description: Integration specific blob. Must be a valid JSON string. type: object diff --git a/airbyte-config/config-models/src/main/resources/types/WorkerSourceConfig.yaml b/airbyte-config/config-models/src/main/resources/types/WorkerSourceConfig.yaml index f0a73ea7a2c1..9ee1bace575f 100644 --- a/airbyte-config/config-models/src/main/resources/types/WorkerSourceConfig.yaml +++ b/airbyte-config/config-models/src/main/resources/types/WorkerSourceConfig.yaml @@ -6,9 +6,13 @@ description: WorkerSourceConfig type: object additionalProperties: false required: + - actorId - sourceConnectionConfiguration - catalog properties: + actorId: + type: string + format: uuid sourceConnectionConfiguration: description: Integration specific blob. Must be a valid JSON string. type: object 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 2a550a62eb36..64b052631e7a 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 @@ -19,6 +19,7 @@ import io.airbyte.config.JobSyncConfig; import io.airbyte.config.ResetSourceConfiguration; import io.airbyte.config.StandardDestinationDefinition; +import io.airbyte.config.StandardSync; import io.airbyte.config.StandardSyncInput; import io.airbyte.config.persistence.ConfigRepository; import io.airbyte.metrics.lib.ApmTraceUtils; @@ -31,8 +32,7 @@ import jakarta.inject.Singleton; import java.util.List; import java.util.Map; -import java.util.Objects; -import java.util.Optional; +import java.util.UUID; @Singleton @Requires(env = WorkerMode.CONTROL_PLANE) @@ -90,25 +90,18 @@ public GeneratedJobInput getSyncWorkflowInput(final SyncInput input) { final JobRunConfig jobRunConfig = TemporalWorkflowUtils.createJobRunConfig(jobId, attempt); - List destinationDefinitionList = configRepository.listStandardDestinationDefinitions(true); - Optional optionalDestinationDefinition = destinationDefinitionList.stream() - .filter(destinationDefinition -> config.getDestinationDockerImage() - .equalsIgnoreCase( - DockerUtils.getTaggedImageName(destinationDefinition.getDockerRepository(), destinationDefinition.getDockerImageTag()))) - .findFirst(); - final String destinationNormalizationDockerImage = optionalDestinationDefinition - .filter(standardDestinationDefinition -> Objects.nonNull(standardDestinationDefinition.getNormalizationConfig())) - .map(standardDestinationDefinition -> String.format("%s:%s", - standardDestinationDefinition.getNormalizationConfig().getNormalizationRepository(), - standardDestinationDefinition.getNormalizationConfig().getNormalizationTag())) - .orElse(null); - final boolean supportstDbt = optionalDestinationDefinition.isPresent() && Objects.nonNull(optionalDestinationDefinition.get().getSupportsDbt()) - ? optionalDestinationDefinition.get().getSupportsDbt() - : false; - final String normalizationIntegrationType = optionalDestinationDefinition - .filter(standardDestinationDefinition -> Objects.nonNull(standardDestinationDefinition.getNormalizationConfig())) - .map(standardDestinationDefinition -> standardDestinationDefinition.getNormalizationConfig().getNormalizationIntegrationType()) - .orElse(null); + final UUID connectionId = UUID.fromString(job.getScope()); + final StandardSync standardSync = configRepository.getStandardSync(connectionId); + + final StandardDestinationDefinition destinationDefinition = configRepository.getStandardDestinationDefinition(standardSync.getDestinationId()); + final String destinationNormalizationDockerImage = destinationDefinition.getNormalizationConfig() != null + ? DockerUtils.getTaggedImageName(destinationDefinition.getNormalizationConfig().getNormalizationRepository(), + destinationDefinition.getNormalizationConfig().getNormalizationTag()) + : null; + final boolean supportsDbt = destinationDefinition.getSupportsDbt(); + final String normalizationIntegrationType = + destinationDefinition.getNormalizationConfig() != null ? destinationDefinition.getNormalizationConfig().getNormalizationIntegrationType() + : null; final IntegrationLauncherConfig sourceLauncherConfig = new IntegrationLauncherConfig() .withJobId(String.valueOf(jobId)) @@ -124,13 +117,15 @@ public GeneratedJobInput getSyncWorkflowInput(final SyncInput input) { .withProtocolVersion(config.getDestinationProtocolVersion()) .withIsCustomConnector(config.getIsDestinationCustomConnector()) .withNormalizationDockerImage(destinationNormalizationDockerImage) - .withSupportsDbt(supportstDbt) + .withSupportsDbt(supportsDbt) .withNormalizationIntegrationType(normalizationIntegrationType); final StandardSyncInput syncInput = new StandardSyncInput() .withNamespaceDefinition(config.getNamespaceDefinition()) .withNamespaceFormat(config.getNamespaceFormat()) .withPrefix(config.getPrefix()) + .withSourceActorId(standardSync.getSourceId()) + .withDestinationActorId(standardSync.getDestinationId()) .withSourceConfiguration(config.getSourceConfiguration()) .withDestinationConfiguration(config.getDestinationConfiguration()) .withOperationSequence(config.getOperationSequence()) From 8d3deab49ea6fa8b5fafef46d8b7a73ed9f463ba Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Tue, 20 Dec 2022 09:31:10 -0400 Subject: [PATCH 36/63] fix: get destination definition --- .../scheduling/activities/GenerateInputActivityImpl.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 64b052631e7a..55c43eb8b4e2 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 @@ -93,7 +93,8 @@ public GeneratedJobInput getSyncWorkflowInput(final SyncInput input) { final UUID connectionId = UUID.fromString(job.getScope()); final StandardSync standardSync = configRepository.getStandardSync(connectionId); - final StandardDestinationDefinition destinationDefinition = configRepository.getStandardDestinationDefinition(standardSync.getDestinationId()); + final StandardDestinationDefinition destinationDefinition = + configRepository.getDestinationDefinitionFromDestination(standardSync.getDestinationId()); final String destinationNormalizationDockerImage = destinationDefinition.getNormalizationConfig() != null ? DockerUtils.getTaggedImageName(destinationDefinition.getNormalizationConfig().getNormalizationRepository(), destinationDefinition.getNormalizationConfig().getNormalizationTag()) From 0f8124809b77e0b43513cbb414fbfa29afe0ad4b Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Tue, 20 Dec 2022 09:54:54 -0400 Subject: [PATCH 37/63] fix null ptr --- .../scheduling/activities/GenerateInputActivityImpl.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 55c43eb8b4e2..c0a761a96262 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 @@ -99,7 +99,6 @@ public GeneratedJobInput getSyncWorkflowInput(final SyncInput input) { ? DockerUtils.getTaggedImageName(destinationDefinition.getNormalizationConfig().getNormalizationRepository(), destinationDefinition.getNormalizationConfig().getNormalizationTag()) : null; - final boolean supportsDbt = destinationDefinition.getSupportsDbt(); final String normalizationIntegrationType = destinationDefinition.getNormalizationConfig() != null ? destinationDefinition.getNormalizationConfig().getNormalizationIntegrationType() : null; @@ -118,7 +117,7 @@ public GeneratedJobInput getSyncWorkflowInput(final SyncInput input) { .withProtocolVersion(config.getDestinationProtocolVersion()) .withIsCustomConnector(config.getIsDestinationCustomConnector()) .withNormalizationDockerImage(destinationNormalizationDockerImage) - .withSupportsDbt(supportsDbt) + .withSupportsDbt(destinationDefinition.getSupportsDbt()) .withNormalizationIntegrationType(normalizationIntegrationType); final StandardSyncInput syncInput = new StandardSyncInput() From 8b64f4a4ae2edd9733b92d7f11b593ea10b7acb8 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Tue, 20 Dec 2022 10:37:37 -0400 Subject: [PATCH 38/63] remove "actor" from naming --- .../src/main/java/io/airbyte/workers/WorkerUtils.java | 4 ++-- .../io/airbyte/workers/test_utils/TestConfigHelpers.java | 4 ++-- .../workers/general/DefaultReplicationWorkerTest.java | 8 ++++---- .../src/main/resources/types/StandardSyncInput.yaml | 8 ++++---- .../src/main/resources/types/WorkerDestinationConfig.yaml | 4 ++-- .../src/main/resources/types/WorkerSourceConfig.yaml | 4 ++-- .../scheduling/activities/GenerateInputActivityImpl.java | 4 ++-- 7 files changed, 18 insertions(+), 18 deletions(-) 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 993ef17b754f..216371c248b7 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 @@ -91,7 +91,7 @@ public static void cancelProcess(final Process process) { */ public static WorkerSourceConfig syncToWorkerSourceConfig(final StandardSyncInput sync) { return new WorkerSourceConfig() - .withActorId(sync.getSourceActorId()) + .withSourceId(sync.getSourceId()) .withSourceConnectionConfiguration(sync.getSourceConfiguration()) .withCatalog(sync.getCatalog()) .withState(sync.getState()); @@ -103,7 +103,7 @@ public static WorkerSourceConfig syncToWorkerSourceConfig(final StandardSyncInpu */ public static WorkerDestinationConfig syncToWorkerDestinationConfig(final StandardSyncInput sync) { return new WorkerDestinationConfig() - .withActorId(sync.getDestinationActorId()) + .withDestinationId(sync.getDestinationId()) .withDestinationConnectionConfiguration(sync.getDestinationConfiguration()) .withCatalog(sync.getCatalog()) .withState(sync.getState()); diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/test_utils/TestConfigHelpers.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/test_utils/TestConfigHelpers.java index aa82889da75a..afac5fb4f401 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/test_utils/TestConfigHelpers.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/test_utils/TestConfigHelpers.java @@ -128,8 +128,8 @@ public static ImmutablePair createSyncConfig(fi final StandardSyncInput syncInput = new StandardSyncInput() .withNamespaceDefinition(standardSync.getNamespaceDefinition()) .withPrefix(standardSync.getPrefix()) - .withSourceActorId(sourceId) - .withDestinationActorId(destinationId) + .withSourceId(sourceId) + .withDestinationId(destinationId) .withDestinationConfiguration(destinationConnectionConfig.getConfiguration()) .withCatalog(standardSync.getCatalog()) .withSourceConfiguration(sourceConnectionConfig.getConfiguration()) diff --git a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java index 1d8524b3e9d7..97abbe04c5bf 100644 --- a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java +++ b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java @@ -271,7 +271,7 @@ void testReplicationRunnableSourceUpdateConfig() throws Exception { final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.COMPLETED, output.getReplicationAttemptSummary().getStatus()); - verify(updateConnectorConfigHelper).updateSource(syncInput.getSourceActorId(), CONNECTOR_CONFIG); + verify(updateConnectorConfigHelper).updateSource(syncInput.getSourceId(), CONNECTOR_CONFIG); } @Test @@ -298,7 +298,7 @@ void testReplicationFailureOnSourceConfigPersistError() throws Exception { assertTrue(output.getFailures().stream() .anyMatch(f -> f.getFailureOrigin().equals(FailureOrigin.REPLICATION) && f.getStacktrace().contains(PERSIST_ERROR_MESSAGE))); - verify(updateConnectorConfigHelper).updateSource(syncInput.getSourceActorId(), CONNECTOR_CONFIG); + verify(updateConnectorConfigHelper).updateSource(syncInput.getSourceId(), CONNECTOR_CONFIG); } @Test @@ -320,7 +320,7 @@ void testReplicationRunnableDestinationUpdateConfig() throws Exception { final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.COMPLETED, output.getReplicationAttemptSummary().getStatus()); - verify(updateConnectorConfigHelper).updateDestination(syncInput.getDestinationActorId(), CONNECTOR_CONFIG); + verify(updateConnectorConfigHelper).updateDestination(syncInput.getDestinationId(), CONNECTOR_CONFIG); } @Test @@ -347,7 +347,7 @@ void testReplicationFailureOnDestinationConfigPersistError() throws Exception { assertTrue(output.getFailures().stream() .anyMatch(f -> f.getFailureOrigin().equals(FailureOrigin.REPLICATION) && f.getStacktrace().contains(PERSIST_ERROR_MESSAGE))); - verify(updateConnectorConfigHelper).updateDestination(syncInput.getDestinationActorId(), CONNECTOR_CONFIG); + verify(updateConnectorConfigHelper).updateDestination(syncInput.getDestinationId(), CONNECTOR_CONFIG); } @Test diff --git a/airbyte-config/config-models/src/main/resources/types/StandardSyncInput.yaml b/airbyte-config/config-models/src/main/resources/types/StandardSyncInput.yaml index 75940f7a7886..f682e92d143d 100644 --- a/airbyte-config/config-models/src/main/resources/types/StandardSyncInput.yaml +++ b/airbyte-config/config-models/src/main/resources/types/StandardSyncInput.yaml @@ -6,9 +6,9 @@ description: job sync config type: object additionalProperties: false required: - - sourceActorId + - sourceId - sourceConfiguration - - destinationActorId + - destinationId - destinationConfiguration - catalog properties: @@ -21,11 +21,11 @@ properties: prefix: description: Prefix that will be prepended to the name of each stream when it is written to the destination. type: string - sourceActorId: + sourceId: description: Actor ID for the source used in the sync - this is used to update the actor configuration when requested. type: string format: uuid - destinationActorId: + destinationId: description: Actor ID for the destination used in the sync - this is used to update the actor configuration when requested. type: string format: uuid diff --git a/airbyte-config/config-models/src/main/resources/types/WorkerDestinationConfig.yaml b/airbyte-config/config-models/src/main/resources/types/WorkerDestinationConfig.yaml index d2afeb4e290c..d2e3c0f6b50e 100644 --- a/airbyte-config/config-models/src/main/resources/types/WorkerDestinationConfig.yaml +++ b/airbyte-config/config-models/src/main/resources/types/WorkerDestinationConfig.yaml @@ -6,12 +6,12 @@ description: WorkerDestinationConfig type: object additionalProperties: false required: - - actorId + - destinationId - destinationConnectionConfiguration - catalog - connectionId properties: - actorId: + destinationId: type: string format: uuid destinationConnectionConfiguration: diff --git a/airbyte-config/config-models/src/main/resources/types/WorkerSourceConfig.yaml b/airbyte-config/config-models/src/main/resources/types/WorkerSourceConfig.yaml index 9ee1bace575f..043bec51dea0 100644 --- a/airbyte-config/config-models/src/main/resources/types/WorkerSourceConfig.yaml +++ b/airbyte-config/config-models/src/main/resources/types/WorkerSourceConfig.yaml @@ -6,11 +6,11 @@ description: WorkerSourceConfig type: object additionalProperties: false required: - - actorId + - sourceId - sourceConnectionConfiguration - catalog properties: - actorId: + sourceId: type: string format: uuid sourceConnectionConfiguration: 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 c0a761a96262..c14d3780889b 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 @@ -124,8 +124,8 @@ public GeneratedJobInput getSyncWorkflowInput(final SyncInput input) { .withNamespaceDefinition(config.getNamespaceDefinition()) .withNamespaceFormat(config.getNamespaceFormat()) .withPrefix(config.getPrefix()) - .withSourceActorId(standardSync.getSourceId()) - .withDestinationActorId(standardSync.getDestinationId()) + .withSourceId(standardSync.getSourceId()) + .withDestinationId(standardSync.getDestinationId()) .withSourceConfiguration(config.getSourceConfiguration()) .withDestinationConfiguration(config.getDestinationConfiguration()) .withOperationSequence(config.getOperationSequence()) From e4e3584276382caf9e2c529eae0513df6f2c934b Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Tue, 20 Dec 2022 10:55:38 -0400 Subject: [PATCH 39/63] fix missing change from rename --- .../io/airbyte/workers/general/DefaultReplicationWorker.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java index f97220079a37..58e8070428f1 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java @@ -416,7 +416,7 @@ private static void acceptSrcControlMessage(final WorkerSourceConfig sourceConfi final AirbyteControlMessage controlMessage, final UpdateConnectorConfigHelper updateConnectorConfigHelper) { if (controlMessage.getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG) { - updateConnectorConfigHelper.updateSource(sourceConfig.getActorId(), controlMessage.getConnectorConfig().getConfig()); + updateConnectorConfigHelper.updateSource(sourceConfig.getSourceId(), controlMessage.getConnectorConfig().getConfig()); } } @@ -424,7 +424,7 @@ private static void acceptDstControlMessage(final WorkerDestinationConfig destin final AirbyteControlMessage controlMessage, final UpdateConnectorConfigHelper updateConnectorConfigHelper) { if (controlMessage.getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG) { - updateConnectorConfigHelper.updateDestination(destinationConfig.getActorId(), controlMessage.getConnectorConfig().getConfig()); + updateConnectorConfigHelper.updateDestination(destinationConfig.getDestinationId(), controlMessage.getConnectorConfig().getConfig()); } } From 875228eb8a7ffc54a13da903236cc6acd9a54843 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Tue, 20 Dec 2022 11:33:36 -0400 Subject: [PATCH 40/63] revert ContainerOrchestratorConfigBeanFactory changes --- ...ontainerOrchestratorConfigBeanFactory.java | 27 +------------------ .../src/main/resources/application.yml | 5 ---- 2 files changed, 1 insertion(+), 31 deletions(-) diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/config/ContainerOrchestratorConfigBeanFactory.java b/airbyte-workers/src/main/java/io/airbyte/workers/config/ContainerOrchestratorConfigBeanFactory.java index e852b1839f87..d9259005b3ce 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/config/ContainerOrchestratorConfigBeanFactory.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/config/ContainerOrchestratorConfigBeanFactory.java @@ -15,7 +15,6 @@ import io.micronaut.context.annotation.Factory; import io.micronaut.context.annotation.Requires; import io.micronaut.context.annotation.Value; -import io.micronaut.context.env.Environment; import io.micronaut.core.util.StringUtils; import jakarta.inject.Named; import jakarta.inject.Singleton; @@ -38,12 +37,6 @@ public class ContainerOrchestratorConfigBeanFactory { private static final String DD_VERSION_ENV_VAR = "DD_VERSION"; private static final String JAVA_OPTS_ENV_VAR = "JAVA_OPTS"; private static final String PUBLISH_METRICS_ENV_VAR = "PUBLISH_METRICS"; - private static final String CONTROL_PLANE_AUTH_ENDPOINT_ENV_VAR = "CONTROL_PLANE_AUTH_ENDPOINT"; - private static final String DATA_PLANE_SERVICE_ACCOUNT_CREDENTIALS_PATH_ENV_VAR = "DATA_PLANE_SERVICE_ACCOUNT_CREDENTIALS_PATH"; - private static final String DATA_PLANE_SERVICE_ACCOUNT_EMAIL_ENV_VAR = "DATA_PLANE_SERVICE_ACCOUNT_EMAIL"; - private static final String AIRBYTE_API_AUTH_HEADER_NAME_ENV_VAR = "AIRBYTE_API_AUTH_HEADER_NAME"; - private static final String AIRBYTE_API_AUTH_HEADER_VALUE_ENV_VAR = "AIRBYTE_API_AUTH_HEADER_VALUE"; - private static final String INTERNAL_API_HOST_ENV_VAR = "INTERNAL_API_HOST"; // IMPORTANT: Changing the storage location will orphan already existing kube pods when the new // version is deployed! @@ -68,13 +61,7 @@ public ContainerOrchestratorConfig kubernetesContainerOrchestratorConfig( @Value("${airbyte.metric.should-publish}") final String shouldPublishMetrics, final FeatureFlags featureFlags, @Value("${airbyte.container.orchestrator.java-opts}") final String containerOrchestratorJavaOpts, - final WorkerEnvironment workerEnvironment, - @Value("${airbyte.container.orchestrator.api.host}") final String containerOrchestratorApiHost, - @Value("${airbyte.container.orchestrator.api.auth-header.name}") final String containerOrchestratorApiAuthHeaderName, - @Value("${airbyte.container.orchestrator.api.auth-header.value}") final String containerOrchestratorApiAuthHeaderValue, - @Value("${airbyte.control.plane.auth-endpoint}") final String controlPlaneAuthEndpoint, - @Value("${airbyte.data.plane.service-account.email}") final String dataPlaneServiceAccountEmail, - @Value("${airbyte.data.plane.service-account.credentials-path}") final String dataPlaneServiceAccountCredentialsPath) { + final WorkerEnvironment workerEnvironment) { final var kubernetesClient = new DefaultKubernetesClient(); final DocumentStoreClient documentStoreClient = StateClients.create( @@ -100,18 +87,6 @@ public ContainerOrchestratorConfig kubernetesContainerOrchestratorConfig( environmentVariables.put(DD_VERSION_ENV_VAR, System.getenv(DD_VERSION_ENV_VAR)); } - // Environment variables for ApiClientBeanFactory - environmentVariables.put(CONTROL_PLANE_AUTH_ENDPOINT_ENV_VAR, controlPlaneAuthEndpoint); - environmentVariables.put(DATA_PLANE_SERVICE_ACCOUNT_CREDENTIALS_PATH_ENV_VAR, dataPlaneServiceAccountCredentialsPath); - environmentVariables.put(DATA_PLANE_SERVICE_ACCOUNT_EMAIL_ENV_VAR, dataPlaneServiceAccountEmail); - environmentVariables.put(AIRBYTE_API_AUTH_HEADER_NAME_ENV_VAR, containerOrchestratorApiAuthHeaderName); - environmentVariables.put(AIRBYTE_API_AUTH_HEADER_VALUE_ENV_VAR, containerOrchestratorApiAuthHeaderValue); - environmentVariables.put(INTERNAL_API_HOST_ENV_VAR, containerOrchestratorApiHost); - - if (System.getenv(Environment.ENVIRONMENTS_ENV) != null) { - environmentVariables.put(Environment.ENVIRONMENTS_ENV, System.getenv(Environment.ENVIRONMENTS_ENV)); - } - return new ContainerOrchestratorConfig( namespace, documentStoreClient, diff --git a/airbyte-workers/src/main/resources/application.yml b/airbyte-workers/src/main/resources/application.yml index cc7d4099e60c..43708b53d805 100644 --- a/airbyte-workers/src/main/resources/application.yml +++ b/airbyte-workers/src/main/resources/application.yml @@ -52,11 +52,6 @@ airbyte: specific-resource-defaults-enabled: ${CONNECTOR_SPECIFIC_RESOURCE_DEFAULTS_ENABLED:false} container: orchestrator: - api: - auth-header: - name: ${AIRBYTE_API_AUTH_HEADER_NAME:} - value: ${AIRBYTE_API_AUTH_HEADER_VALUE:} - host: ${CONTAINER_ORCHESTRATOR_API_HOST:`airbyte-server-svc.ab:80`} enabled: ${CONTAINER_ORCHESTRATOR_ENABLED:false} image: ${CONTAINER_ORCHESTRATOR_IMAGE:} java-opts: ${CONTAINER_ORCHESTRATOR_JAVA_OPTS:} From 6ecac44ac2f7c0d1e2ea4be33cb5079fce913b00 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 21 Dec 2022 17:06:06 -0400 Subject: [PATCH 41/63] inject sourceapi/destinationapi directly rather than airbyteapiclient --- .../workers/config/ApiClientBeanFactory.java | 8 +++++++- .../helper/UpdateConnectorConfigHelper.java | 10 +++++----- .../helper/UpdateConnectorConfigHelperTest.java | 7 +------ .../config/ContainerOrchestratorFactory.java | 8 +++++--- .../orchestrator/ReplicationJobOrchestrator.java | 16 ++++++++++------ .../temporal/sync/ReplicationActivityImpl.java | 2 +- 6 files changed, 29 insertions(+), 22 deletions(-) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/config/ApiClientBeanFactory.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/config/ApiClientBeanFactory.java index c35c88dc1e10..2ffd22899ad5 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/config/ApiClientBeanFactory.java +++ b/airbyte-commons-worker/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.api.client.generated.DestinationApi; import io.airbyte.api.client.generated.SourceApi; import io.airbyte.api.client.invoker.generated.ApiClient; import io.airbyte.commons.temporal.config.WorkerMode; @@ -58,7 +59,7 @@ public ApiClient apiClient(@Value("${airbyte.internal.api.auth-header.name}") fi } @Singleton - public AirbyteApiClient airbyteApiClient(ApiClient apiClient) { + public AirbyteApiClient airbyteApiClient(final ApiClient apiClient) { return new AirbyteApiClient(apiClient); } @@ -67,6 +68,11 @@ public SourceApi sourceApi(final ApiClient apiClient) { return new SourceApi(apiClient); } + @Singleton + public DestinationApi destinationApi(final ApiClient apiClient) { + return new DestinationApi(apiClient); + } + @Singleton public HttpClient httpClient() { return HttpClient.newHttpClient(); diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/UpdateConnectorConfigHelper.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/UpdateConnectorConfigHelper.java index 22b06217f077..9ef5d9c0095f 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/UpdateConnectorConfigHelper.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/UpdateConnectorConfigHelper.java @@ -33,10 +33,12 @@ public class UpdateConnectorConfigHelper { private static final Logger LOGGER = LoggerFactory.getLogger(UpdateConnectorConfigHelper.class); - private final AirbyteApiClient apiClient; + private final SourceApi sourceApi; + private final DestinationApi destinationApi; - public UpdateConnectorConfigHelper(final AirbyteApiClient apiClient) { - this.apiClient = apiClient; + public UpdateConnectorConfigHelper(final SourceApi sourceApi, final DestinationApi destinationApi) { + this.sourceApi = sourceApi; + this.destinationApi = destinationApi; } /** @@ -44,7 +46,6 @@ public UpdateConnectorConfigHelper(final AirbyteApiClient apiClient) { * parameters will be masked when saving. */ public void updateSource(final UUID sourceId, final Config config) { - final SourceApi sourceApi = apiClient.getSourceApi(); final SourceRead source = AirbyteApiClient.retryWithJitter( () -> sourceApi.getSource(new SourceIdRequestBody().sourceId(sourceId)), "get source"); @@ -67,7 +68,6 @@ public void updateSource(final UUID sourceId, final Config config) { * parameters will be masked when saving. */ public void updateDestination(final UUID destinationId, final Config config) { - final DestinationApi destinationApi = apiClient.getDestinationApi(); final DestinationRead destination = AirbyteApiClient.retryWithJitter( () -> destinationApi.getDestination(new DestinationIdRequestBody().destinationId(destinationId)), "get destination"); diff --git a/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/UpdateConnectorConfigHelperTest.java b/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/UpdateConnectorConfigHelperTest.java index cbe9bf596a7d..81db2b47cc10 100644 --- a/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/UpdateConnectorConfigHelperTest.java +++ b/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/UpdateConnectorConfigHelperTest.java @@ -9,7 +9,6 @@ import static org.mockito.Mockito.when; import com.fasterxml.jackson.databind.JsonNode; -import io.airbyte.api.client.AirbyteApiClient; import io.airbyte.api.client.generated.DestinationApi; import io.airbyte.api.client.generated.SourceApi; import io.airbyte.api.client.invoker.generated.ApiException; @@ -33,7 +32,6 @@ class UpdateConnectorConfigHelperTest { private static final UUID DESTINATION_ID = UUID.randomUUID(); private static final String DESTINATION_NAME = "destination-google-sheets"; - private final AirbyteApiClient airbyteApiClient = mock(AirbyteApiClient.class); private final SourceApi mSourceApi = mock(SourceApi.class); private final DestinationApi mDestinationApi = mock(DestinationApi.class); @@ -41,9 +39,6 @@ class UpdateConnectorConfigHelperTest { @BeforeEach void setUp() throws ApiException { - when(airbyteApiClient.getSourceApi()).thenReturn(mSourceApi); - when(airbyteApiClient.getDestinationApi()).thenReturn(mDestinationApi); - when(mSourceApi.getSource(new SourceIdRequestBody() .sourceId(SOURCE_ID))).thenReturn(new SourceRead() .sourceId(SOURCE_ID) @@ -54,7 +49,7 @@ void setUp() throws ApiException { .destinationId(DESTINATION_ID) .name(DESTINATION_NAME)); - updateConnectorConfigHelper = new UpdateConnectorConfigHelper(airbyteApiClient); + updateConnectorConfigHelper = new UpdateConnectorConfigHelper(mSourceApi, mDestinationApi); } @Test diff --git a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/config/ContainerOrchestratorFactory.java b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/config/ContainerOrchestratorFactory.java index e266670de9e5..776bf81c73fa 100644 --- a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/config/ContainerOrchestratorFactory.java +++ b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/config/ContainerOrchestratorFactory.java @@ -4,7 +4,8 @@ package io.airbyte.container_orchestrator.config; -import io.airbyte.api.client.AirbyteApiClient; +import io.airbyte.api.client.generated.DestinationApi; +import io.airbyte.api.client.generated.SourceApi; import io.airbyte.commons.features.EnvVariableFeatureFlags; import io.airbyte.commons.features.FeatureFlags; import io.airbyte.commons.protocol.AirbyteMessageSerDeProvider; @@ -102,10 +103,11 @@ JobOrchestrator jobOrchestrator( final AirbyteMessageSerDeProvider serdeProvider, final AirbyteMessageVersionedMigratorFactory migratorFactory, final JobRunConfig jobRunConfig, - final AirbyteApiClient airbyteApiClient) { + final SourceApi sourceApi, + final DestinationApi destinationApi) { return switch (application) { case ReplicationLauncherWorker.REPLICATION -> new ReplicationJobOrchestrator(envConfigs, processFactory, featureFlags, serdeProvider, - migratorFactory, jobRunConfig, airbyteApiClient); + migratorFactory, jobRunConfig, sourceApi, destinationApi); case NormalizationLauncherWorker.NORMALIZATION -> new NormalizationJobOrchestrator(envConfigs, processFactory, jobRunConfig); case DbtLauncherWorker.DBT -> new DbtJobOrchestrator(envConfigs, workerConfigs, processFactory, jobRunConfig); case AsyncOrchestratorPodProcess.NO_OP -> new NoOpOrchestrator(); diff --git a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java index f00a944adbc0..32b63e940dbb 100644 --- a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java +++ b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java @@ -10,7 +10,8 @@ import static io.airbyte.metrics.lib.ApmTraceConstants.Tags.SOURCE_DOCKER_IMAGE_KEY; import datadog.trace.api.Trace; -import io.airbyte.api.client.AirbyteApiClient; +import io.airbyte.api.client.generated.DestinationApi; +import io.airbyte.api.client.generated.SourceApi; import io.airbyte.commons.features.FeatureFlags; import io.airbyte.commons.json.Jsons; import io.airbyte.commons.logging.MdcScope; @@ -61,7 +62,8 @@ public class ReplicationJobOrchestrator implements JobOrchestrator runJob() throws Exception { // At this moment, if either source or destination is from custom connector image, we will put all // jobs into isolated pool to run. - boolean useIsolatedPool = sourceLauncherConfig.getIsCustomConnector() || destinationLauncherConfig.getIsCustomConnector(); + final boolean useIsolatedPool = sourceLauncherConfig.getIsCustomConnector() || destinationLauncherConfig.getIsCustomConnector(); log.info("Setting up source launcher..."); final var sourceLauncher = new AirbyteIntegrationLauncher( sourceLauncherConfig.getJobId(), @@ -154,7 +158,7 @@ public Optional runJob() throws Exception { new AirbyteMessageTracker(), new RecordSchemaValidator(WorkerUtils.mapStreamNamesToSchemas(syncInput)), metricReporter, - new UpdateConnectorConfigHelper(airbyteApiClient), + new UpdateConnectorConfigHelper(sourceApi, destinationApi), featureFlags.applyFieldSelection()); log.info("Running replication worker..."); diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java index 9650fcdd17fb..f0f48ea527d5 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java @@ -306,7 +306,7 @@ private CheckedSupplier, Exception> new AirbyteMessageTracker(), new RecordSchemaValidator(WorkerUtils.mapStreamNamesToSchemas(syncInput)), metricReporter, - new UpdateConnectorConfigHelper(airbyteApiClient), false); + new UpdateConnectorConfigHelper(airbyteApiClient.getSourceApi(), airbyteApiClient.getDestinationApi()), false); }; } From c20f61e673363d293d33ea3ccf7b5c6fb38c0fea Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 21 Dec 2022 17:09:32 -0400 Subject: [PATCH 42/63] UpdateConnectorConfigHelper -> ConnectorConfigUpdater --- .../general/DefaultReplicationWorker.java | 28 ++++---- ...elper.java => ConnectorConfigUpdater.java} | 6 +- .../general/DefaultReplicationWorkerTest.java | 68 +++++++++---------- ...t.java => ConnectorConfigUpdaterTest.java} | 10 +-- .../ReplicationJobOrchestrator.java | 4 +- .../sync/ReplicationActivityImpl.java | 4 +- 6 files changed, 60 insertions(+), 60 deletions(-) rename airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/{UpdateConnectorConfigHelper.java => ConnectorConfigUpdater.java} (93%) rename airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/{UpdateConnectorConfigHelperTest.java => ConnectorConfigUpdaterTest.java} (89%) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java index 58e8070428f1..fa3bdba52f41 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java @@ -37,9 +37,9 @@ import io.airbyte.workers.WorkerUtils; import io.airbyte.workers.exception.RecordSchemaValidationException; import io.airbyte.workers.exception.WorkerException; +import io.airbyte.workers.helper.ConnectorConfigUpdater; import io.airbyte.workers.helper.FailureHelper; import io.airbyte.workers.helper.ThreadedTimeTracker; -import io.airbyte.workers.helper.UpdateConnectorConfigHelper; import io.airbyte.workers.internal.AirbyteDestination; import io.airbyte.workers.internal.AirbyteMapper; import io.airbyte.workers.internal.AirbyteSource; @@ -100,7 +100,7 @@ public class DefaultReplicationWorker implements ReplicationWorker { private final AtomicBoolean hasFailed; private final RecordSchemaValidator recordSchemaValidator; private final WorkerMetricReporter metricReporter; - private final UpdateConnectorConfigHelper updateConnectorConfigHelper; + private final ConnectorConfigUpdater connectorConfigUpdater; private final boolean fieldSelectionEnabled; public DefaultReplicationWorker(final String jobId, @@ -111,7 +111,7 @@ public DefaultReplicationWorker(final String jobId, final MessageTracker messageTracker, final RecordSchemaValidator recordSchemaValidator, final WorkerMetricReporter metricReporter, - final UpdateConnectorConfigHelper updateConnectorConfigHelper, + final ConnectorConfigUpdater connectorConfigUpdater, final boolean fieldSelectionEnabled) { this.jobId = jobId; this.attempt = attempt; @@ -122,7 +122,7 @@ public DefaultReplicationWorker(final String jobId, this.executors = Executors.newFixedThreadPool(2); this.recordSchemaValidator = recordSchemaValidator; this.metricReporter = metricReporter; - this.updateConnectorConfigHelper = updateConnectorConfigHelper; + this.connectorConfigUpdater = connectorConfigUpdater; this.fieldSelectionEnabled = fieldSelectionEnabled; this.cancelled = new AtomicBoolean(false); @@ -196,7 +196,7 @@ private void replicate(final Path jobRoot, // note: `whenComplete` is used instead of `exceptionally` so that the original exception is still // thrown final CompletableFuture readFromDstThread = CompletableFuture.runAsync( - readFromDstRunnable(destination, cancelled, messageTracker, updateConnectorConfigHelper, mdc, timeTracker, destinationConfig), + readFromDstRunnable(destination, cancelled, messageTracker, connectorConfigUpdater, mdc, timeTracker, destinationConfig), executors) .whenComplete((msg, ex) -> { if (ex != null) { @@ -217,7 +217,7 @@ private void replicate(final Path jobRoot, cancelled, mapper, messageTracker, - updateConnectorConfigHelper, + connectorConfigUpdater, mdc, recordSchemaValidator, metricReporter, @@ -260,7 +260,7 @@ private void replicate(final Path jobRoot, private static Runnable readFromDstRunnable(final AirbyteDestination destination, final AtomicBoolean cancelled, final MessageTracker messageTracker, - final UpdateConnectorConfigHelper updateConnectorConfigHelper, + final ConnectorConfigUpdater connectorConfigUpdater, final Map mdc, final ThreadedTimeTracker timeHolder, final WorkerDestinationConfig destinationConfig) { @@ -282,7 +282,7 @@ private static Runnable readFromDstRunnable(final AirbyteDestination destination messageTracker.acceptFromDestination(message); if (message.getType() == Type.CONTROL) { - acceptDstControlMessage(destinationConfig, message.getControl(), updateConnectorConfigHelper); + acceptDstControlMessage(destinationConfig, message.getControl(), connectorConfigUpdater); } } } @@ -315,7 +315,7 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou final AtomicBoolean cancelled, final AirbyteMapper mapper, final MessageTracker messageTracker, - final UpdateConnectorConfigHelper updateConnectorConfigHelper, + final ConnectorConfigUpdater connectorConfigUpdater, final Map mdc, final RecordSchemaValidator recordSchemaValidator, final WorkerMetricReporter metricReporter, @@ -351,7 +351,7 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou messageTracker.acceptFromSource(message); if (message.getType() == Type.CONTROL) { - acceptSrcControlMessage(sourceConfig, message.getControl(), updateConnectorConfigHelper); + acceptSrcControlMessage(sourceConfig, message.getControl(), connectorConfigUpdater); } try { @@ -414,17 +414,17 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou private static void acceptSrcControlMessage(final WorkerSourceConfig sourceConfig, final AirbyteControlMessage controlMessage, - final UpdateConnectorConfigHelper updateConnectorConfigHelper) { + final ConnectorConfigUpdater connectorConfigUpdater) { if (controlMessage.getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG) { - updateConnectorConfigHelper.updateSource(sourceConfig.getSourceId(), controlMessage.getConnectorConfig().getConfig()); + connectorConfigUpdater.updateSource(sourceConfig.getSourceId(), controlMessage.getConnectorConfig().getConfig()); } } private static void acceptDstControlMessage(final WorkerDestinationConfig destinationConfig, final AirbyteControlMessage controlMessage, - final UpdateConnectorConfigHelper updateConnectorConfigHelper) { + final ConnectorConfigUpdater connectorConfigUpdater) { if (controlMessage.getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG) { - updateConnectorConfigHelper.updateDestination(destinationConfig.getDestinationId(), controlMessage.getConnectorConfig().getConfig()); + connectorConfigUpdater.updateDestination(destinationConfig.getDestinationId(), controlMessage.getConnectorConfig().getConfig()); } } diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/UpdateConnectorConfigHelper.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/ConnectorConfigUpdater.java similarity index 93% rename from airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/UpdateConnectorConfigHelper.java rename to airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/ConnectorConfigUpdater.java index 9ef5d9c0095f..c878249ea0ca 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/UpdateConnectorConfigHelper.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/helper/ConnectorConfigUpdater.java @@ -29,14 +29,14 @@ * useful for migrating configuration to a new version or for enabling connectors that require * single-use or short-lived OAuth tokens. */ -public class UpdateConnectorConfigHelper { +public class ConnectorConfigUpdater { - private static final Logger LOGGER = LoggerFactory.getLogger(UpdateConnectorConfigHelper.class); + private static final Logger LOGGER = LoggerFactory.getLogger(ConnectorConfigUpdater.class); private final SourceApi sourceApi; private final DestinationApi destinationApi; - public UpdateConnectorConfigHelper(final SourceApi sourceApi, final DestinationApi destinationApi) { + public ConnectorConfigUpdater(final SourceApi sourceApi, final DestinationApi destinationApi) { this.sourceApi = sourceApi; this.destinationApi = destinationApi; } diff --git a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java index 97abbe04c5bf..fb98f8c541f5 100644 --- a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java +++ b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java @@ -50,8 +50,8 @@ import io.airbyte.validation.json.JsonSchemaValidator; import io.airbyte.workers.*; import io.airbyte.workers.exception.WorkerException; +import io.airbyte.workers.helper.ConnectorConfigUpdater; import io.airbyte.workers.helper.FailureHelper; -import io.airbyte.workers.helper.UpdateConnectorConfigHelper; import io.airbyte.workers.internal.AirbyteDestination; import io.airbyte.workers.internal.AirbyteSource; import io.airbyte.workers.internal.NamespacingMapper; @@ -112,7 +112,7 @@ class DefaultReplicationWorkerTest { private RecordSchemaValidator recordSchemaValidator; private MetricClient metricClient; private WorkerMetricReporter workerMetricReporter; - private UpdateConnectorConfigHelper updateConnectorConfigHelper; + private ConnectorConfigUpdater connectorConfigUpdater; @SuppressWarnings("unchecked") @BeforeEach @@ -132,7 +132,7 @@ void setup() throws Exception { destination = mock(AirbyteDestination.class); messageTracker = mock(AirbyteMessageTracker.class); recordSchemaValidator = mock(RecordSchemaValidator.class); - updateConnectorConfigHelper = mock(UpdateConnectorConfigHelper.class); + connectorConfigUpdater = mock(ConnectorConfigUpdater.class); metricClient = MetricClientFactory.getMetricClient(); workerMetricReporter = new WorkerMetricReporter(metricClient, "docker_image:v1.0.0"); @@ -163,7 +163,7 @@ void test() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); worker.run(syncInput, jobRoot); @@ -192,7 +192,7 @@ void testInvalidSchema() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, + connectorConfigUpdater, false); worker.run(syncInput, jobRoot); @@ -224,7 +224,7 @@ void testSourceNonZeroExitValue() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); assertTrue(output.getFailures().stream().anyMatch(f -> f.getFailureOrigin().equals(FailureOrigin.SOURCE))); @@ -245,7 +245,7 @@ void testReplicationRunnableSourceFailure() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); @@ -266,12 +266,12 @@ void testReplicationRunnableSourceUpdateConfig() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.COMPLETED, output.getReplicationAttemptSummary().getStatus()); - verify(updateConnectorConfigHelper).updateSource(syncInput.getSourceId(), CONNECTOR_CONFIG); + verify(connectorConfigUpdater).updateSource(syncInput.getSourceId(), CONNECTOR_CONFIG); } @Test @@ -280,7 +280,7 @@ void testReplicationFailureOnSourceConfigPersistError() throws Exception { when(source.isFinished()).thenReturn(false, true); final String PERSIST_ERROR_MESSAGE = "there was a problem persisting the new config"; - doThrow(new RuntimeException(PERSIST_ERROR_MESSAGE)).when(updateConnectorConfigHelper).updateSource(Mockito.any(), Mockito.any()); + doThrow(new RuntimeException(PERSIST_ERROR_MESSAGE)).when(connectorConfigUpdater).updateSource(Mockito.any(), Mockito.any()); final ReplicationWorker worker = new DefaultReplicationWorker( JOB_ID, @@ -291,14 +291,14 @@ void testReplicationFailureOnSourceConfigPersistError() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); assertTrue(output.getFailures().stream() .anyMatch(f -> f.getFailureOrigin().equals(FailureOrigin.REPLICATION) && f.getStacktrace().contains(PERSIST_ERROR_MESSAGE))); - verify(updateConnectorConfigHelper).updateSource(syncInput.getSourceId(), CONNECTOR_CONFIG); + verify(connectorConfigUpdater).updateSource(syncInput.getSourceId(), CONNECTOR_CONFIG); } @Test @@ -315,12 +315,12 @@ void testReplicationRunnableDestinationUpdateConfig() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.COMPLETED, output.getReplicationAttemptSummary().getStatus()); - verify(updateConnectorConfigHelper).updateDestination(syncInput.getDestinationId(), CONNECTOR_CONFIG); + verify(connectorConfigUpdater).updateDestination(syncInput.getDestinationId(), CONNECTOR_CONFIG); } @Test @@ -329,7 +329,7 @@ void testReplicationFailureOnDestinationConfigPersistError() throws Exception { when(destination.isFinished()).thenReturn(false, true); final String PERSIST_ERROR_MESSAGE = "there was a problem persisting the new config"; - doThrow(new RuntimeException(PERSIST_ERROR_MESSAGE)).when(updateConnectorConfigHelper).updateDestination(Mockito.any(), Mockito.any()); + doThrow(new RuntimeException(PERSIST_ERROR_MESSAGE)).when(connectorConfigUpdater).updateDestination(Mockito.any(), Mockito.any()); final ReplicationWorker worker = new DefaultReplicationWorker( JOB_ID, @@ -340,14 +340,14 @@ void testReplicationFailureOnDestinationConfigPersistError() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); assertTrue(output.getFailures().stream() .anyMatch(f -> f.getFailureOrigin().equals(FailureOrigin.REPLICATION) && f.getStacktrace().contains(PERSIST_ERROR_MESSAGE))); - verify(updateConnectorConfigHelper).updateDestination(syncInput.getDestinationId(), CONNECTOR_CONFIG); + verify(connectorConfigUpdater).updateDestination(syncInput.getDestinationId(), CONNECTOR_CONFIG); } @Test @@ -365,7 +365,7 @@ void testReplicationRunnableDestinationFailure() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); @@ -387,7 +387,7 @@ void testReplicationRunnableDestinationFailureViaTraceMessage() throws Exception messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertTrue(output.getFailures().stream() @@ -410,7 +410,7 @@ void testReplicationRunnableWorkerFailure() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); @@ -437,7 +437,7 @@ void testOnlyStateAndRecordMessagesDeliveredToDestination() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); worker.run(syncInput, jobRoot); @@ -471,7 +471,7 @@ void testOnlySelectedFieldsDeliveredToDestinationWithFieldSelectionEnabled() thr messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, true); + connectorConfigUpdater, true); worker.run(syncInput, jobRoot); @@ -502,7 +502,7 @@ void testAllFieldsDeliveredWithFieldSelectionDisabled() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); worker.run(syncInput, jobRoot); @@ -524,7 +524,7 @@ void testDestinationNonZeroExitValue() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); @@ -546,7 +546,7 @@ void testDestinationRunnableDestinationFailure() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); @@ -569,7 +569,7 @@ void testDestinationRunnableWorkerFailure() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); final ReplicationOutput output = worker.run(syncInput, jobRoot); assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); @@ -593,7 +593,7 @@ void testLoggingInThreads() throws IOException, WorkerException { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); worker.run(syncInput, jobRoot); @@ -635,7 +635,7 @@ void testCancellation() throws InterruptedException { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); final Thread workerThread = new Thread(() -> { try { @@ -684,7 +684,7 @@ void testPopulatesOutputOnSuccess() throws WorkerException { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); final ReplicationOutput actual = worker.run(syncInput, jobRoot); final ReplicationOutput replicationOutput = new ReplicationOutput() @@ -752,7 +752,7 @@ void testPopulatesStateOnFailureIfAvailable() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); final ReplicationOutput actual = worker.run(syncInput, jobRoot); assertNotNull(actual); @@ -772,7 +772,7 @@ void testRetainsStateOnFailureIfNewStateNotAvailable() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); final ReplicationOutput actual = worker.run(syncInput, jobRoot); @@ -807,7 +807,7 @@ void testPopulatesStatsOnFailureIfAvailable() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); final ReplicationOutput actual = worker.run(syncInput, jobRoot); final SyncStats expectedTotalStats = new SyncStats() @@ -854,7 +854,7 @@ void testDoesNotPopulatesStateOnFailureIfNotAvailable() throws Exception { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); final ReplicationOutput actual = worker.run(syncInputWithoutState, jobRoot); @@ -875,7 +875,7 @@ void testDoesNotPopulateOnIrrecoverableFailure() { messageTracker, recordSchemaValidator, workerMetricReporter, - updateConnectorConfigHelper, false); + connectorConfigUpdater, false); assertThrows(WorkerException.class, () -> worker.run(syncInput, jobRoot)); } diff --git a/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/UpdateConnectorConfigHelperTest.java b/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/ConnectorConfigUpdaterTest.java similarity index 89% rename from airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/UpdateConnectorConfigHelperTest.java rename to airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/ConnectorConfigUpdaterTest.java index 81db2b47cc10..8711a1af67ab 100644 --- a/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/UpdateConnectorConfigHelperTest.java +++ b/airbyte-commons-worker/src/test/java/io/airbyte/workers/helper/ConnectorConfigUpdaterTest.java @@ -25,7 +25,7 @@ import org.junit.jupiter.api.Test; import org.mockito.Mockito; -class UpdateConnectorConfigHelperTest { +class ConnectorConfigUpdaterTest { private static final UUID SOURCE_ID = UUID.randomUUID(); private static final String SOURCE_NAME = "source-stripe"; @@ -35,7 +35,7 @@ class UpdateConnectorConfigHelperTest { private final SourceApi mSourceApi = mock(SourceApi.class); private final DestinationApi mDestinationApi = mock(DestinationApi.class); - private UpdateConnectorConfigHelper updateConnectorConfigHelper; + private ConnectorConfigUpdater connectorConfigUpdater; @BeforeEach void setUp() throws ApiException { @@ -49,7 +49,7 @@ void setUp() throws ApiException { .destinationId(DESTINATION_ID) .name(DESTINATION_NAME)); - updateConnectorConfigHelper = new UpdateConnectorConfigHelper(mSourceApi, mDestinationApi); + connectorConfigUpdater = new ConnectorConfigUpdater(mSourceApi, mDestinationApi); } @Test @@ -64,7 +64,7 @@ void testPersistSourceConfig() throws ApiException { when(mSourceApi.updateSource(Mockito.any())).thenReturn(new SourceRead().connectionConfiguration(configJson)); - updateConnectorConfigHelper.updateSource(SOURCE_ID, newConfiguration); + connectorConfigUpdater.updateSource(SOURCE_ID, newConfiguration); verify(mSourceApi).updateSource(expectedSourceUpdate); } @@ -80,7 +80,7 @@ void testPersistDestinationConfig() throws ApiException { when(mDestinationApi.updateDestination(Mockito.any())).thenReturn(new DestinationRead().connectionConfiguration(configJson)); - updateConnectorConfigHelper.updateDestination(DESTINATION_ID, newConfiguration); + connectorConfigUpdater.updateDestination(DESTINATION_ID, newConfiguration); verify(mDestinationApi).updateDestination(expectedDestinationUpdate); } diff --git a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java index 32b63e940dbb..281763489a16 100644 --- a/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java +++ b/airbyte-container-orchestrator/src/main/java/io/airbyte/container_orchestrator/orchestrator/ReplicationJobOrchestrator.java @@ -32,7 +32,7 @@ import io.airbyte.workers.WorkerMetricReporter; import io.airbyte.workers.WorkerUtils; import io.airbyte.workers.general.DefaultReplicationWorker; -import io.airbyte.workers.helper.UpdateConnectorConfigHelper; +import io.airbyte.workers.helper.ConnectorConfigUpdater; import io.airbyte.workers.internal.AirbyteStreamFactory; import io.airbyte.workers.internal.DefaultAirbyteDestination; import io.airbyte.workers.internal.DefaultAirbyteSource; @@ -158,7 +158,7 @@ public Optional runJob() throws Exception { new AirbyteMessageTracker(), new RecordSchemaValidator(WorkerUtils.mapStreamNamesToSchemas(syncInput)), metricReporter, - new UpdateConnectorConfigHelper(sourceApi, destinationApi), + new ConnectorConfigUpdater(sourceApi, destinationApi), featureFlags.applyFieldSelection()); log.info("Running replication worker..."); diff --git a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java index f0f48ea527d5..746974e0fc7e 100644 --- a/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java +++ b/airbyte-workers/src/main/java/io/airbyte/workers/temporal/sync/ReplicationActivityImpl.java @@ -52,7 +52,7 @@ import io.airbyte.workers.WorkerMetricReporter; import io.airbyte.workers.WorkerUtils; import io.airbyte.workers.general.DefaultReplicationWorker; -import io.airbyte.workers.helper.UpdateConnectorConfigHelper; +import io.airbyte.workers.helper.ConnectorConfigUpdater; import io.airbyte.workers.internal.AirbyteSource; import io.airbyte.workers.internal.DefaultAirbyteDestination; import io.airbyte.workers.internal.DefaultAirbyteSource; @@ -306,7 +306,7 @@ private CheckedSupplier, Exception> new AirbyteMessageTracker(), new RecordSchemaValidator(WorkerUtils.mapStreamNamesToSchemas(syncInput)), metricReporter, - new UpdateConnectorConfigHelper(airbyteApiClient.getSourceApi(), airbyteApiClient.getDestinationApi()), false); + new ConnectorConfigUpdater(airbyteApiClient.getSourceApi(), airbyteApiClient.getDestinationApi()), false); }; } From b3a443c0f3f4635e2a2f8578cda8f9db70f05929 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 21 Dec 2022 19:37:40 -0400 Subject: [PATCH 43/63] rm log --- .../src/main/java/io/airbyte/api/client/AirbyteApiClient.java | 1 - 1 file changed, 1 deletion(-) diff --git a/airbyte-api/src/main/java/io/airbyte/api/client/AirbyteApiClient.java b/airbyte-api/src/main/java/io/airbyte/api/client/AirbyteApiClient.java index a3afdf592299..ce1fd820d9e5 100644 --- a/airbyte-api/src/main/java/io/airbyte/api/client/AirbyteApiClient.java +++ b/airbyte-api/src/main/java/io/airbyte/api/client/AirbyteApiClient.java @@ -175,7 +175,6 @@ public static T retryWithJitter(final Callable call, keepTrying = false; } catch (final Exception e) { LOGGER.info("Attempt {} to {} error: {}", currRetries, desc, e); - LOGGER.error("Error", e); currRetries++; // Sleep anywhere from 1 to jitterMaxIntervalSecs seconds. From 079c25dfdbca61f07d9b1cff866d2d6bd52523dc Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 21 Dec 2022 19:37:48 -0400 Subject: [PATCH 44/63] fix test --- .../ContainerOrchestratorFactoryTest.java | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/airbyte-container-orchestrator/src/test/java/io/airbyte/container_orchestrator/config/ContainerOrchestratorFactoryTest.java b/airbyte-container-orchestrator/src/test/java/io/airbyte/container_orchestrator/config/ContainerOrchestratorFactoryTest.java index 64b468f85206..1ece23705a9e 100644 --- a/airbyte-container-orchestrator/src/test/java/io/airbyte/container_orchestrator/config/ContainerOrchestratorFactoryTest.java +++ b/airbyte-container-orchestrator/src/test/java/io/airbyte/container_orchestrator/config/ContainerOrchestratorFactoryTest.java @@ -9,7 +9,8 @@ import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; -import io.airbyte.api.client.AirbyteApiClient; +import io.airbyte.api.client.generated.DestinationApi; +import io.airbyte.api.client.generated.SourceApi; import io.airbyte.commons.features.FeatureFlags; import io.airbyte.commons.protocol.AirbyteMessageSerDeProvider; import io.airbyte.commons.protocol.AirbyteMessageVersionedMigratorFactory; @@ -51,7 +52,10 @@ class ContainerOrchestratorFactoryTest { JobRunConfig jobRunConfig; @Inject - AirbyteApiClient airbyteApiClient; + SourceApi sourceApi; + + @Inject + DestinationApi destinationApi; // Tests will fail if this is uncommented, due to how the implementation of the DocumentStoreClient // is being created @@ -91,29 +95,29 @@ void jobOrchestrator() { final var repl = factory.jobOrchestrator( ReplicationLauncherWorker.REPLICATION, envConfigs, processFactory, featureFlags, workerConfigs, - airbyteMessageSerDeProvider, airbyteMessageVersionedMigratorFactory, jobRunConfig, airbyteApiClient); + airbyteMessageSerDeProvider, airbyteMessageVersionedMigratorFactory, jobRunConfig, sourceApi, destinationApi); assertEquals("Replication", repl.getOrchestratorName()); final var norm = factory.jobOrchestrator( NormalizationLauncherWorker.NORMALIZATION, envConfigs, processFactory, featureFlags, workerConfigs, - airbyteMessageSerDeProvider, airbyteMessageVersionedMigratorFactory, jobRunConfig, airbyteApiClient); + airbyteMessageSerDeProvider, airbyteMessageVersionedMigratorFactory, jobRunConfig, sourceApi, destinationApi); assertEquals("Normalization", norm.getOrchestratorName()); final var dbt = factory.jobOrchestrator( DbtLauncherWorker.DBT, envConfigs, processFactory, featureFlags, workerConfigs, - airbyteMessageSerDeProvider, airbyteMessageVersionedMigratorFactory, jobRunConfig, airbyteApiClient); + airbyteMessageSerDeProvider, airbyteMessageVersionedMigratorFactory, jobRunConfig, sourceApi, destinationApi); assertEquals("DBT Transformation", dbt.getOrchestratorName()); final var noop = factory.jobOrchestrator( AsyncOrchestratorPodProcess.NO_OP, envConfigs, processFactory, featureFlags, workerConfigs, - airbyteMessageSerDeProvider, airbyteMessageVersionedMigratorFactory, jobRunConfig, airbyteApiClient); + airbyteMessageSerDeProvider, airbyteMessageVersionedMigratorFactory, jobRunConfig, sourceApi, destinationApi); assertEquals("NO_OP", noop.getOrchestratorName()); var caught = false; try { factory.jobOrchestrator( "does not exist", envConfigs, processFactory, featureFlags, workerConfigs, - airbyteMessageSerDeProvider, airbyteMessageVersionedMigratorFactory, jobRunConfig, airbyteApiClient); + airbyteMessageSerDeProvider, airbyteMessageVersionedMigratorFactory, jobRunConfig, sourceApi, destinationApi); } catch (final Exception e) { caught = true; } From e7f4736e343d0ccc2fc1b8716eda30588a462fa8 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Mon, 26 Dec 2022 13:18:12 -0400 Subject: [PATCH 45/63] dont fail on config update error --- .../general/DefaultReplicationWorker.java | 16 ++++++++++++---- .../general/DefaultReplicationWorkerTest.java | 12 ++++-------- 2 files changed, 16 insertions(+), 12 deletions(-) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java index fa3bdba52f41..a0e10b94c72d 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java @@ -281,8 +281,12 @@ private static Runnable readFromDstRunnable(final AirbyteDestination destination messageTracker.acceptFromDestination(message); - if (message.getType() == Type.CONTROL) { - acceptDstControlMessage(destinationConfig, message.getControl(), connectorConfigUpdater); + try { + if (message.getType() == Type.CONTROL) { + acceptDstControlMessage(destinationConfig, message.getControl(), connectorConfigUpdater); + } + } catch (final Exception e) { + LOGGER.error("Error updating destination configuration", e); } } } @@ -350,8 +354,12 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou messageTracker.acceptFromSource(message); - if (message.getType() == Type.CONTROL) { - acceptSrcControlMessage(sourceConfig, message.getControl(), connectorConfigUpdater); + try { + if (message.getType() == Type.CONTROL) { + acceptSrcControlMessage(sourceConfig, message.getControl(), connectorConfigUpdater); + } + } catch (final Exception e) { + LOGGER.error("Error updating source configuration", e); } try { diff --git a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java index fb98f8c541f5..321af214fe80 100644 --- a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java +++ b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/DefaultReplicationWorkerTest.java @@ -275,7 +275,7 @@ void testReplicationRunnableSourceUpdateConfig() throws Exception { } @Test - void testReplicationFailureOnSourceConfigPersistError() throws Exception { + void testSourceConfigPersistError() throws Exception { when(source.attemptRead()).thenReturn(Optional.of(CONFIG_MESSAGE)); when(source.isFinished()).thenReturn(false, true); @@ -294,9 +294,7 @@ void testReplicationFailureOnSourceConfigPersistError() throws Exception { connectorConfigUpdater, false); final ReplicationOutput output = worker.run(syncInput, jobRoot); - assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); - assertTrue(output.getFailures().stream() - .anyMatch(f -> f.getFailureOrigin().equals(FailureOrigin.REPLICATION) && f.getStacktrace().contains(PERSIST_ERROR_MESSAGE))); + assertEquals(ReplicationStatus.COMPLETED, output.getReplicationAttemptSummary().getStatus()); verify(connectorConfigUpdater).updateSource(syncInput.getSourceId(), CONNECTOR_CONFIG); } @@ -324,7 +322,7 @@ void testReplicationRunnableDestinationUpdateConfig() throws Exception { } @Test - void testReplicationFailureOnDestinationConfigPersistError() throws Exception { + void testDestinationConfigPersistError() throws Exception { when(destination.attemptRead()).thenReturn(Optional.of(CONFIG_MESSAGE)); when(destination.isFinished()).thenReturn(false, true); @@ -343,9 +341,7 @@ void testReplicationFailureOnDestinationConfigPersistError() throws Exception { connectorConfigUpdater, false); final ReplicationOutput output = worker.run(syncInput, jobRoot); - assertEquals(ReplicationStatus.FAILED, output.getReplicationAttemptSummary().getStatus()); - assertTrue(output.getFailures().stream() - .anyMatch(f -> f.getFailureOrigin().equals(FailureOrigin.REPLICATION) && f.getStacktrace().contains(PERSIST_ERROR_MESSAGE))); + assertEquals(ReplicationStatus.COMPLETED, output.getReplicationAttemptSummary().getStatus()); verify(connectorConfigUpdater).updateDestination(syncInput.getDestinationId(), CONNECTOR_CONFIG); } From c4ff5dcb3f82ab1d9d995f4b52066279a352d2d0 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Tue, 27 Dec 2022 12:12:51 -0400 Subject: [PATCH 46/63] process control messages for discover jobs --- .../java/io/airbyte/workers/WorkerUtils.java | 10 ++++++++++ .../general/DefaultDiscoverCatalogWorker.java | 16 ++++++++++++++-- .../source/AbstractSourceConnectorTest.java | 6 +++++- .../catalog/DiscoverCatalogActivityImpl.java | 5 ++++- .../DefaultDiscoverCatalogWorkerTest.java | 18 +++++++++++++----- 5 files changed, 46 insertions(+), 9 deletions(-) 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 216371c248b7..db89cf05b55e 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,6 +11,8 @@ import io.airbyte.config.StandardSyncInput; import io.airbyte.config.WorkerDestinationConfig; import io.airbyte.config.WorkerSourceConfig; +import io.airbyte.protocol.models.AirbyteControlConnectorConfigMessage; +import io.airbyte.protocol.models.AirbyteControlMessage; import io.airbyte.protocol.models.AirbyteMessage; import io.airbyte.protocol.models.AirbyteMessage.Type; import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair; @@ -109,6 +111,14 @@ public static WorkerDestinationConfig syncToWorkerDestinationConfig(final Standa .withState(sync.getState()); } + public static Optional getConfigControlMessage(final Map> messagesByType) { + return messagesByType.getOrDefault(Type.CONTROL, new ArrayList<>()).stream() + .map(AirbyteMessage::getControl) + .filter(control -> control.getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG) + .map(AirbyteControlMessage::getConnectorConfig) + .findFirst(); + } + public static ConnectorJobOutput getJobFailureOutputOrThrow(final OutputType outputType, final Map> messagesByType, final String defaultErrorMessage) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorker.java index 8c5a961f1fdd..ac119b2664a6 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorker.java @@ -19,11 +19,13 @@ import io.airbyte.config.persistence.ConfigRepository; import io.airbyte.metrics.lib.ApmTraceUtils; import io.airbyte.protocol.models.AirbyteCatalog; +import io.airbyte.protocol.models.AirbyteControlConnectorConfigMessage; import io.airbyte.protocol.models.AirbyteMessage; import io.airbyte.protocol.models.AirbyteMessage.Type; import io.airbyte.workers.WorkerConstants; import io.airbyte.workers.WorkerUtils; import io.airbyte.workers.exception.WorkerException; +import io.airbyte.workers.helper.ConnectorConfigUpdater; import io.airbyte.workers.internal.AirbyteStreamFactory; import io.airbyte.workers.internal.DefaultAirbyteStreamFactory; import io.airbyte.workers.process.IntegrationLauncher; @@ -48,20 +50,24 @@ public class DefaultDiscoverCatalogWorker implements DiscoverCatalogWorker { private final IntegrationLauncher integrationLauncher; private final AirbyteStreamFactory streamFactory; + private final ConnectorConfigUpdater connectorConfigUpdater; private volatile Process process; public DefaultDiscoverCatalogWorker(final ConfigRepository configRepository, final IntegrationLauncher integrationLauncher, + final ConnectorConfigUpdater connectorConfigUpdater, final AirbyteStreamFactory streamFactory) { this.configRepository = configRepository; this.integrationLauncher = integrationLauncher; this.streamFactory = streamFactory; + this.connectorConfigUpdater = connectorConfigUpdater; } public DefaultDiscoverCatalogWorker(final ConfigRepository configRepository, - final IntegrationLauncher integrationLauncher) { - this(configRepository, integrationLauncher, new DefaultAirbyteStreamFactory()); + final IntegrationLauncher integrationLauncher, + final ConnectorConfigUpdater connectorConfigUpdater) { + this(configRepository, integrationLauncher, connectorConfigUpdater, new DefaultAirbyteStreamFactory()); } @Trace(operationName = WORKER_OPERATION_NAME) @@ -90,6 +96,12 @@ public ConnectorJobOutput run(final StandardDiscoverCatalogInput discoverSchemaI .map(AirbyteMessage::getCatalog) .findFirst(); + final Optional optionalConfigMsg = WorkerUtils.getConfigControlMessage(messagesByType); + optionalConfigMsg.ifPresent( + configMessage -> connectorConfigUpdater.updateSource( + UUID.fromString(discoverSchemaInput.getSourceId()), + configMessage.getConfig())); + final int exitCode = process.exitValue(); if (exitCode == 0) { if (catalog.isEmpty()) { diff --git a/airbyte-integrations/bases/standard-source-test/src/main/java/io/airbyte/integrations/standardtest/source/AbstractSourceConnectorTest.java b/airbyte-integrations/bases/standard-source-test/src/main/java/io/airbyte/integrations/standardtest/source/AbstractSourceConnectorTest.java index 9e811d49c5ba..0e05028eac71 100644 --- a/airbyte-integrations/bases/standard-source-test/src/main/java/io/airbyte/integrations/standardtest/source/AbstractSourceConnectorTest.java +++ b/airbyte-integrations/bases/standard-source-test/src/main/java/io/airbyte/integrations/standardtest/source/AbstractSourceConnectorTest.java @@ -32,6 +32,7 @@ import io.airbyte.workers.general.DefaultCheckConnectionWorker; import io.airbyte.workers.general.DefaultDiscoverCatalogWorker; import io.airbyte.workers.general.DefaultGetSpecWorker; +import io.airbyte.workers.helper.ConnectorConfigUpdater; import io.airbyte.workers.helper.EntrypointEnvChecker; import io.airbyte.workers.internal.AirbyteSource; import io.airbyte.workers.internal.DefaultAirbyteSource; @@ -111,6 +112,7 @@ public abstract class AbstractSourceConnectorTest { private WorkerConfigs workerConfigs; private ConfigRepository mConfigRepository; + private ConnectorConfigUpdater mConnectorConfigUpdater; // This has to be using the protocol version of the platform in order to capture the arg private final ArgumentCaptor lastPersistedCatalog = @@ -131,6 +133,7 @@ public void setUpInternal() throws Exception { setupEnvironment(environment); workerConfigs = new WorkerConfigs(new EnvConfigs()); mConfigRepository = mock(ConfigRepository.class); + mConnectorConfigUpdater = mock(ConnectorConfigUpdater.class); processFactory = new DockerProcessFactory( workerConfigs, workspaceRoot, @@ -174,7 +177,8 @@ protected String runCheckAndGetStatusAsString(final JsonNode config) throws Exce protected UUID runDiscover() throws Exception { final UUID toReturn = new DefaultDiscoverCatalogWorker( mConfigRepository, - new AirbyteIntegrationLauncher(JOB_ID, JOB_ATTEMPT, getImageName(), processFactory, workerConfigs.getResourceRequirements(), false)) + new AirbyteIntegrationLauncher(JOB_ID, JOB_ATTEMPT, getImageName(), processFactory, workerConfigs.getResourceRequirements(), false), + mConnectorConfigUpdater) .run(new StandardDiscoverCatalogInput().withSourceId(SOURCE_ID.toString()).withConnectionConfiguration(getConfig()), jobRoot) .getDiscoverCatalogId(); verify(mConfigRepository).writeActorCatalogFetchEvent(lastPersistedCatalog.capture(), any(), any(), any()); 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 5a36c144fc43..6796e32ec139 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 @@ -29,6 +29,7 @@ import io.airbyte.workers.Worker; import io.airbyte.workers.WorkerConfigs; import io.airbyte.workers.general.DefaultDiscoverCatalogWorker; +import io.airbyte.workers.helper.ConnectorConfigUpdater; import io.airbyte.workers.internal.AirbyteStreamFactory; import io.airbyte.workers.internal.VersionedAirbyteStreamFactory; import io.airbyte.workers.process.AirbyteIntegrationLauncher; @@ -127,7 +128,9 @@ private CheckedSupplier processFactory, workerConfigs.getResourceRequirements(), launcherConfig.getIsCustomConnector()); final AirbyteStreamFactory streamFactory = new VersionedAirbyteStreamFactory<>(serDeProvider, migratorFactory, launcherConfig.getProtocolVersion()); - return new DefaultDiscoverCatalogWorker(configRepository, integrationLauncher, streamFactory); + final ConnectorConfigUpdater connectorConfigUpdater = + new ConnectorConfigUpdater(airbyteApiClient.getSourceApi(), airbyteApiClient.getDestinationApi()); + return new DefaultDiscoverCatalogWorker(configRepository, integrationLauncher, connectorConfigUpdater, streamFactory); }; } diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorkerTest.java b/airbyte-workers/src/test/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorkerTest.java index b9fe4417657d..cadd93c1b886 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorkerTest.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorkerTest.java @@ -38,6 +38,7 @@ import io.airbyte.protocol.models.JsonSchemaType; import io.airbyte.workers.WorkerConstants; import io.airbyte.workers.exception.WorkerException; +import io.airbyte.workers.helper.ConnectorConfigUpdater; import io.airbyte.workers.internal.AirbyteStreamFactory; import io.airbyte.workers.process.IntegrationLauncher; import io.airbyte.workers.test_utils.AirbyteMessageUtils; @@ -77,6 +78,7 @@ class DefaultDiscoverCatalogWorkerTest { private IntegrationLauncher integrationLauncher; private Process process; private AirbyteStreamFactory streamFactory; + private ConnectorConfigUpdater connectorConfigUpdater; private UUID CATALOG_ID; @@ -86,6 +88,7 @@ void setup() throws Exception { integrationLauncher = mock(IntegrationLauncher.class, RETURNS_DEEP_STUBS); process = mock(Process.class); mConfigRepository = mock(ConfigRepository.class); + connectorConfigUpdater = mock(ConnectorConfigUpdater.class); CATALOG_ID = UUID.randomUUID(); when(mConfigRepository.writeActorCatalogFetchEvent(any(), any(), any(), any())).thenReturn(CATALOG_ID); @@ -103,7 +106,8 @@ void setup() throws Exception { @SuppressWarnings("BusyWait") @Test void testDiscoverSchema() throws Exception { - final DefaultDiscoverCatalogWorker worker = new DefaultDiscoverCatalogWorker(mConfigRepository, integrationLauncher, streamFactory); + final DefaultDiscoverCatalogWorker worker = + new DefaultDiscoverCatalogWorker(mConfigRepository, integrationLauncher, connectorConfigUpdater, streamFactory); final ConnectorJobOutput output = worker.run(INPUT, jobRoot); assertNull(output.getFailureReason()); @@ -125,7 +129,8 @@ void testDiscoverSchema() throws Exception { void testDiscoverSchemaProcessFail() throws Exception { when(process.exitValue()).thenReturn(1); - final DefaultDiscoverCatalogWorker worker = new DefaultDiscoverCatalogWorker(mConfigRepository, integrationLauncher, streamFactory); + final DefaultDiscoverCatalogWorker worker = + new DefaultDiscoverCatalogWorker(mConfigRepository, integrationLauncher, connectorConfigUpdater, streamFactory); assertThrows(WorkerException.class, () -> worker.run(INPUT, jobRoot)); Assertions.assertTimeout(Duration.ofSeconds(5), () -> { @@ -145,7 +150,8 @@ void testDiscoverSchemaProcessFailWithTraceMessage() throws Exception { when(process.exitValue()).thenReturn(1); - final DefaultDiscoverCatalogWorker worker = new DefaultDiscoverCatalogWorker(mConfigRepository, integrationLauncher, traceStreamFactory); + final DefaultDiscoverCatalogWorker worker = + new DefaultDiscoverCatalogWorker(mConfigRepository, integrationLauncher, connectorConfigUpdater, traceStreamFactory); final ConnectorJobOutput output = worker.run(INPUT, jobRoot); // assertEquals(OutputType.DISCOVER_CATALOG, output.getOutputType()); // assertNull(output.getDiscoverCatalog()); @@ -168,13 +174,15 @@ void testDiscoverSchemaException() throws WorkerException { when(integrationLauncher.discover(jobRoot, WorkerConstants.SOURCE_CONFIG_JSON_FILENAME, Jsons.serialize(CREDENTIALS))) .thenThrow(new RuntimeException()); - final DefaultDiscoverCatalogWorker worker = new DefaultDiscoverCatalogWorker(mConfigRepository, integrationLauncher, streamFactory); + final DefaultDiscoverCatalogWorker worker = + new DefaultDiscoverCatalogWorker(mConfigRepository, integrationLauncher, connectorConfigUpdater, streamFactory); assertThrows(WorkerException.class, () -> worker.run(INPUT, jobRoot)); } @Test void testCancel() throws WorkerException { - final DefaultDiscoverCatalogWorker worker = new DefaultDiscoverCatalogWorker(mConfigRepository, integrationLauncher, streamFactory); + final DefaultDiscoverCatalogWorker worker = + new DefaultDiscoverCatalogWorker(mConfigRepository, integrationLauncher, connectorConfigUpdater, streamFactory); worker.run(INPUT, jobRoot); worker.cancel(); From a7f036523cbb2ff6ee8e2412fcdd489209a1e0ba Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Tue, 27 Dec 2022 15:22:26 -0400 Subject: [PATCH 47/63] process control messages for CHECK --- .../commons/temporal/TemporalClient.java | 5 +++- .../general/DefaultCheckConnectionWorker.java | 29 +++++++++++++++++-- .../types/JobCheckConnectionConfig.yaml | 8 +++++ .../types/StandardCheckConnectionInput.yaml | 8 +++++ .../DestinationAcceptanceTest.java | 25 ++++++++++------ .../source/AbstractSourceConnectorTest.java | 6 ++-- .../DefaultSynchronousSchedulerClient.java | 5 ++++ .../CheckConnectionActivityImpl.java | 13 +++++++-- .../ConnectionManagerWorkflowImpl.java | 16 +++++++--- .../DefaultCheckConnectionWorkerTest.java | 23 ++++++++++----- 10 files changed, 111 insertions(+), 27 deletions(-) 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 3c8ec42e4985..0371536db32d 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 @@ -349,7 +349,10 @@ public TemporalResponse submitCheckConnection(final UUID job .withDockerImage(config.getDockerImage()) .withProtocolVersion(config.getProtocolVersion()) .withIsCustomConnector(config.getIsCustomConnector()); - final StandardCheckConnectionInput input = new StandardCheckConnectionInput().withConnectionConfiguration(config.getConnectionConfiguration()); + final StandardCheckConnectionInput input = new StandardCheckConnectionInput() + .withActorType(config.getActorType()) + .withActorId(config.getActorId()) + .withConnectionConfiguration(config.getConnectionConfiguration()); return execute(jobRunConfig, () -> getWorkflowStub(CheckConnectionWorkflow.class, TemporalJobType.CHECK_CONNECTION).run(jobRunConfig, launcherConfig, input)); diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultCheckConnectionWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultCheckConnectionWorker.java index 66bc2d271b24..5d2e9cc403cd 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultCheckConnectionWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultCheckConnectionWorker.java @@ -19,11 +19,13 @@ import io.airbyte.config.StandardCheckConnectionOutput.Status; import io.airbyte.metrics.lib.ApmTraceUtils; import io.airbyte.protocol.models.AirbyteConnectionStatus; +import io.airbyte.protocol.models.AirbyteControlConnectorConfigMessage; import io.airbyte.protocol.models.AirbyteMessage; import io.airbyte.protocol.models.AirbyteMessage.Type; import io.airbyte.workers.WorkerConstants; import io.airbyte.workers.WorkerUtils; import io.airbyte.workers.exception.WorkerException; +import io.airbyte.workers.helper.ConnectorConfigUpdater; import io.airbyte.workers.internal.AirbyteStreamFactory; import io.airbyte.workers.internal.DefaultAirbyteStreamFactory; import io.airbyte.workers.process.IntegrationLauncher; @@ -43,18 +45,21 @@ public class DefaultCheckConnectionWorker implements CheckConnectionWorker { private static final Logger LOGGER = LoggerFactory.getLogger(DefaultCheckConnectionWorker.class); private final IntegrationLauncher integrationLauncher; + private final ConnectorConfigUpdater connectorConfigUpdater; private final AirbyteStreamFactory streamFactory; private Process process; public DefaultCheckConnectionWorker(final IntegrationLauncher integrationLauncher, + final ConnectorConfigUpdater connectorConfigUpdater, final AirbyteStreamFactory streamFactory) { this.integrationLauncher = integrationLauncher; + this.connectorConfigUpdater = connectorConfigUpdater; this.streamFactory = streamFactory; } - public DefaultCheckConnectionWorker(final IntegrationLauncher integrationLauncher) { - this(integrationLauncher, new DefaultAirbyteStreamFactory()); + public DefaultCheckConnectionWorker(final IntegrationLauncher integrationLauncher, final ConnectorConfigUpdater connectorConfigUpdater) { + this(integrationLauncher, connectorConfigUpdater, new DefaultAirbyteStreamFactory()); } @Trace(operationName = WORKER_OPERATION_NAME) @@ -84,6 +89,26 @@ public ConnectorJobOutput run(final StandardCheckConnectionInput input, final Pa .map(AirbyteMessage::getConnectionStatus) .findFirst(); + if (input.getActorId() != null && input.getActorType() != null) { + LOGGER.info("has actor and type"); + final Optional optionalConfigMsg = WorkerUtils.getConfigControlMessage(messagesByType); + LOGGER.info("opt config msg {}", optionalConfigMsg.orElse(null)); + optionalConfigMsg.ifPresent( + configMessage -> { + LOGGER.info("the thing was present"); + switch (input.getActorType()) { + case SOURCE -> connectorConfigUpdater.updateSource( + input.getActorId(), + configMessage.getConfig()); + case DESTINATION -> connectorConfigUpdater.updateDestination( + input.getActorId(), + configMessage.getConfig()); + } + }); + } else { + LOGGER.info("no ids"); + } + if (status.isPresent() && exitCode == 0) { final StandardCheckConnectionOutput output = new StandardCheckConnectionOutput() .withStatus(Enums.convertTo(status.get().getStatus(), Status.class)) diff --git a/airbyte-config/config-models/src/main/resources/types/JobCheckConnectionConfig.yaml b/airbyte-config/config-models/src/main/resources/types/JobCheckConnectionConfig.yaml index 3ec7819bfca0..d2706dfe9532 100644 --- a/airbyte-config/config-models/src/main/resources/types/JobCheckConnectionConfig.yaml +++ b/airbyte-config/config-models/src/main/resources/types/JobCheckConnectionConfig.yaml @@ -6,9 +6,17 @@ description: job check connection config type: object additionalProperties: false required: + - actorType + - actorId - connectionConfiguration - dockerImage properties: + actorType: + "$ref": ActorType.yaml + actorId: + description: The ID of the actor being checked, so we can persist config updates + type: string + format: uuid connectionConfiguration: description: Integration specific blob. Must be a valid JSON string. type: object diff --git a/airbyte-config/config-models/src/main/resources/types/StandardCheckConnectionInput.yaml b/airbyte-config/config-models/src/main/resources/types/StandardCheckConnectionInput.yaml index 0fef700bccc4..8579a5338b75 100644 --- a/airbyte-config/config-models/src/main/resources/types/StandardCheckConnectionInput.yaml +++ b/airbyte-config/config-models/src/main/resources/types/StandardCheckConnectionInput.yaml @@ -6,8 +6,16 @@ description: information required for connection. type: object required: - connectionConfiguration + - actorType + - actorId additionalProperties: false properties: + actorType: + "$ref": ActorType.yaml + actorId: + description: The ID of the actor being checked, so we can persist config updates + type: string + format: uuid connectionConfiguration: description: Integration specific blob. Must be a valid JSON string. type: object diff --git a/airbyte-integrations/bases/standard-destination-test/src/main/java/io/airbyte/integrations/standardtest/destination/DestinationAcceptanceTest.java b/airbyte-integrations/bases/standard-destination-test/src/main/java/io/airbyte/integrations/standardtest/destination/DestinationAcceptanceTest.java index 23703a3d47dd..d569d7d178af 100644 --- a/airbyte-integrations/bases/standard-destination-test/src/main/java/io/airbyte/integrations/standardtest/destination/DestinationAcceptanceTest.java +++ b/airbyte-integrations/bases/standard-destination-test/src/main/java/io/airbyte/integrations/standardtest/destination/DestinationAcceptanceTest.java @@ -56,6 +56,7 @@ import io.airbyte.workers.general.DbtTransformationRunner; import io.airbyte.workers.general.DefaultCheckConnectionWorker; import io.airbyte.workers.general.DefaultGetSpecWorker; +import io.airbyte.workers.helper.ConnectorConfigUpdater; import io.airbyte.workers.helper.EntrypointEnvChecker; import io.airbyte.workers.internal.AirbyteDestination; import io.airbyte.workers.internal.DefaultAirbyteDestination; @@ -93,6 +94,7 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.ArgumentsProvider; import org.junit.jupiter.params.provider.ArgumentsSource; +import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -113,6 +115,7 @@ public abstract class DestinationAcceptanceTest { private Path jobRoot; private ProcessFactory processFactory; private WorkerConfigs workerConfigs; + private ConnectorConfigUpdater mConnectorConfigUpdater; protected Path localRoot; protected TestDataComparator testDataComparator = getTestDataComparator(); @@ -130,11 +133,11 @@ private String getImageNameWithoutTag() { private Optional getOptionalDestinationDefinitionFromProvider(final String imageNameWithoutTag) { try { - LocalDefinitionsProvider provider = new LocalDefinitionsProvider(LocalDefinitionsProvider.DEFAULT_SEED_DEFINITION_RESOURCE_CLASS); + final LocalDefinitionsProvider provider = new LocalDefinitionsProvider(LocalDefinitionsProvider.DEFAULT_SEED_DEFINITION_RESOURCE_CLASS); return provider.getDestinationDefinitions().stream() .filter(definition -> imageNameWithoutTag.equalsIgnoreCase(definition.getDockerRepository())) .findFirst(); - } catch (IOException e) { + } catch (final IOException e) { return Optional.empty(); } } @@ -361,6 +364,7 @@ void setUpInternal() throws Exception { LOGGER.info("localRoot: {}", localRoot); testEnv = new TestDestinationEnv(localRoot); workerConfigs = new WorkerConfigs(new EnvConfigs()); + mConnectorConfigUpdater = Mockito.mock(ConnectorConfigUpdater.class); setup(testEnv); @@ -1206,7 +1210,8 @@ private ConnectorSpecification runSpec() throws WorkerException { protected StandardCheckConnectionOutput runCheck(final JsonNode config) throws WorkerException { return new DefaultCheckConnectionWorker( - new AirbyteIntegrationLauncher(JOB_ID, JOB_ATTEMPT, getImageName(), processFactory, null, false)) + new AirbyteIntegrationLauncher(JOB_ID, JOB_ATTEMPT, getImageName(), processFactory, null, false), + mConnectorConfigUpdater) .run(new StandardCheckConnectionInput().withConnectionConfiguration(config), jobRoot) .getCheckConnection(); } @@ -1215,7 +1220,8 @@ protected StandardCheckConnectionOutput.Status runCheckWithCatchedException( final JsonNode config) { try { final StandardCheckConnectionOutput standardCheckConnectionOutput = new DefaultCheckConnectionWorker( - new AirbyteIntegrationLauncher(JOB_ID, JOB_ATTEMPT, getImageName(), processFactory, null, false)) + new AirbyteIntegrationLauncher(JOB_ID, JOB_ATTEMPT, getImageName(), processFactory, null, false), + mConnectorConfigUpdater) .run(new StandardCheckConnectionInput().withConnectionConfiguration(config), jobRoot) .getCheckConnection(); return standardCheckConnectionOutput.getStatus(); @@ -1644,7 +1650,7 @@ public void testDataTypeTestWithNormalization(final String messagesFilename, @Test public void testSyncNumberNanDataType() throws Exception { // NaN/Infinity protocol supports started from V1 version or higher - SpecialNumericTypes numericTypesSupport = getSpecialNumericTypesSupportTest(); + final SpecialNumericTypes numericTypesSupport = getSpecialNumericTypesSupportTest(); if (getProtocolVersion().equals(ProtocolVersion.V0) || !numericTypesSupport.isSupportNumberNan()) { return; } @@ -1660,7 +1666,7 @@ public void testSyncNumberNanDataType() throws Exception { @Test public void testSyncIntegerNanDataType() throws Exception { // NaN/Infinity protocol supports started from V1 version or higher - SpecialNumericTypes numericTypesSupport = getSpecialNumericTypesSupportTest(); + final SpecialNumericTypes numericTypesSupport = getSpecialNumericTypesSupportTest(); if (getProtocolVersion().equals(ProtocolVersion.V0) || !numericTypesSupport.isSupportIntegerNan()) { return; } @@ -1676,7 +1682,7 @@ public void testSyncIntegerNanDataType() throws Exception { @Test public void testSyncNumberInfinityDataType() throws Exception { // NaN/Infinity protocol supports started from V1 version or higher - SpecialNumericTypes numericTypesSupport = getSpecialNumericTypesSupportTest(); + final SpecialNumericTypes numericTypesSupport = getSpecialNumericTypesSupportTest(); if (getProtocolVersion().equals(ProtocolVersion.V0) || !numericTypesSupport.isSupportNumberInfinity()) { return; } @@ -1692,7 +1698,7 @@ public void testSyncNumberInfinityDataType() throws Exception { @Test public void testSyncIntegerInfinityDataType() throws Exception { // NaN/Infinity protocol supports started from V1 version or higher - SpecialNumericTypes numericTypesSupport = getSpecialNumericTypesSupportTest(); + final SpecialNumericTypes numericTypesSupport = getSpecialNumericTypesSupportTest(); if (getProtocolVersion().equals(ProtocolVersion.V0) || !numericTypesSupport.isSupportIntegerInfinity()) { return; } @@ -1705,7 +1711,8 @@ public void testSyncIntegerInfinityDataType() throws Exception { runAndCheck(catalog, configuredCatalog, messages); } - private void runAndCheck(AirbyteCatalog catalog, ConfiguredAirbyteCatalog configuredCatalog, List messages) throws Exception { + private void runAndCheck(final AirbyteCatalog catalog, final ConfiguredAirbyteCatalog configuredCatalog, final List messages) + throws Exception { if (supportsNormalization()) { LOGGER.info("Normalization is supported! Run test with normalization."); runAndCheckWithNormalization(messages, configuredCatalog, catalog); diff --git a/airbyte-integrations/bases/standard-source-test/src/main/java/io/airbyte/integrations/standardtest/source/AbstractSourceConnectorTest.java b/airbyte-integrations/bases/standard-source-test/src/main/java/io/airbyte/integrations/standardtest/source/AbstractSourceConnectorTest.java index 0e05028eac71..e95f2ec7dc2b 100644 --- a/airbyte-integrations/bases/standard-source-test/src/main/java/io/airbyte/integrations/standardtest/source/AbstractSourceConnectorTest.java +++ b/airbyte-integrations/bases/standard-source-test/src/main/java/io/airbyte/integrations/standardtest/source/AbstractSourceConnectorTest.java @@ -164,13 +164,15 @@ protected ConnectorSpecification runSpec() throws WorkerException { protected StandardCheckConnectionOutput runCheck() throws Exception { return new DefaultCheckConnectionWorker( - new AirbyteIntegrationLauncher(JOB_ID, JOB_ATTEMPT, getImageName(), processFactory, workerConfigs.getResourceRequirements(), false)) + new AirbyteIntegrationLauncher(JOB_ID, JOB_ATTEMPT, getImageName(), processFactory, workerConfigs.getResourceRequirements(), false), + mConnectorConfigUpdater) .run(new StandardCheckConnectionInput().withConnectionConfiguration(getConfig()), jobRoot).getCheckConnection(); } protected String runCheckAndGetStatusAsString(final JsonNode config) throws Exception { return new DefaultCheckConnectionWorker( - new AirbyteIntegrationLauncher(JOB_ID, JOB_ATTEMPT, getImageName(), processFactory, workerConfigs.getResourceRequirements(), false)) + new AirbyteIntegrationLauncher(JOB_ID, JOB_ATTEMPT, getImageName(), processFactory, workerConfigs.getResourceRequirements(), false), + mConnectorConfigUpdater) .run(new StandardCheckConnectionInput().withConnectionConfiguration(config), jobRoot).getCheckConnection().getStatus().toString(); } 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 6ab0bacac3f3..9bb36a9ea1fa 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 @@ -14,6 +14,7 @@ import io.airbyte.commons.temporal.TemporalClient; import io.airbyte.commons.temporal.TemporalResponse; import io.airbyte.commons.version.Version; +import io.airbyte.config.ActorType; import io.airbyte.config.ConnectorJobOutput; import io.airbyte.config.DestinationConnection; import io.airbyte.config.JobCheckConnectionConfig; @@ -70,6 +71,8 @@ public SynchronousResponse createSourceCheckConne source.getWorkspaceId(), source.getConfiguration()); final JobCheckConnectionConfig jobCheckConnectionConfig = new JobCheckConnectionConfig() + .withActorType(ActorType.SOURCE) + .withActorId(source.getSourceId()) .withConnectionConfiguration(sourceConfiguration) .withDockerImage(dockerImage) .withProtocolVersion(protocolVersion) @@ -98,6 +101,8 @@ public SynchronousResponse createDestinationCheck destination.getWorkspaceId(), destination.getConfiguration()); final JobCheckConnectionConfig jobCheckConnectionConfig = new JobCheckConnectionConfig() + .withActorType(ActorType.DESTINATION) + .withActorId(destination.getDestinationId()) .withConnectionConfiguration(destinationConfiguration) .withDockerImage(dockerImage) .withProtocolVersion(protocolVersion) 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 ba21f2af375a..8447ec40c775 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 @@ -29,6 +29,7 @@ import io.airbyte.workers.Worker; import io.airbyte.workers.WorkerConfigs; import io.airbyte.workers.general.DefaultCheckConnectionWorker; +import io.airbyte.workers.helper.ConnectorConfigUpdater; import io.airbyte.workers.internal.AirbyteStreamFactory; import io.airbyte.workers.internal.DefaultAirbyteStreamFactory; import io.airbyte.workers.internal.VersionedAirbyteStreamFactory; @@ -88,9 +89,12 @@ public ConnectorJobOutput runWithJobOutput(final CheckConnectionInput args) { ApmTraceUtils .addTagsToTrace(Map.of(ATTEMPT_NUMBER_KEY, args.getJobRunConfig().getAttemptId(), JOB_ID_KEY, args.getJobRunConfig().getJobId(), DOCKER_IMAGE_KEY, args.getLauncherConfig().getDockerImage())); - final JsonNode fullConfig = secretsHydrator.hydrate(args.getConnectionConfiguration().getConnectionConfiguration()); + final StandardCheckConnectionInput rawInput = args.getConnectionConfiguration(); + final JsonNode fullConfig = secretsHydrator.hydrate(rawInput.getConnectionConfiguration()); final StandardCheckConnectionInput input = new StandardCheckConnectionInput() + .withActorId(rawInput.getActorId()) + .withActorType(rawInput.getActorType()) .withConnectionConfiguration(fullConfig); final ActivityExecutionContext context = Activity.getExecutionContext(); @@ -132,11 +136,16 @@ private CheckedSupplier processFactory, workerConfigs.getResourceRequirements(), launcherConfig.getIsCustomConnector()); + + final ConnectorConfigUpdater connectorConfigUpdater = new ConnectorConfigUpdater( + airbyteApiClient.getSourceApi(), + airbyteApiClient.getDestinationApi()); + final AirbyteStreamFactory streamFactory = launcherConfig.getProtocolVersion() != null ? new VersionedAirbyteStreamFactory<>(serDeProvider, migratorFactory, launcherConfig.getProtocolVersion()) : new DefaultAirbyteStreamFactory(); - return new DefaultCheckConnectionWorker(integrationLauncher, streamFactory); + return new DefaultCheckConnectionWorker(integrationLauncher, connectorConfigUpdater, streamFactory); }; } 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 d05db6d4fa73..0adc24f5810f 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 @@ -20,6 +20,7 @@ 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; +import io.airbyte.config.ActorType; import io.airbyte.config.ConnectorJobOutput; import io.airbyte.config.ConnectorJobOutput.OutputType; import io.airbyte.config.FailureReason; @@ -420,8 +421,11 @@ private SyncCheckConnectionFailure checkConnections(final GenerateInputActivity. return checkFailure; } - final StandardCheckConnectionInput sourceConfiguration = new StandardCheckConnectionInput().withConnectionConfiguration(sourceConfig); - final CheckConnectionInput checkSourceInput = new CheckConnectionInput(jobRunConfig, sourceLauncherConfig, sourceConfiguration); + final StandardCheckConnectionInput standardCheckInputSource = new StandardCheckConnectionInput() + .withActorType(ActorType.SOURCE) + .withActorId(syncInput.getSourceId()) + .withConnectionConfiguration(sourceConfig); + final CheckConnectionInput checkSourceInput = new CheckConnectionInput(jobRunConfig, sourceLauncherConfig, standardCheckInputSource); final int checkJobOutputVersion = Workflow.getVersion(CHECK_PREVIOUS_JOB_OR_ATTEMPT_TAG, Workflow.DEFAULT_VERSION, CHECK_PREVIOUS_JOB_OR_ATTEMPT_TAG_CURRENT_VERSION); @@ -447,8 +451,12 @@ private SyncCheckConnectionFailure checkConnections(final GenerateInputActivity. } } - final StandardCheckConnectionInput destinationConfiguration = new StandardCheckConnectionInput().withConnectionConfiguration(destinationConfig); - final CheckConnectionInput checkDestinationInput = new CheckConnectionInput(jobRunConfig, destinationLauncherConfig, destinationConfiguration); + final StandardCheckConnectionInput standardCheckInputDestination = new StandardCheckConnectionInput() + .withActorType(ActorType.DESTINATION) + .withActorId(syncInput.getDestinationId()) + .withConnectionConfiguration(destinationConfig); + final CheckConnectionInput checkDestinationInput = + new CheckConnectionInput(jobRunConfig, destinationLauncherConfig, standardCheckInputDestination); if (checkFailure.isFailed() || !isLastJobOrAttemptFailure) { log.info("DESTINATION CHECK: Skipped"); diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/general/DefaultCheckConnectionWorkerTest.java b/airbyte-workers/src/test/java/io/airbyte/workers/general/DefaultCheckConnectionWorkerTest.java index f5685c245e44..76e212f9584b 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/general/DefaultCheckConnectionWorkerTest.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/general/DefaultCheckConnectionWorkerTest.java @@ -18,6 +18,7 @@ import com.google.common.collect.Lists; import io.airbyte.commons.enums.Enums; import io.airbyte.commons.json.Jsons; +import io.airbyte.config.ActorType; import io.airbyte.config.ConnectorJobOutput; import io.airbyte.config.ConnectorJobOutput.OutputType; import io.airbyte.config.FailureReason; @@ -29,6 +30,7 @@ import io.airbyte.protocol.models.AirbyteMessage.Type; import io.airbyte.workers.WorkerConstants; import io.airbyte.workers.exception.WorkerException; +import io.airbyte.workers.helper.ConnectorConfigUpdater; import io.airbyte.workers.internal.AirbyteStreamFactory; import io.airbyte.workers.process.IntegrationLauncher; import io.airbyte.workers.test_utils.AirbyteMessageUtils; @@ -37,6 +39,7 @@ import java.io.InputStream; import java.nio.file.Files; import java.nio.file.Path; +import java.util.UUID; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -46,9 +49,13 @@ class DefaultCheckConnectionWorkerTest { private static final Path TEST_ROOT = Path.of("/tmp/airbyte_tests"); private static final JsonNode CREDS = Jsons.jsonNode(ImmutableMap.builder().put("apiKey", "123").build()); + private static final ActorType ACTOR_TYPE = ActorType.SOURCE; + private static final UUID ACTOR_ID = UUID.randomUUID(); + private Path jobRoot; private StandardCheckConnectionInput input; private IntegrationLauncher integrationLauncher; + private ConnectorConfigUpdater connectorConfigUpdater; private Process process; private AirbyteStreamFactory successStreamFactory; private AirbyteStreamFactory failureStreamFactory; @@ -56,10 +63,11 @@ class DefaultCheckConnectionWorkerTest { @BeforeEach void setup() throws IOException, WorkerException { - input = new StandardCheckConnectionInput().withConnectionConfiguration(CREDS); + input = new StandardCheckConnectionInput().withConnectionConfiguration(CREDS).withActorType(ACTOR_TYPE).withActorId(ACTOR_ID); jobRoot = Files.createTempDirectory(Files.createDirectories(TEST_ROOT), ""); integrationLauncher = mock(IntegrationLauncher.class, RETURNS_DEEP_STUBS); + connectorConfigUpdater = mock(ConnectorConfigUpdater.class); process = mock(Process.class); when(integrationLauncher.check(jobRoot, WorkerConstants.SOURCE_CONFIG_JSON_FILENAME, Jsons.serialize(CREDS))).thenReturn(process); @@ -88,7 +96,7 @@ void testEnums() { @Test void testSuccessfulConnection() throws WorkerException { - final DefaultCheckConnectionWorker worker = new DefaultCheckConnectionWorker(integrationLauncher, successStreamFactory); + final DefaultCheckConnectionWorker worker = new DefaultCheckConnectionWorker(integrationLauncher, connectorConfigUpdater, successStreamFactory); final ConnectorJobOutput output = worker.run(input, jobRoot); assertEquals(output.getOutputType(), OutputType.CHECK_CONNECTION); @@ -101,7 +109,7 @@ void testSuccessfulConnection() throws WorkerException { @Test void testFailedConnection() throws WorkerException { - final DefaultCheckConnectionWorker worker = new DefaultCheckConnectionWorker(integrationLauncher, failureStreamFactory); + final DefaultCheckConnectionWorker worker = new DefaultCheckConnectionWorker(integrationLauncher, connectorConfigUpdater, failureStreamFactory); final ConnectorJobOutput output = worker.run(input, jobRoot); assertEquals(output.getOutputType(), OutputType.CHECK_CONNECTION); @@ -116,7 +124,7 @@ void testFailedConnection() throws WorkerException { void testProcessFail() { when(process.exitValue()).thenReturn(1); - final DefaultCheckConnectionWorker worker = new DefaultCheckConnectionWorker(integrationLauncher, failureStreamFactory); + final DefaultCheckConnectionWorker worker = new DefaultCheckConnectionWorker(integrationLauncher, connectorConfigUpdater, failureStreamFactory); assertThrows(WorkerException.class, () -> worker.run(input, jobRoot)); } @@ -124,7 +132,8 @@ void testProcessFail() { void testProcessFailWithTraceMessage() throws WorkerException { when(process.exitValue()).thenReturn(1); - final DefaultCheckConnectionWorker worker = new DefaultCheckConnectionWorker(integrationLauncher, traceMessageStreamFactory); + final DefaultCheckConnectionWorker worker = + new DefaultCheckConnectionWorker(integrationLauncher, connectorConfigUpdater, traceMessageStreamFactory); final ConnectorJobOutput output = worker.run(input, jobRoot); assertEquals(output.getOutputType(), OutputType.CHECK_CONNECTION); @@ -138,14 +147,14 @@ void testProcessFailWithTraceMessage() throws WorkerException { void testExceptionThrownInRun() throws WorkerException { doThrow(new RuntimeException()).when(integrationLauncher).check(jobRoot, WorkerConstants.SOURCE_CONFIG_JSON_FILENAME, Jsons.serialize(CREDS)); - final DefaultCheckConnectionWorker worker = new DefaultCheckConnectionWorker(integrationLauncher, failureStreamFactory); + final DefaultCheckConnectionWorker worker = new DefaultCheckConnectionWorker(integrationLauncher, connectorConfigUpdater, failureStreamFactory); assertThrows(WorkerException.class, () -> worker.run(input, jobRoot)); } @Test void testCancel() throws WorkerException { - final DefaultCheckConnectionWorker worker = new DefaultCheckConnectionWorker(integrationLauncher, successStreamFactory); + final DefaultCheckConnectionWorker worker = new DefaultCheckConnectionWorker(integrationLauncher, connectorConfigUpdater, successStreamFactory); worker.run(input, jobRoot); worker.cancel(); From 784414d88d0da4e108d21bd550034adefd13bf31 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 28 Dec 2022 00:57:19 -0400 Subject: [PATCH 48/63] persist config updates on check_connection_for_update --- airbyte-api/src/main/openapi/config.yaml | 4 ++++ .../server/handlers/SchedulerHandler.java | 19 ++++++++++++------- .../api/generated-api-html/index.html | 6 ++++-- 3 files changed, 20 insertions(+), 9 deletions(-) diff --git a/airbyte-api/src/main/openapi/config.yaml b/airbyte-api/src/main/openapi/config.yaml index 4883fb4acc39..96aa122b2fcf 100644 --- a/airbyte-api/src/main/openapi/config.yaml +++ b/airbyte-api/src/main/openapi/config.yaml @@ -2667,6 +2667,8 @@ components: - connectionConfiguration - workspaceId properties: + sourceId: + $ref: "#/components/schemas/SourceId" sourceDefinitionId: $ref: "#/components/schemas/SourceDefinitionId" connectionConfiguration: @@ -2959,6 +2961,8 @@ components: - destinationDefinitionId - connectionConfiguration properties: + destinationId: + $ref: "#/components/schemas/DestinationId" destinationDefinitionId: $ref: "#/components/schemas/DestinationDefinitionId" connectionConfiguration: 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 f0041e0c4d86..ff9e519051f4 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 @@ -169,6 +169,7 @@ public CheckConnectionRead checkSourceConnectionFromSourceCreate(final SourceCor // todo (cgardens) - narrow the struct passed to the client. we are not setting fields that are // technically declared as required. final SourceConnection source = new SourceConnection() + .withSourceId(sourceConfig.getSourceId()) .withSourceDefinitionId(sourceConfig.getSourceDefinitionId()) .withConfiguration(partialConfig) .withWorkspaceId(sourceConfig.getWorkspaceId()); @@ -189,6 +190,7 @@ public CheckConnectionRead checkSourceConnectionFromSourceIdForUpdate(final Sour jsonSchemaValidator.ensure(spec.getConnectionSpecification(), updatedSource.getConfiguration()); final SourceCoreConfig sourceCoreConfig = new SourceCoreConfig() + .sourceId(updatedSource.getSourceId()) .connectionConfiguration(updatedSource.getConfiguration()) .sourceDefinitionId(updatedSource.getSourceDefinitionId()); @@ -217,6 +219,7 @@ public CheckConnectionRead checkDestinationConnectionFromDestinationCreate(final // todo (cgardens) - narrow the struct passed to the client. we are not setting fields that are // technically declared as required. final DestinationConnection destination = new DestinationConnection() + .withDestinationId(destinationConfig.getDestinationId()) .withDestinationDefinitionId(destinationConfig.getDestinationDefinitionId()) .withConfiguration(partialConfig) .withWorkspaceId(destinationConfig.getWorkspaceId()); @@ -235,6 +238,7 @@ public CheckConnectionRead checkDestinationConnectionFromDestinationIdForUpdate( jsonSchemaValidator.ensure(spec.getConnectionSpecification(), updatedDestination.getConfiguration()); final DestinationCoreConfig destinationCoreConfig = new DestinationCoreConfig() + .destinationId(updatedDestination.getDestinationId()) .connectionConfiguration(updatedDestination.getConfiguration()) .destinationDefinitionId(updatedDestination.getDestinationDefinitionId()); @@ -390,8 +394,8 @@ public JobInfoRead cancelJob(final JobIdRequestBody jobIdRequestBody) throws IOE // wants the connection disabled when non-breaking changes are detected. If so, disable that // connection. Modify the current discoveredSchema object to add a CatalogDiff, // containsBreakingChange paramter, and connectionStatus parameter. - private void generateCatalogDiffsAndDisableConnectionsIfNeeded(SourceDiscoverSchemaRead discoveredSchema, - SourceDiscoverSchemaRequestBody discoverSchemaRequestBody) + private void generateCatalogDiffsAndDisableConnectionsIfNeeded(final SourceDiscoverSchemaRead discoveredSchema, + final SourceDiscoverSchemaRequestBody discoverSchemaRequestBody) throws JsonValidationException, ConfigNotFoundException, IOException { final ConnectionReadList connectionsForSource = connectionsHandler.listConnectionsForSource(discoverSchemaRequestBody.getSourceId(), false); for (final ConnectionRead connectionRead : connectionsForSource.getConnections()) { @@ -399,12 +403,13 @@ private void generateCatalogDiffsAndDisableConnectionsIfNeeded(SourceDiscoverSch .getConnectionAirbyteCatalog(connectionRead.getConnectionId()); final io.airbyte.api.model.generated.@NotNull AirbyteCatalog currentAirbyteCatalog = connectionRead.getSyncCatalog(); - CatalogDiff diff = connectionsHandler.getDiff(catalogUsedToMakeConfiguredCatalog.orElse(currentAirbyteCatalog), discoveredSchema.getCatalog(), - CatalogConverter.toProtocol(currentAirbyteCatalog)); - boolean containsBreakingChange = containsBreakingChange(diff); - ConnectionUpdate updateObject = + final CatalogDiff diff = + connectionsHandler.getDiff(catalogUsedToMakeConfiguredCatalog.orElse(currentAirbyteCatalog), discoveredSchema.getCatalog(), + CatalogConverter.toProtocol(currentAirbyteCatalog)); + final boolean containsBreakingChange = containsBreakingChange(diff); + final ConnectionUpdate updateObject = new ConnectionUpdate().breakingChange(containsBreakingChange).connectionId(connectionRead.getConnectionId()); - ConnectionStatus connectionStatus; + final ConnectionStatus connectionStatus; if (shouldDisableConnection(containsBreakingChange, connectionRead.getNonBreakingChangesPreference(), diff)) { connectionStatus = ConnectionStatus.INACTIVE; } else { diff --git a/docs/reference/api/generated-api-html/index.html b/docs/reference/api/generated-api-html/index.html index 0f5b3cfee5bb..ad4385dc4424 100644 --- a/docs/reference/api/generated-api-html/index.html +++ b/docs/reference/api/generated-api-html/index.html @@ -10670,7 +10670,8 @@

DestinationCloneRequestBodyDestinationCoreConfig - Up

-
destinationDefinitionId
UUID format: uuid
+
destinationId (optional)
UUID format: uuid
+
destinationDefinitionId
UUID format: uuid
connectionConfiguration
workspaceId
UUID format: uuid
@@ -11386,7 +11387,8 @@

SourceCloneRequestBody -

SourceCoreConfig - Up

-
sourceDefinitionId
UUID format: uuid
+
sourceId (optional)
UUID format: uuid
+
sourceDefinitionId
UUID format: uuid
connectionConfiguration
workspaceId
UUID format: uuid
From 6fd7ad300d7c4e5f48695cac490b77a2b18554de Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 28 Dec 2022 01:03:37 -0400 Subject: [PATCH 49/63] get last config message rather than first --- .../src/main/java/io/airbyte/workers/WorkerUtils.java | 10 +++++++--- .../workers/general/DefaultCheckConnectionWorker.java | 2 +- .../workers/general/DefaultDiscoverCatalogWorker.java | 2 +- 3 files changed, 9 insertions(+), 5 deletions(-) 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 db89cf05b55e..a39a68c84636 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 @@ -111,12 +111,16 @@ public static WorkerDestinationConfig syncToWorkerDestinationConfig(final Standa .withState(sync.getState()); } - public static Optional getConfigControlMessage(final Map> messagesByType) { - return messagesByType.getOrDefault(Type.CONTROL, new ArrayList<>()).stream() + public static Optional getLastConfigControlMessage(final Map> messagesByType) { + final List configMessages = messagesByType.getOrDefault(Type.CONTROL, new ArrayList<>()).stream() .map(AirbyteMessage::getControl) .filter(control -> control.getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG) .map(AirbyteControlMessage::getConnectorConfig) - .findFirst(); + .toList(); + + if (configMessages.size() == 0) + return Optional.empty(); + return Optional.of(configMessages.get(configMessages.size() - 1)); } public static ConnectorJobOutput getJobFailureOutputOrThrow(final OutputType outputType, diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultCheckConnectionWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultCheckConnectionWorker.java index 5d2e9cc403cd..afe2dae6b410 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultCheckConnectionWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultCheckConnectionWorker.java @@ -91,7 +91,7 @@ public ConnectorJobOutput run(final StandardCheckConnectionInput input, final Pa if (input.getActorId() != null && input.getActorType() != null) { LOGGER.info("has actor and type"); - final Optional optionalConfigMsg = WorkerUtils.getConfigControlMessage(messagesByType); + final Optional optionalConfigMsg = WorkerUtils.getLastConfigControlMessage(messagesByType); LOGGER.info("opt config msg {}", optionalConfigMsg.orElse(null)); optionalConfigMsg.ifPresent( configMessage -> { diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorker.java index ac119b2664a6..009142f9230d 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorker.java @@ -96,7 +96,7 @@ public ConnectorJobOutput run(final StandardDiscoverCatalogInput discoverSchemaI .map(AirbyteMessage::getCatalog) .findFirst(); - final Optional optionalConfigMsg = WorkerUtils.getConfigControlMessage(messagesByType); + final Optional optionalConfigMsg = WorkerUtils.getLastConfigControlMessage(messagesByType); optionalConfigMsg.ifPresent( configMessage -> connectorConfigUpdater.updateSource( UUID.fromString(discoverSchemaInput.getSourceId()), From c655be8deaeac4cc00b6b8a9ee55ff1b4b194ccc Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 28 Dec 2022 11:23:40 -0400 Subject: [PATCH 50/63] fix pmd --- .../src/main/java/io/airbyte/workers/WorkerUtils.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) 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 a39a68c84636..9c4f9988983e 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 @@ -118,9 +118,7 @@ public static Optional getLastConfigContro .map(AirbyteControlMessage::getConnectorConfig) .toList(); - if (configMessages.size() == 0) - return Optional.empty(); - return Optional.of(configMessages.get(configMessages.size() - 1)); + return configMessages.isEmpty() ? Optional.empty() : Optional.of(configMessages.get(configMessages.size() - 1)); } public static ConnectorJobOutput getJobFailureOutputOrThrow(final OutputType outputType, From c658271605fefb688dc98e44e31dc676c969a19a Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 28 Dec 2022 14:25:42 -0400 Subject: [PATCH 51/63] fix failing tests --- .../io/airbyte/server/handlers/SchedulerHandlerTest.java | 6 ++++-- .../scheduler/DefaultSynchronousSchedulerClientTest.java | 5 +++++ 2 files changed, 9 insertions(+), 2 deletions(-) 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 957f80de8885..352fb8d24f4d 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 @@ -259,6 +259,7 @@ void testCheckSourceConnectionFromUpdate() throws IOException, JsonValidationExc when(configRepository.getSourceConnection(source.getSourceId())).thenReturn(source); when(configurationUpdate.source(source.getSourceId(), source.getName(), sourceUpdate.getConnectionConfiguration())).thenReturn(source); final SourceConnection submittedSource = new SourceConnection() + .withSourceId(source.getSourceId()) .withSourceDefinitionId(source.getSourceDefinitionId()) .withConfiguration(source.getConfiguration()); when(synchronousSchedulerClient.createSourceCheckConnectionJob(submittedSource, DESTINATION_DOCKER_IMAGE, protocolVersion, false)) @@ -383,6 +384,7 @@ void testCheckDestinationConnectionFromUpdate() throws IOException, JsonValidati when(configurationUpdate.destination(destination.getDestinationId(), destination.getName(), destinationUpdate.getConnectionConfiguration())) .thenReturn(destination); final DestinationConnection submittedDestination = new DestinationConnection() + .withDestinationId(destination.getDestinationId()) .withDestinationDefinitionId(destination.getDestinationDefinitionId()) .withConfiguration(destination.getConfiguration()); when(synchronousSchedulerClient.createDestinationCheckConnectionJob(submittedDestination, DESTINATION_DOCKER_IMAGE, @@ -952,9 +954,9 @@ void testDiscoverSchemaForSourceMultipleConnectionsFeatureFlagOn() throws IOExce assertEquals(expectedActorCatalog, actual.getCatalog()); assertEquals(ConnectionStatus.ACTIVE, actual.getConnectionStatus()); - ArgumentCaptor expectedArgumentCaptor = ArgumentCaptor.forClass(ConnectionUpdate.class); + final ArgumentCaptor expectedArgumentCaptor = ArgumentCaptor.forClass(ConnectionUpdate.class); verify(connectionsHandler, times(3)).updateConnection(expectedArgumentCaptor.capture()); - List connectionUpdateValues = expectedArgumentCaptor.getAllValues(); + final List connectionUpdateValues = expectedArgumentCaptor.getAllValues(); assertEquals(ConnectionStatus.ACTIVE, connectionUpdateValues.get(0).getStatus()); assertEquals(ConnectionStatus.ACTIVE, connectionUpdateValues.get(1).getStatus()); assertEquals(ConnectionStatus.INACTIVE, connectionUpdateValues.get(2).getStatus()); 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 909c21a4b71b..c8a22ef658db 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 @@ -24,6 +24,7 @@ import io.airbyte.commons.temporal.TemporalClient; import io.airbyte.commons.temporal.TemporalResponse; import io.airbyte.commons.version.Version; +import io.airbyte.config.ActorType; import io.airbyte.config.ConnectorJobOutput; import io.airbyte.config.DestinationConnection; import io.airbyte.config.JobCheckConnectionConfig; @@ -202,6 +203,8 @@ class TestJobCreation { @Test void testCreateSourceCheckConnectionJob() throws IOException { final JobCheckConnectionConfig jobCheckConnectionConfig = new JobCheckConnectionConfig() + .withActorType(ActorType.SOURCE) + .withActorId(SOURCE_CONNECTION.getSourceId()) .withConnectionConfiguration(SOURCE_CONNECTION.getConfiguration()) .withDockerImage(DOCKER_IMAGE) .withProtocolVersion(PROTOCOL_VERSION).withIsCustomConnector(false); @@ -218,6 +221,8 @@ void testCreateSourceCheckConnectionJob() throws IOException { @Test void testCreateDestinationCheckConnectionJob() throws IOException { final JobCheckConnectionConfig jobCheckConnectionConfig = new JobCheckConnectionConfig() + .withActorType(ActorType.DESTINATION) + .withActorId(DESTINATION_CONNECTION.getDestinationId()) .withConnectionConfiguration(DESTINATION_CONNECTION.getConfiguration()) .withDockerImage(DOCKER_IMAGE) .withProtocolVersion(PROTOCOL_VERSION) From ee133d23b39ffc18b65774f672a9fb38f08141e6 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 28 Dec 2022 14:49:27 -0400 Subject: [PATCH 52/63] add tests --- .../DefaultCheckConnectionWorkerTest.java | 65 +++++++++++++++++++ .../DefaultDiscoverCatalogWorkerTest.java | 32 +++++++++ 2 files changed, 97 insertions(+) diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/general/DefaultCheckConnectionWorkerTest.java b/airbyte-workers/src/test/java/io/airbyte/workers/general/DefaultCheckConnectionWorkerTest.java index 76e212f9584b..a7375fe9e8f0 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/general/DefaultCheckConnectionWorkerTest.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/general/DefaultCheckConnectionWorkerTest.java @@ -11,6 +11,8 @@ import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; +import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; import com.fasterxml.jackson.databind.JsonNode; @@ -28,6 +30,7 @@ import io.airbyte.protocol.models.AirbyteConnectionStatus; import io.airbyte.protocol.models.AirbyteMessage; import io.airbyte.protocol.models.AirbyteMessage.Type; +import io.airbyte.protocol.models.Config; import io.airbyte.workers.WorkerConstants; import io.airbyte.workers.exception.WorkerException; import io.airbyte.workers.helper.ConnectorConfigUpdater; @@ -48,6 +51,7 @@ class DefaultCheckConnectionWorkerTest { private static final Path TEST_ROOT = Path.of("/tmp/airbyte_tests"); private static final JsonNode CREDS = Jsons.jsonNode(ImmutableMap.builder().put("apiKey", "123").build()); + private static final Config CONNECTOR_CONFIG = new Config().withAdditionalProperty("apiKey", "321"); private static final ActorType ACTOR_TYPE = ActorType.SOURCE; private static final UUID ACTOR_ID = UUID.randomUUID(); @@ -60,6 +64,7 @@ class DefaultCheckConnectionWorkerTest { private AirbyteStreamFactory successStreamFactory; private AirbyteStreamFactory failureStreamFactory; private AirbyteStreamFactory traceMessageStreamFactory; + private AirbyteStreamFactory configMessageStreamFactory; @BeforeEach void setup() throws IOException, WorkerException { @@ -87,6 +92,10 @@ void setup() throws IOException, WorkerException { final AirbyteMessage traceMessage = AirbyteMessageUtils.createErrorMessage("some error from the connector", 123.0); traceMessageStreamFactory = noop -> Lists.newArrayList(traceMessage).stream(); + + final AirbyteMessage configMessage1 = AirbyteMessageUtils.createConfigControlMessage(new Config().withAdditionalProperty("apiKey", "123"), 1D); + final AirbyteMessage configMessage2 = AirbyteMessageUtils.createConfigControlMessage(CONNECTOR_CONFIG, 2D); + configMessageStreamFactory = noop -> Lists.newArrayList(configMessage1, configMessage2, successMessage).stream(); } @Test @@ -99,6 +108,62 @@ void testSuccessfulConnection() throws WorkerException { final DefaultCheckConnectionWorker worker = new DefaultCheckConnectionWorker(integrationLauncher, connectorConfigUpdater, successStreamFactory); final ConnectorJobOutput output = worker.run(input, jobRoot); + verifyNoInteractions(connectorConfigUpdater); + + assertEquals(output.getOutputType(), OutputType.CHECK_CONNECTION); + assertNull(output.getFailureReason()); + + final StandardCheckConnectionOutput checkOutput = output.getCheckConnection(); + assertEquals(Status.SUCCEEDED, checkOutput.getStatus()); + assertNull(checkOutput.getMessage()); + } + + @Test + void testCheckConnectionWithConfigUpdateSource() throws WorkerException { + final DefaultCheckConnectionWorker worker = + new DefaultCheckConnectionWorker(integrationLauncher, connectorConfigUpdater, configMessageStreamFactory); + final ConnectorJobOutput output = worker.run(input, jobRoot); + + verify(connectorConfigUpdater).updateSource(ACTOR_ID, CONNECTOR_CONFIG); + verifyNoMoreInteractions(connectorConfigUpdater); + + assertEquals(output.getOutputType(), OutputType.CHECK_CONNECTION); + assertNull(output.getFailureReason()); + + final StandardCheckConnectionOutput checkOutput = output.getCheckConnection(); + assertEquals(Status.SUCCEEDED, checkOutput.getStatus()); + assertNull(checkOutput.getMessage()); + } + + @Test + void testCheckConnectionWithConfigUpdateDestination() throws WorkerException { + final DefaultCheckConnectionWorker worker = + new DefaultCheckConnectionWorker(integrationLauncher, connectorConfigUpdater, configMessageStreamFactory); + + final StandardCheckConnectionInput destinationInput = input.withActorType(ActorType.DESTINATION); + final ConnectorJobOutput output = worker.run(destinationInput, jobRoot); + + verify(connectorConfigUpdater).updateDestination(ACTOR_ID, CONNECTOR_CONFIG); + verifyNoMoreInteractions(connectorConfigUpdater); + + assertEquals(output.getOutputType(), OutputType.CHECK_CONNECTION); + assertNull(output.getFailureReason()); + + final StandardCheckConnectionOutput checkOutput = output.getCheckConnection(); + assertEquals(Status.SUCCEEDED, checkOutput.getStatus()); + assertNull(checkOutput.getMessage()); + } + + @Test + void testCheckConnectionWithConfigUpdateNoActor() throws WorkerException { + final DefaultCheckConnectionWorker worker = + new DefaultCheckConnectionWorker(integrationLauncher, connectorConfigUpdater, configMessageStreamFactory); + + final StandardCheckConnectionInput noActorInput = input.withActorId(null); + final ConnectorJobOutput output = worker.run(noActorInput, jobRoot); + + verifyNoInteractions(connectorConfigUpdater); + assertEquals(output.getOutputType(), OutputType.CHECK_CONNECTION); assertNull(output.getFailureReason()); diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorkerTest.java b/airbyte-workers/src/test/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorkerTest.java index cadd93c1b886..bec8809e647b 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorkerTest.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorkerTest.java @@ -17,6 +17,7 @@ import static org.mockito.Mockito.RETURNS_DEEP_STUBS; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; import com.fasterxml.jackson.databind.JsonNode; @@ -34,6 +35,7 @@ import io.airbyte.protocol.models.AirbyteMessage; import io.airbyte.protocol.models.AirbyteMessage.Type; import io.airbyte.protocol.models.CatalogHelpers; +import io.airbyte.protocol.models.Config; import io.airbyte.protocol.models.Field; import io.airbyte.protocol.models.JsonSchemaType; import io.airbyte.workers.WorkerConstants; @@ -114,6 +116,36 @@ void testDiscoverSchema() throws Exception { assertEquals(OutputType.DISCOVER_CATALOG_ID, output.getOutputType()); assertEquals(CATALOG_ID, output.getDiscoverCatalogId()); verify(mConfigRepository).writeActorCatalogFetchEvent(eq(CATALOG), eq(SOURCE_ID), any(), any()); + verifyNoInteractions(connectorConfigUpdater); + + Assertions.assertTimeout(Duration.ofSeconds(5), () -> { + while (process.getErrorStream().available() != 0) { + Thread.sleep(50); + } + }); + + verify(process).exitValue(); + } + + @SuppressWarnings("BusyWait") + @Test + void testDiscoverSchemaWithConfigUpdate() throws Exception { + final Config connectorConfig1 = new Config().withAdditionalProperty("apiKey", "123"); + final Config connectorConfig2 = new Config().withAdditionalProperty("apiKey", "321"); + final AirbyteStreamFactory configMsgStreamFactory = noop -> Lists.newArrayList( + AirbyteMessageUtils.createConfigControlMessage(connectorConfig1, 1D), + AirbyteMessageUtils.createConfigControlMessage(connectorConfig2, 2D), + new AirbyteMessage().withType(Type.CATALOG).withCatalog(CATALOG)).stream(); + + final DefaultDiscoverCatalogWorker worker = + new DefaultDiscoverCatalogWorker(mConfigRepository, integrationLauncher, connectorConfigUpdater, configMsgStreamFactory); + final ConnectorJobOutput output = worker.run(INPUT, jobRoot); + + assertNull(output.getFailureReason()); + assertEquals(OutputType.DISCOVER_CATALOG_ID, output.getOutputType()); + assertEquals(CATALOG_ID, output.getDiscoverCatalogId()); + verify(mConfigRepository).writeActorCatalogFetchEvent(eq(CATALOG), eq(SOURCE_ID), any(), any()); + verify(connectorConfigUpdater).updateSource(SOURCE_ID, connectorConfig2); Assertions.assertTimeout(Duration.ofSeconds(5), () -> { while (process.getErrorStream().available() != 0) { From 90b707bf8c7b773722c6e8b09dd12bdcfeeabc54 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 28 Dec 2022 14:51:50 -0400 Subject: [PATCH 53/63] source id not required for check connection (create case) --- .../src/main/resources/types/StandardCheckConnectionInput.yaml | 2 -- 1 file changed, 2 deletions(-) diff --git a/airbyte-config/config-models/src/main/resources/types/StandardCheckConnectionInput.yaml b/airbyte-config/config-models/src/main/resources/types/StandardCheckConnectionInput.yaml index 8579a5338b75..f0ea0caaba4e 100644 --- a/airbyte-config/config-models/src/main/resources/types/StandardCheckConnectionInput.yaml +++ b/airbyte-config/config-models/src/main/resources/types/StandardCheckConnectionInput.yaml @@ -6,8 +6,6 @@ description: information required for connection. type: object required: - connectionConfiguration - - actorType - - actorId additionalProperties: false properties: actorType: From 935c3ee9ef0004f6042db9f8eeab0111e4619d55 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 28 Dec 2022 15:42:04 -0400 Subject: [PATCH 54/63] suppress pmd warning for BusyWait literal --- .../workers/general/DefaultDiscoverCatalogWorkerTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/airbyte-workers/src/test/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorkerTest.java b/airbyte-workers/src/test/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorkerTest.java index bec8809e647b..67b76d951043 100644 --- a/airbyte-workers/src/test/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorkerTest.java +++ b/airbyte-workers/src/test/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorkerTest.java @@ -55,6 +55,7 @@ import org.junit.jupiter.api.Test; import org.mockito.Mock; +@SuppressWarnings("PMD.AvoidDuplicateLiterals") class DefaultDiscoverCatalogWorkerTest { @Mock From d0f865a635f30f7413b86e2bb8208a52d2eb7109 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Wed, 28 Dec 2022 17:12:53 -0400 Subject: [PATCH 55/63] source id not required for checkc onnection (create case) (p2) --- .../src/main/resources/types/JobCheckConnectionConfig.yaml | 2 -- 1 file changed, 2 deletions(-) diff --git a/airbyte-config/config-models/src/main/resources/types/JobCheckConnectionConfig.yaml b/airbyte-config/config-models/src/main/resources/types/JobCheckConnectionConfig.yaml index d2706dfe9532..bf93ce699872 100644 --- a/airbyte-config/config-models/src/main/resources/types/JobCheckConnectionConfig.yaml +++ b/airbyte-config/config-models/src/main/resources/types/JobCheckConnectionConfig.yaml @@ -6,8 +6,6 @@ description: job check connection config type: object additionalProperties: false required: - - actorType - - actorId - connectionConfiguration - dockerImage properties: From d6eb05f4a9a4edfee6c566ed888a7711f9c41449 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Tue, 3 Jan 2023 17:46:49 -0400 Subject: [PATCH 56/63] pass id, not full config to runnables/accept control message --- .../general/DefaultReplicationWorker.java | 21 ++++++++++--------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java index a0e10b94c72d..4cbec8810ef6 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultReplicationWorker.java @@ -52,6 +52,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -196,7 +197,7 @@ private void replicate(final Path jobRoot, // note: `whenComplete` is used instead of `exceptionally` so that the original exception is still // thrown final CompletableFuture readFromDstThread = CompletableFuture.runAsync( - readFromDstRunnable(destination, cancelled, messageTracker, connectorConfigUpdater, mdc, timeTracker, destinationConfig), + readFromDstRunnable(destination, cancelled, messageTracker, connectorConfigUpdater, mdc, timeTracker, destinationConfig.getDestinationId()), executors) .whenComplete((msg, ex) -> { if (ex != null) { @@ -222,7 +223,7 @@ private void replicate(final Path jobRoot, recordSchemaValidator, metricReporter, timeTracker, - sourceConfig, + sourceConfig.getSourceId(), fieldSelectionEnabled), executors) .whenComplete((msg, ex) -> { @@ -263,7 +264,7 @@ private static Runnable readFromDstRunnable(final AirbyteDestination destination final ConnectorConfigUpdater connectorConfigUpdater, final Map mdc, final ThreadedTimeTracker timeHolder, - final WorkerDestinationConfig destinationConfig) { + final UUID destinationId) { return () -> { MDC.setContextMap(mdc); LOGGER.info("Destination output thread started."); @@ -283,7 +284,7 @@ private static Runnable readFromDstRunnable(final AirbyteDestination destination try { if (message.getType() == Type.CONTROL) { - acceptDstControlMessage(destinationConfig, message.getControl(), connectorConfigUpdater); + acceptDstControlMessage(destinationId, message.getControl(), connectorConfigUpdater); } } catch (final Exception e) { LOGGER.error("Error updating destination configuration", e); @@ -324,7 +325,7 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou final RecordSchemaValidator recordSchemaValidator, final WorkerMetricReporter metricReporter, final ThreadedTimeTracker timeHolder, - final WorkerSourceConfig sourceConfig, + final UUID sourceId, final boolean fieldSelectionEnabled) { return () -> { MDC.setContextMap(mdc); @@ -356,7 +357,7 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou try { if (message.getType() == Type.CONTROL) { - acceptSrcControlMessage(sourceConfig, message.getControl(), connectorConfigUpdater); + acceptSrcControlMessage(sourceId, message.getControl(), connectorConfigUpdater); } } catch (final Exception e) { LOGGER.error("Error updating source configuration", e); @@ -420,19 +421,19 @@ private static Runnable readFromSrcAndWriteToDstRunnable(final AirbyteSource sou }; } - private static void acceptSrcControlMessage(final WorkerSourceConfig sourceConfig, + private static void acceptSrcControlMessage(final UUID sourceId, final AirbyteControlMessage controlMessage, final ConnectorConfigUpdater connectorConfigUpdater) { if (controlMessage.getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG) { - connectorConfigUpdater.updateSource(sourceConfig.getSourceId(), controlMessage.getConnectorConfig().getConfig()); + connectorConfigUpdater.updateSource(sourceId, controlMessage.getConnectorConfig().getConfig()); } } - private static void acceptDstControlMessage(final WorkerDestinationConfig destinationConfig, + private static void acceptDstControlMessage(final UUID destinationId, final AirbyteControlMessage controlMessage, final ConnectorConfigUpdater connectorConfigUpdater) { if (controlMessage.getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG) { - connectorConfigUpdater.updateDestination(destinationConfig.getDestinationId(), controlMessage.getConnectorConfig().getConfig()); + connectorConfigUpdater.updateDestination(destinationId, controlMessage.getConnectorConfig().getConfig()); } } From 87c8fe20addaa7608b5c231b916961460b1ab6d1 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Tue, 3 Jan 2023 18:21:45 -0400 Subject: [PATCH 57/63] add new config required for api client --- .../src/main/resources/application.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/airbyte-container-orchestrator/src/main/resources/application.yml b/airbyte-container-orchestrator/src/main/resources/application.yml index 42263e9a7382..fcfbefa0432b 100644 --- a/airbyte-container-orchestrator/src/main/resources/application.yml +++ b/airbyte-container-orchestrator/src/main/resources/application.yml @@ -3,6 +3,9 @@ micronaut: port: 9000 airbyte: + acceptance: + test: + enabled: ${ACCEPTANCE_TEST_ENABLED:false} config-dir: /config control: plane: From beb9f7017662deb5e18e6e5732b05ca80d98c490 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Tue, 3 Jan 2023 18:25:29 -0400 Subject: [PATCH 58/63] add test file --- .../src/test/resources/application.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/airbyte-container-orchestrator/src/test/resources/application.yml b/airbyte-container-orchestrator/src/test/resources/application.yml index 8856e9fab93c..b789c4757d51 100644 --- a/airbyte-container-orchestrator/src/test/resources/application.yml +++ b/airbyte-container-orchestrator/src/test/resources/application.yml @@ -3,6 +3,9 @@ micronaut: port: 9000 airbyte: + acceptance: + test: + enabled: false config-dir: src/test/resources/files control: plane: From 89622f620e04088c24e9df7735fc74b2a2ab3442 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Tue, 3 Jan 2023 21:32:56 -0400 Subject: [PATCH 59/63] remove debugging logs --- .../workers/general/DefaultCheckConnectionWorker.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultCheckConnectionWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultCheckConnectionWorker.java index afe2dae6b410..55496c1fab46 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultCheckConnectionWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultCheckConnectionWorker.java @@ -90,12 +90,9 @@ public ConnectorJobOutput run(final StandardCheckConnectionInput input, final Pa .findFirst(); if (input.getActorId() != null && input.getActorType() != null) { - LOGGER.info("has actor and type"); final Optional optionalConfigMsg = WorkerUtils.getLastConfigControlMessage(messagesByType); - LOGGER.info("opt config msg {}", optionalConfigMsg.orElse(null)); optionalConfigMsg.ifPresent( configMessage -> { - LOGGER.info("the thing was present"); switch (input.getActorType()) { case SOURCE -> connectorConfigUpdater.updateSource( input.getActorId(), @@ -105,8 +102,6 @@ public ConnectorJobOutput run(final StandardCheckConnectionInput input, final Pa configMessage.getConfig()); } }); - } else { - LOGGER.info("no ids"); } if (status.isPresent() && exitCode == 0) { From d94feb1d439f6aef4e876428965b1534cf5f3d65 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Tue, 3 Jan 2023 21:35:41 -0400 Subject: [PATCH 60/63] rename method (getLast -> getMostRecent) --- .../src/main/java/io/airbyte/workers/WorkerUtils.java | 2 +- .../airbyte/workers/general/DefaultCheckConnectionWorker.java | 2 +- .../airbyte/workers/general/DefaultDiscoverCatalogWorker.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) 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 9c4f9988983e..c4225dd65e04 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 @@ -111,7 +111,7 @@ public static WorkerDestinationConfig syncToWorkerDestinationConfig(final Standa .withState(sync.getState()); } - public static Optional getLastConfigControlMessage(final Map> messagesByType) { + public static Optional getMostRecentConfigControlMessage(final Map> messagesByType) { final List configMessages = messagesByType.getOrDefault(Type.CONTROL, new ArrayList<>()).stream() .map(AirbyteMessage::getControl) .filter(control -> control.getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG) diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultCheckConnectionWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultCheckConnectionWorker.java index 55496c1fab46..21c4039981cd 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultCheckConnectionWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultCheckConnectionWorker.java @@ -90,7 +90,7 @@ public ConnectorJobOutput run(final StandardCheckConnectionInput input, final Pa .findFirst(); if (input.getActorId() != null && input.getActorType() != null) { - final Optional optionalConfigMsg = WorkerUtils.getLastConfigControlMessage(messagesByType); + final Optional optionalConfigMsg = WorkerUtils.getMostRecentConfigControlMessage(messagesByType); optionalConfigMsg.ifPresent( configMessage -> { switch (input.getActorType()) { diff --git a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorker.java b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorker.java index 009142f9230d..34eefa7e390d 100644 --- a/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorker.java +++ b/airbyte-commons-worker/src/main/java/io/airbyte/workers/general/DefaultDiscoverCatalogWorker.java @@ -96,7 +96,7 @@ public ConnectorJobOutput run(final StandardDiscoverCatalogInput discoverSchemaI .map(AirbyteMessage::getCatalog) .findFirst(); - final Optional optionalConfigMsg = WorkerUtils.getLastConfigControlMessage(messagesByType); + final Optional optionalConfigMsg = WorkerUtils.getMostRecentConfigControlMessage(messagesByType); optionalConfigMsg.ifPresent( configMessage -> connectorConfigUpdater.updateSource( UUID.fromString(discoverSchemaInput.getSourceId()), From 80029683d61677b736f07c1511bbd55a48834324 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Fri, 6 Jan 2023 01:27:40 -0400 Subject: [PATCH 61/63] rm version check (re-added this in by mistake on merge) --- .../scheduling/ConnectionManagerWorkflowImpl.java | 8 -------- 1 file changed, 8 deletions(-) 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 149f97081dab..97a999951957 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 @@ -345,14 +345,6 @@ private SyncCheckConnectionFailure checkConnections(final GenerateInputActivity. final IntegrationLauncherConfig destinationLauncherConfig = jobInputs.getDestinationLauncherConfig(); final SyncCheckConnectionFailure checkFailure = new SyncCheckConnectionFailure(jobRunConfig); - final int attemptCreationVersion = - Workflow.getVersion(CHECK_BEFORE_SYNC_TAG, Workflow.DEFAULT_VERSION, CHECK_BEFORE_SYNC_CURRENT_VERSION); - - if (attemptCreationVersion < CHECK_BEFORE_SYNC_CURRENT_VERSION) { - // return early if this instance of the workflow was created beforehand - return checkFailure; - } - final StandardCheckConnectionInput standardCheckInputSource = new StandardCheckConnectionInput() .withActorType(ActorType.SOURCE) .withActorId(syncInput.getSourceId()) From 6e4269972fd543974cacf3e4eb0dc70398901086 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Fri, 6 Jan 2023 01:34:58 -0400 Subject: [PATCH 62/63] fix test compatibility --- .../workers/general/ReplicationWorkerPerformanceTest.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/ReplicationWorkerPerformanceTest.java b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/ReplicationWorkerPerformanceTest.java index 13728602785c..860acbec2315 100644 --- a/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/ReplicationWorkerPerformanceTest.java +++ b/airbyte-commons-worker/src/test/java/io/airbyte/workers/general/ReplicationWorkerPerformanceTest.java @@ -18,6 +18,7 @@ import io.airbyte.workers.RecordSchemaValidator; import io.airbyte.workers.WorkerMetricReporter; import io.airbyte.workers.exception.WorkerException; +import io.airbyte.workers.helper.ConnectorConfigUpdater; import io.airbyte.workers.internal.NamespacingMapper; import io.airbyte.workers.internal.book_keeping.AirbyteMessageTracker; import java.io.IOException; @@ -26,6 +27,7 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicReference; import lombok.extern.slf4j.Slf4j; +import org.mockito.Mockito; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -64,6 +66,7 @@ public void executeOneSync() throws InterruptedException { final var perSource = new LimitedAirbyteSource(); final var perDestination = new EmptyAirbyteDestination(); final var messageTracker = new AirbyteMessageTracker(); + final var connectorConfigUpdater = Mockito.mock(ConnectorConfigUpdater.class); final var metricReporter = new WorkerMetricReporter(new NotImplementedMetricClient(), "test-image:0.01"); final var dstNamespaceMapper = new NamespacingMapper(NamespaceDefinitionType.DESTINATION, "", ""); final var validator = new RecordSchemaValidator(Map.of( @@ -77,6 +80,7 @@ public void executeOneSync() throws InterruptedException { messageTracker, validator, metricReporter, + connectorConfigUpdater, false); final AtomicReference output = new AtomicReference<>(); final Thread workerThread = new Thread(() -> { @@ -97,7 +101,7 @@ public void executeOneSync() throws InterruptedException { log.info("MBs read: {}, Time taken sec: {}, MB/s: {}", mbRead, timeTakenSec, mbRead / timeTakenSec); } - public static void main(String[] args) throws IOException { + public static void main(final String[] args) throws IOException { // Run this main class to start benchmarking. org.openjdk.jmh.Main.main(args); } From 60987e21f072b9f1afd67305aeb9816009012e13 Mon Sep 17 00:00:00 2001 From: Pedro Lopez Date: Fri, 6 Jan 2023 14:08:03 -0400 Subject: [PATCH 63/63] simplify --- .../src/main/java/io/airbyte/workers/WorkerUtils.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) 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 c4225dd65e04..947e2edbdee6 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 @@ -112,13 +112,11 @@ public static WorkerDestinationConfig syncToWorkerDestinationConfig(final Standa } public static Optional getMostRecentConfigControlMessage(final Map> messagesByType) { - final List configMessages = messagesByType.getOrDefault(Type.CONTROL, new ArrayList<>()).stream() + return messagesByType.getOrDefault(Type.CONTROL, new ArrayList<>()).stream() .map(AirbyteMessage::getControl) .filter(control -> control.getType() == AirbyteControlMessage.Type.CONNECTOR_CONFIG) .map(AirbyteControlMessage::getConnectorConfig) - .toList(); - - return configMessages.isEmpty() ? Optional.empty() : Optional.of(configMessages.get(configMessages.size() - 1)); + .reduce((first, second) -> second); } public static ConnectorJobOutput getJobFailureOutputOrThrow(final OutputType outputType,