nextPhase) {
+ if (!prevPhase.isEmpty()) {
+ nextPhase.putAll(prevPhase);
+ prevPhase.clear();
+ }
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/normalization/NormalizationLogParser.java b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/normalization/NormalizationLogParser.java
new file mode 100644
index 0000000000000..73a1411b72ec9
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/normalization/NormalizationLogParser.java
@@ -0,0 +1,153 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.normalization;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.google.common.annotations.VisibleForTesting;
+import io.airbyte.commons.json.Jsons;
+import io.airbyte.integrations.destination.normalization.SentryExceptionHelper.ErrorMapKeys;
+import io.airbyte.protocol.models.AirbyteErrorTraceMessage;
+import io.airbyte.protocol.models.AirbyteErrorTraceMessage.FailureType;
+import io.airbyte.protocol.models.AirbyteLogMessage;
+import io.airbyte.protocol.models.AirbyteLogMessage.Level;
+import io.airbyte.protocol.models.AirbyteMessage;
+import io.airbyte.protocol.models.AirbyteMessage.Type;
+import io.airbyte.protocol.models.AirbyteTraceMessage;
+import java.io.BufferedReader;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.logging.log4j.util.Strings;
+
+/**
+ * A simple wrapper for base-normalization logs. Reads messages off of stdin and sticks them into
+ * appropriate AirbyteMessages (log or trace), then dumps those messages to stdout
+ *
+ * does mostly the same thing as
+ * {@link io.airbyte.workers.normalization.NormalizationAirbyteStreamFactory}. That class is not
+ * actively developed, and will be deleted after all destinations run normalization in-connector.
+ *
+ * Aggregates all error logs and emits them as a single trace message at the end. If the underlying
+ * process emits any trace messages, they are passed through immediately.
+ */
+public class NormalizationLogParser {
+
+ private final List dbtErrors = new ArrayList<>();
+
+ public Stream create(final BufferedReader bufferedReader) {
+ return bufferedReader.lines().flatMap(this::toMessages);
+ }
+
+ public List getDbtErrors() {
+ return dbtErrors;
+ }
+
+ @VisibleForTesting
+ Stream toMessages(final String line) {
+ if (Strings.isEmpty(line)) {
+ return Stream.of(logMessage(Level.INFO, ""));
+ }
+ final Optional json = Jsons.tryDeserialize(line);
+ if (json.isPresent()) {
+ return jsonToMessage(json.get());
+ } else {
+ return nonJsonLineToMessage(line);
+ }
+ }
+
+ /**
+ * Wrap the line in an AirbyteLogMessage, and do very naive dbt error log detection.
+ *
+ * This is needed for dbt < 1.0.0, which don't support json-format logs.
+ */
+ private Stream nonJsonLineToMessage(final String line) {
+ // Super hacky thing to try and detect error lines
+ if (line.contains("[error]")) {
+ dbtErrors.add(line);
+ }
+ return Stream.of(logMessage(Level.INFO, line));
+ }
+
+ /**
+ * There are two cases here: Either the json is already an AirbyteMessage (and we should just emit
+ * it without change), or it's dbt json log, and we need to do some extra work to convert it to a
+ * log message + aggregate error logs.
+ */
+ private Stream jsonToMessage(final JsonNode jsonLine) {
+ final Optional message = Jsons.tryObject(jsonLine, AirbyteMessage.class);
+ if (message.isPresent()) {
+ // This line is already an AirbyteMessage; we can just return it directly
+ // (these messages come from the transform_config / transform_catalog scripts)
+ return message.stream();
+ } else {
+ /*
+ * This line is a JSON-format dbt log. We need to extract the message and wrap it in a logmessage
+ * And if it's an error, we also need to collect it into dbtErrors. Example log message, formatted
+ * for readability: { "code": "A001", "data": { "v": "=1.0.9" }, "invocation_id":
+ * "3f9a0b9f-9623-4c25-8708-1f6ae851e738", "level": "info", "log_version": 1, "msg":
+ * "Running with dbt=1.0.9", "node_info": {}, "pid": 65, "thread_name": "MainThread", "ts":
+ * "2023-04-12T21:03:23.079315Z", "type": "log_line" }
+ */
+ final String logLevel = (jsonLine.hasNonNull("level")) ? jsonLine.get("level").asText() : "";
+ String logMsg = jsonLine.hasNonNull("msg") ? jsonLine.get("msg").asText() : "";
+ Level level;
+ switch (logLevel) {
+ case "debug" -> level = Level.DEBUG;
+ case "info" -> level = Level.INFO;
+ case "warn" -> level = Level.WARN;
+ case "error" -> {
+ // This is also not _amazing_, but we make the assumption that all error logs should be emitted in
+ // the trace message
+ // In practice, this seems to be a valid assumption.
+ level = Level.ERROR;
+ dbtErrors.add(logMsg);
+ }
+ default -> {
+ level = Level.INFO;
+ logMsg = jsonLine.toPrettyString();
+ }
+ }
+ return Stream.of(logMessage(level, logMsg));
+ }
+ }
+
+ private static AirbyteMessage logMessage(Level level, String message) {
+ return new AirbyteMessage()
+ .withType(Type.LOG)
+ .withLog(new AirbyteLogMessage()
+ .withLevel(level)
+ .withMessage(message));
+ }
+
+ public static void main(String[] args) {
+ final NormalizationLogParser normalizationLogParser = new NormalizationLogParser();
+ final Stream airbyteMessageStream =
+ normalizationLogParser.create(new BufferedReader(new InputStreamReader(System.in, StandardCharsets.UTF_8)));
+ airbyteMessageStream.forEachOrdered(message -> System.out.println(Jsons.serialize(message)));
+
+ final List errors = normalizationLogParser.getDbtErrors();
+ final String dbtErrorStack = String.join("\n", errors);
+ if (!"".equals(dbtErrorStack)) {
+ final Map errorMap = SentryExceptionHelper.getUsefulErrorMessageAndTypeFromDbtError(dbtErrorStack);
+ String internalMessage = errorMap.get(ErrorMapKeys.ERROR_MAP_MESSAGE_KEY);
+ AirbyteMessage traceMessage = new AirbyteMessage()
+ .withType(Type.TRACE)
+ .withTrace(new AirbyteTraceMessage()
+ .withType(AirbyteTraceMessage.Type.ERROR)
+ .withEmittedAt((double) System.currentTimeMillis())
+ .withError(new AirbyteErrorTraceMessage()
+ .withFailureType(FailureType.SYSTEM_ERROR)
+ .withMessage("Normalization failed during the dbt run. This may indicate a problem with the data itself.")
+ .withStackTrace("AirbyteDbtError: \n" + dbtErrorStack)
+ .withInternalMessage(internalMessage)));
+ System.out.println(Jsons.serialize(traceMessage));
+ }
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/normalization/SentryExceptionHelper.java b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/normalization/SentryExceptionHelper.java
new file mode 100644
index 0000000000000..3f604e568e1cd
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/normalization/SentryExceptionHelper.java
@@ -0,0 +1,131 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.normalization;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This is copied out of platform
+ * (https://github.com/airbytehq/airbyte-platform/blob/main/airbyte-persistence/job-persistence/src/main/java/io/airbyte/persistence/job/errorreporter/SentryExceptionHelper.java#L257)
+ */
+public class SentryExceptionHelper {
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(SentryExceptionHelper.class);
+
+ /**
+ * Keys to known error types.
+ */
+ public enum ErrorMapKeys {
+ ERROR_MAP_MESSAGE_KEY,
+ ERROR_MAP_TYPE_KEY
+ }
+
+ public static Map getUsefulErrorMessageAndTypeFromDbtError(final String stacktrace) {
+ // the dbt 'stacktrace' is really just all the log messages at 'error' level, stuck together.
+ // therefore there is not a totally consistent structure to these,
+ // see the docs: https://docs.getdbt.com/guides/legacy/debugging-errors
+ // the logic below is built based on the ~450 unique dbt errors we encountered before this PR
+ // and is a best effort to isolate the useful part of the error logs for debugging and grouping
+ // and bring some semblance of exception 'types' to differentiate between errors.
+ final Map errorMessageAndType = new HashMap<>();
+ final String[] stacktraceLines = stacktrace.split("\n");
+
+ boolean defaultNextLine = false;
+ // TODO: this whole code block is quite ugh, commented to try and make each part clear but could be
+ // much more readable.
+ mainLoop: for (int i = 0; i < stacktraceLines.length; i++) {
+ // This order is important due to how these errors can co-occur.
+ // This order attempts to keep error definitions consistent based on our observations of possible
+ // dbt error structures.
+ try {
+ // Database Errors
+ if (stacktraceLines[i].contains("Database Error in model")) {
+ // Database Error : SQL compilation error
+ if (stacktraceLines[i + 1].contains("SQL compilation error")) {
+ errorMessageAndType.put(ErrorMapKeys.ERROR_MAP_MESSAGE_KEY,
+ String.format("%s %s", stacktraceLines[i + 1].trim(), stacktraceLines[i + 2].trim()));
+ errorMessageAndType.put(ErrorMapKeys.ERROR_MAP_TYPE_KEY, "DbtDatabaseSQLCompilationError");
+ break;
+ // Database Error: Invalid input
+ } else if (stacktraceLines[i + 1].contains("Invalid input")) {
+ for (final String followingLine : Arrays.copyOfRange(stacktraceLines, i + 1, stacktraceLines.length)) {
+ if (followingLine.trim().startsWith("context:")) {
+ errorMessageAndType.put(ErrorMapKeys.ERROR_MAP_MESSAGE_KEY,
+ String.format("%s\n%s", stacktraceLines[i + 1].trim(), followingLine.trim()));
+ errorMessageAndType.put(ErrorMapKeys.ERROR_MAP_TYPE_KEY, "DbtDatabaseInvalidInputError");
+ break mainLoop;
+ }
+ }
+ // Database Error: Syntax error
+ } else if (stacktraceLines[i + 1].contains("syntax error at or near \"")) {
+ errorMessageAndType.put(ErrorMapKeys.ERROR_MAP_MESSAGE_KEY,
+ String.format("%s\n%s", stacktraceLines[i + 1].trim(), stacktraceLines[i + 2].trim()));
+ errorMessageAndType.put(ErrorMapKeys.ERROR_MAP_TYPE_KEY, "DbtDatabaseSyntaxError");
+ break;
+ // Database Error: default
+ } else {
+ errorMessageAndType.put(ErrorMapKeys.ERROR_MAP_TYPE_KEY, "DbtDatabaseError");
+ defaultNextLine = true;
+ }
+ // Unhandled Error
+ } else if (stacktraceLines[i].contains("Unhandled error while executing model")) {
+ errorMessageAndType.put(ErrorMapKeys.ERROR_MAP_TYPE_KEY, "DbtUnhandledError");
+ defaultNextLine = true;
+ // Compilation Errors
+ } else if (stacktraceLines[i].contains("Compilation Error")) {
+ // Compilation Error: Ambiguous Relation
+ if (stacktraceLines[i + 1].contains("When searching for a relation, dbt found an approximate match.")) {
+ errorMessageAndType.put(ErrorMapKeys.ERROR_MAP_MESSAGE_KEY,
+ String.format("%s %s", stacktraceLines[i + 1].trim(), stacktraceLines[i + 2].trim()));
+ errorMessageAndType.put(ErrorMapKeys.ERROR_MAP_TYPE_KEY, "DbtCompilationAmbiguousRelationError");
+ break;
+ // Compilation Error: default
+ } else {
+ errorMessageAndType.put(ErrorMapKeys.ERROR_MAP_TYPE_KEY, "DbtCompilationError");
+ defaultNextLine = true;
+ }
+ // Runtime Errors
+ } else if (stacktraceLines[i].contains("Runtime Error")) {
+ // Runtime Error: Database error
+ for (final String followingLine : Arrays.copyOfRange(stacktraceLines, i + 1, stacktraceLines.length)) {
+ if ("Database Error".equals(followingLine.trim())) {
+ errorMessageAndType.put(ErrorMapKeys.ERROR_MAP_MESSAGE_KEY,
+ String.format("%s", stacktraceLines[Arrays.stream(stacktraceLines).toList().indexOf(followingLine) + 1].trim()));
+ errorMessageAndType.put(ErrorMapKeys.ERROR_MAP_TYPE_KEY, "DbtRuntimeDatabaseError");
+ break mainLoop;
+ }
+ }
+ // Runtime Error: default
+ errorMessageAndType.put(ErrorMapKeys.ERROR_MAP_TYPE_KEY, "DbtRuntimeError");
+ defaultNextLine = true;
+ // Database Error: formatted differently, catch last to avoid counting other types of errors as
+ // Database Error
+ } else if ("Database Error".equals(stacktraceLines[i].trim())) {
+ errorMessageAndType.put(ErrorMapKeys.ERROR_MAP_TYPE_KEY, "DbtDatabaseError");
+ defaultNextLine = true;
+ }
+ // handle the default case without repeating code
+ if (defaultNextLine) {
+ errorMessageAndType.put(ErrorMapKeys.ERROR_MAP_MESSAGE_KEY, stacktraceLines[i + 1].trim());
+ break;
+ }
+ } catch (final ArrayIndexOutOfBoundsException e) {
+ // this means our logic is slightly off, our assumption of where error lines are is incorrect
+ LOGGER.warn("Failed trying to parse useful error message out of dbt error, defaulting to full stacktrace");
+ }
+ }
+ if (errorMessageAndType.isEmpty()) {
+ // For anything we haven't caught, just return full stacktrace
+ errorMessageAndType.put(ErrorMapKeys.ERROR_MAP_MESSAGE_KEY, stacktrace);
+ errorMessageAndType.put(ErrorMapKeys.ERROR_MAP_TYPE_KEY, "AirbyteDbtError");
+ }
+ return errorMessageAndType;
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/BaseSerializedBuffer.java b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/BaseSerializedBuffer.java
new file mode 100644
index 0000000000000..9d6ce6acc976e
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/BaseSerializedBuffer.java
@@ -0,0 +1,175 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.record_buffer;
+
+import com.google.common.io.CountingOutputStream;
+import io.airbyte.protocol.models.v0.AirbyteRecordMessage;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorOutputStream;
+import org.apache.commons.io.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Base implementation of a {@link SerializableBuffer}. It is composed of a {@link BufferStorage}
+ * where the actual data is being stored in a serialized format.
+ *
+ * Such data format is defined by concrete implementation inheriting from this base abstract class.
+ * To do so, necessary methods on handling "writer" methods should be defined. This writer would
+ * take care of converting {@link AirbyteRecordMessage} into the serialized form of the data such as
+ * it can be stored in the outputStream of the {@link BufferStorage}.
+ */
+public abstract class BaseSerializedBuffer implements SerializableBuffer {
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(BaseSerializedBuffer.class);
+ private static final String GZ_SUFFIX = ".gz";
+
+ private final BufferStorage bufferStorage;
+ private final CountingOutputStream byteCounter;
+
+ private boolean useCompression;
+ private GzipCompressorOutputStream compressedBuffer;
+ private InputStream inputStream;
+ private boolean isStarted;
+ private boolean isClosed;
+
+ protected BaseSerializedBuffer(final BufferStorage bufferStorage) throws Exception {
+ this.bufferStorage = bufferStorage;
+ byteCounter = new CountingOutputStream(bufferStorage.getOutputStream());
+ useCompression = true;
+ compressedBuffer = null;
+ inputStream = null;
+ isStarted = false;
+ isClosed = false;
+ }
+
+ /**
+ * Initializes the writer objects such that it can now write to the downstream @param outputStream
+ */
+ protected abstract void initWriter(OutputStream outputStream) throws Exception;
+
+ /**
+ * Transform the @param record into a serialized form of the data and writes it to the registered
+ * OutputStream provided when {@link BaseSerializedBuffer#initWriter} was called.
+ */
+ protected abstract void writeRecord(AirbyteRecordMessage record) throws IOException;
+
+ /**
+ * Stops the writer from receiving new data and prepares it for being finalized and converted into
+ * an InputStream to read from instead. This is used when flushing the buffer into some other
+ * destination.
+ */
+ protected abstract void flushWriter() throws IOException;
+
+ protected abstract void closeWriter() throws IOException;
+
+ public SerializableBuffer withCompression(final boolean useCompression) {
+ if (!isStarted) {
+ this.useCompression = useCompression;
+ return this;
+ }
+ throw new RuntimeException("Options should be configured before starting to write");
+ }
+
+ @Override
+ public long accept(final AirbyteRecordMessage record) throws Exception {
+ if (!isStarted) {
+ if (useCompression) {
+ compressedBuffer = new GzipCompressorOutputStream(byteCounter);
+ initWriter(compressedBuffer);
+ } else {
+ initWriter(byteCounter);
+ }
+ isStarted = true;
+ }
+ if (inputStream == null && !isClosed) {
+ final long startCount = byteCounter.getCount();
+ writeRecord(record);
+ return byteCounter.getCount() - startCount;
+ } else {
+ throw new IllegalCallerException("Buffer is already closed, it cannot accept more messages");
+ }
+ }
+
+ @Override
+ public String getFilename() throws IOException {
+ if (useCompression && !bufferStorage.getFilename().endsWith(GZ_SUFFIX)) {
+ return bufferStorage.getFilename() + GZ_SUFFIX;
+ }
+ return bufferStorage.getFilename();
+ }
+
+ @Override
+ public File getFile() throws IOException {
+ if (useCompression && !bufferStorage.getFilename().endsWith(GZ_SUFFIX)) {
+ if (bufferStorage.getFile().renameTo(new File(bufferStorage.getFilename() + GZ_SUFFIX))) {
+ LOGGER.info("Renaming compressed file to include .gz file extension");
+ }
+ }
+ return bufferStorage.getFile();
+ }
+
+ protected InputStream convertToInputStream() throws IOException {
+ return bufferStorage.convertToInputStream();
+ }
+
+ @Override
+ public InputStream getInputStream() {
+ return inputStream;
+ }
+
+ @Override
+ public void flush() throws IOException {
+ if (inputStream == null && !isClosed) {
+ flushWriter();
+ if (compressedBuffer != null) {
+ LOGGER.info("Wrapping up compression and write GZIP trailer data.");
+ compressedBuffer.flush();
+ compressedBuffer.close();
+ }
+ closeWriter();
+ bufferStorage.close();
+ inputStream = convertToInputStream();
+ LOGGER.info("Finished writing data to {} ({})", getFilename(), FileUtils.byteCountToDisplaySize(byteCounter.getCount()));
+ }
+ }
+
+ @Override
+ public long getByteCount() {
+ return byteCounter.getCount();
+ }
+
+ @Override
+ public void close() throws Exception {
+ if (!isClosed) {
+ // inputStream can be null if the accept method encounters
+ // an error before inputStream is initialized
+ if (inputStream != null) {
+ inputStream.close();
+ }
+ bufferStorage.deleteFile();
+ isClosed = true;
+ }
+ }
+
+ @Override
+ public long getMaxTotalBufferSizeInBytes() {
+ return bufferStorage.getMaxTotalBufferSizeInBytes();
+ }
+
+ @Override
+ public long getMaxPerStreamBufferSizeInBytes() {
+ return bufferStorage.getMaxPerStreamBufferSizeInBytes();
+ }
+
+ @Override
+ public int getMaxConcurrentStreamsInBuffer() {
+ return bufferStorage.getMaxConcurrentStreamsInBuffer();
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/BufferCreateFunction.java b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/BufferCreateFunction.java
new file mode 100644
index 0000000000000..bda03460ff0b2
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/BufferCreateFunction.java
@@ -0,0 +1,18 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.record_buffer;
+
+import io.airbyte.commons.functional.CheckedBiFunction;
+import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair;
+import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog;
+
+public interface BufferCreateFunction extends
+ CheckedBiFunction {
+
+ @Override
+ SerializableBuffer apply(AirbyteStreamNameNamespacePair stream, ConfiguredAirbyteCatalog configuredCatalog)
+ throws Exception;
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/BufferFlushType.java b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/BufferFlushType.java
new file mode 100644
index 0000000000000..3d2a85b77f968
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/BufferFlushType.java
@@ -0,0 +1,10 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.record_buffer;
+
+public enum BufferFlushType {
+ FLUSH_ALL,
+ FLUSH_SINGLE_STREAM
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/BufferStorage.java b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/BufferStorage.java
new file mode 100644
index 0000000000000..c77329cf41f43
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/BufferStorage.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.record_buffer;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This interface abstract the actual object that is used to store incoming data being buffered. It
+ * could be a file, in-memory or some other objects.
+ *
+ * However, in order to be used as part of the {@link SerializableBuffer}, this
+ * {@link BufferStorage} should implement some methods used to determine how to write into and read
+ * from the storage once we are done buffering
+ *
+ * Some easy methods for manipulating the storage viewed as a file or InputStream are therefore
+ * required.
+ *
+ * Depending on the implementation of the storage medium, it would also determine what storage
+ * limits are possible.
+ */
+public interface BufferStorage {
+
+ /**
+ * Builds a new outputStream on which to write the data for storage.
+ */
+ OutputStream getOutputStream() throws IOException;
+
+ String getFilename() throws IOException;
+
+ File getFile() throws IOException;
+
+ /**
+ * Once buffering has reached some limits, the storage stream should be turned into an InputStream.
+ * This method should assume we are not going to write to buffer anymore, and it is safe to convert
+ * to some other format to be read from now.
+ */
+ InputStream convertToInputStream() throws IOException;
+
+ void close() throws IOException;
+
+ /**
+ * Cleans-up any file that was produced in the process of buffering (if any were produced)
+ */
+ void deleteFile() throws IOException;
+
+ /*
+ * Depending on the implementation of the storage, methods below defined reasonable thresholds
+ * associated with using this kind of buffer storage.
+ *
+ * These could also be dynamically configured/tuned at runtime if needed (from user input for
+ * example?)
+ */
+
+ /**
+ * @return How much storage should be used overall by all buffers
+ */
+ long getMaxTotalBufferSizeInBytes();
+
+ /**
+ * @return How much storage should be used for a particular stream at a time before flushing it
+ */
+ long getMaxPerStreamBufferSizeInBytes();
+
+ /**
+ * @return How many concurrent buffers can be handled at once in parallel
+ */
+ int getMaxConcurrentStreamsInBuffer();
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/BufferingStrategy.java b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/BufferingStrategy.java
new file mode 100644
index 0000000000000..0763e6f5add1a
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/BufferingStrategy.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.record_buffer;
+
+import io.airbyte.protocol.models.v0.AirbyteMessage;
+import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair;
+import java.util.Optional;
+
+/**
+ * High-level interface used by
+ * {@link io.airbyte.integrations.destination.buffered_stream_consumer.BufferedStreamConsumer}
+ *
+ * A Record buffering strategy relies on the capacity available of underlying
+ * {@link SerializableBuffer} to determine what to do when consuming a new {@link AirbyteMessage}
+ * into the buffer. It also defines when to flush such buffers and how to empty them once they fill
+ * up.
+ *
+ */
+public interface BufferingStrategy extends AutoCloseable {
+
+ /**
+ * Add a new message to the buffer while consuming streams, also handles when a buffer flush when
+ * buffer has been filled
+ *
+ * @param stream stream associated with record
+ * @param message {@link AirbyteMessage} to be added to the buffer
+ * @return an optional value if a flushed occur with the respective flush type, otherwise an empty
+ * value means only a record was added
+ * @throws Exception throw on failure
+ */
+ Optional addRecord(AirbyteStreamNameNamespacePair stream, AirbyteMessage message) throws Exception;
+
+ /**
+ * Flush buffered messages in a buffer from a particular stream
+ */
+ void flushSingleBuffer(AirbyteStreamNameNamespacePair stream, SerializableBuffer buffer) throws Exception;
+
+ /**
+ * Flush all buffers that were buffering message data so far.
+ */
+ void flushAllBuffers() throws Exception;
+
+ /**
+ * Removes all stream buffers.
+ */
+ void clear() throws Exception;
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/FileBuffer.java b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/FileBuffer.java
new file mode 100644
index 0000000000000..029877629beff
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/FileBuffer.java
@@ -0,0 +1,128 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.record_buffer;
+
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.util.UUID;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FileBuffer implements BufferStorage {
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(FileBuffer.class);
+
+ // The per stream size limit is following recommendations from:
+ // https://docs.snowflake.com/en/user-guide/data-load-considerations-prepare.html#general-file-sizing-recommendations
+ // "To optimize the number of parallel operations for a load,
+ // we recommend aiming to produce data files roughly 100-250 MB (or larger) in size compressed."
+ public static final long MAX_PER_STREAM_BUFFER_SIZE_BYTES = 200 * 1024 * 1024; // 200 MB
+ /*
+ * Other than the per-file size limit, we also limit the total size (which would limit how many
+ * concurrent streams we can buffer simultaneously too) Since this class is storing data on disk,
+ * the buffer size limits below are tied to the necessary disk storage space.
+ */
+ public static final long MAX_TOTAL_BUFFER_SIZE_BYTES = 1024 * 1024 * 1024; // 1 GB
+ /*
+ * We limit number of stream being buffered simultaneously anyway (limit how many files are
+ * stored/open for writing)
+ *
+ * Note: This value can be tuned to increase performance with the tradeoff of increased memory usage
+ * (~31 MB per buffer). See {@link StreamTransferManager}
+ *
+ * For connections with interleaved data (e.g. Change Data Capture), having less buffers than the
+ * number of streams being synced will cause buffer thrashing where buffers will need to be flushed
+ * before another stream's buffer can be created. Increasing the default max will reduce likelihood
+ * of thrashing but not entirely eliminate unless number of buffers equals streams to be synced
+ */
+ public static final int DEFAULT_MAX_CONCURRENT_STREAM_IN_BUFFER = 10;
+ public static final String FILE_BUFFER_COUNT_KEY = "file_buffer_count";
+ // This max is subject to change as no proper load testing has been done to verify the side effects
+ public static final int MAX_CONCURRENT_STREAM_IN_BUFFER = 50;
+ /*
+ * Use this soft cap as a guidance for customers to not exceed the recommended number of buffers
+ * which is 1 GB (total buffer size) / 31 MB (rough size of each buffer) ~= 32 buffers
+ */
+ public static final int SOFT_CAP_CONCURRENT_STREAM_IN_BUFFER = 20;
+
+ private final String fileExtension;
+ private File tempFile;
+ private OutputStream outputStream;
+ private final int maxConcurrentStreams;
+
+ public FileBuffer(final String fileExtension) {
+ this.fileExtension = fileExtension;
+ this.maxConcurrentStreams = DEFAULT_MAX_CONCURRENT_STREAM_IN_BUFFER;
+ tempFile = null;
+ outputStream = null;
+ }
+
+ public FileBuffer(final String fileExtension, final int maxConcurrentStreams) {
+ this.fileExtension = fileExtension;
+ this.maxConcurrentStreams = maxConcurrentStreams;
+ tempFile = null;
+ outputStream = null;
+ }
+
+ @Override
+ public OutputStream getOutputStream() throws IOException {
+ if (outputStream == null || tempFile == null) {
+ tempFile = Files.createTempFile(UUID.randomUUID().toString(), fileExtension).toFile();
+ outputStream = new BufferedOutputStream(new FileOutputStream(tempFile));
+ }
+ return outputStream;
+ }
+
+ @Override
+ public String getFilename() throws IOException {
+ return getFile().getName();
+ }
+
+ @Override
+ public File getFile() throws IOException {
+ if (tempFile == null) {
+ getOutputStream();
+ }
+ return tempFile;
+ }
+
+ @Override
+ public InputStream convertToInputStream() throws IOException {
+ return new FileInputStream(getFile());
+ }
+
+ @Override
+ public void close() throws IOException {
+ outputStream.close();
+ }
+
+ @Override
+ public void deleteFile() throws IOException {
+ LOGGER.info("Deleting tempFile data {}", getFilename());
+ Files.deleteIfExists(getFile().toPath());
+ }
+
+ @Override
+ public long getMaxTotalBufferSizeInBytes() {
+ return MAX_TOTAL_BUFFER_SIZE_BYTES;
+ }
+
+ @Override
+ public long getMaxPerStreamBufferSizeInBytes() {
+ return MAX_PER_STREAM_BUFFER_SIZE_BYTES;
+ }
+
+ @Override
+ public int getMaxConcurrentStreamsInBuffer() {
+ return maxConcurrentStreams;
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/FlushBufferFunction.java b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/FlushBufferFunction.java
new file mode 100644
index 0000000000000..be43b75c55916
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/FlushBufferFunction.java
@@ -0,0 +1,15 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.record_buffer;
+
+import io.airbyte.commons.functional.CheckedBiConsumer;
+import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair;
+
+public interface FlushBufferFunction extends CheckedBiConsumer {
+
+ @Override
+ void accept(AirbyteStreamNameNamespacePair stream, SerializableBuffer buffer) throws Exception;
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/InMemoryBuffer.java b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/InMemoryBuffer.java
new file mode 100644
index 0000000000000..d94a73dfd07e5
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/InMemoryBuffer.java
@@ -0,0 +1,104 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.record_buffer;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.util.UUID;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Instead of storing buffered data on disk like the {@link FileBuffer}, this {@link BufferStorage}
+ * accumulates message data in-memory instead. Thus, a bigger heap size would be required.
+ */
+public class InMemoryBuffer implements BufferStorage {
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(InMemoryBuffer.class);
+
+ // The per stream size limit is following recommendations from:
+ // https://docs.snowflake.com/en/user-guide/data-load-considerations-prepare.html#general-file-sizing-recommendations
+ // "To optimize the number of parallel operations for a load,
+ // we recommend aiming to produce data files roughly 100-250 MB (or larger) in size compressed."
+ public static final long MAX_PER_STREAM_BUFFER_SIZE_BYTES = 200 * 1024 * 1024; // 200 MB
+ // Other than the per-file size limit, we also limit the total size (which would limit how many
+ // concurrent streams we can buffer simultaneously too)
+ // Since this class is storing data in memory, the buffer size limits below are tied to the
+ // necessary RAM space.
+ public static final long MAX_TOTAL_BUFFER_SIZE_BYTES = 1024 * 1024 * 1024; // 1 GB
+ // we limit number of stream being buffered simultaneously anyway
+ public static final int MAX_CONCURRENT_STREAM_IN_BUFFER = 100;
+
+ private final String fileExtension;
+ private final ByteArrayOutputStream byteBuffer = new ByteArrayOutputStream();
+ private File tempFile;
+ private String filename;
+
+ public InMemoryBuffer(final String fileExtension) {
+ this.fileExtension = fileExtension;
+ tempFile = null;
+ filename = null;
+ }
+
+ @Override
+ public OutputStream getOutputStream() {
+ return byteBuffer;
+ }
+
+ @Override
+ public String getFilename() {
+ if (filename == null) {
+ filename = UUID.randomUUID().toString();
+ }
+ return filename;
+ }
+
+ @Override
+ public File getFile() throws IOException {
+ if (tempFile == null) {
+ tempFile = Files.createTempFile(getFilename(), fileExtension).toFile();
+ }
+ return tempFile;
+ }
+
+ @Override
+ public InputStream convertToInputStream() {
+ return new ByteArrayInputStream(byteBuffer.toByteArray());
+ }
+
+ @Override
+ public void close() throws IOException {
+ byteBuffer.close();
+ }
+
+ @Override
+ public void deleteFile() throws IOException {
+ if (tempFile != null) {
+ LOGGER.info("Deleting tempFile data {}", getFilename());
+ Files.deleteIfExists(tempFile.toPath());
+ }
+ }
+
+ @Override
+ public long getMaxTotalBufferSizeInBytes() {
+ return MAX_TOTAL_BUFFER_SIZE_BYTES;
+ }
+
+ @Override
+ public long getMaxPerStreamBufferSizeInBytes() {
+ return MAX_PER_STREAM_BUFFER_SIZE_BYTES;
+ }
+
+ @Override
+ public int getMaxConcurrentStreamsInBuffer() {
+ return MAX_CONCURRENT_STREAM_IN_BUFFER;
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/InMemoryRecordBufferingStrategy.java b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/InMemoryRecordBufferingStrategy.java
new file mode 100644
index 0000000000000..d16ef8dca1e1b
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/InMemoryRecordBufferingStrategy.java
@@ -0,0 +1,106 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.record_buffer;
+
+import io.airbyte.integrations.destination.buffered_stream_consumer.CheckAndRemoveRecordWriter;
+import io.airbyte.integrations.destination.buffered_stream_consumer.RecordSizeEstimator;
+import io.airbyte.integrations.destination.buffered_stream_consumer.RecordWriter;
+import io.airbyte.protocol.models.v0.AirbyteMessage;
+import io.airbyte.protocol.models.v0.AirbyteRecordMessage;
+import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.commons.io.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This is the default implementation of a {@link BufferStorage} to be backward compatible. Data is
+ * being buffered in a {@link List} as they are being consumed.
+ *
+ * This should be deprecated as we slowly move towards using {@link SerializedBufferingStrategy}
+ * instead.
+ */
+public class InMemoryRecordBufferingStrategy implements BufferingStrategy {
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(InMemoryRecordBufferingStrategy.class);
+
+ private Map> streamBuffer = new HashMap<>();
+ private final RecordWriter recordWriter;
+ private final CheckAndRemoveRecordWriter checkAndRemoveRecordWriter;
+ private String fileName;
+
+ private final RecordSizeEstimator recordSizeEstimator;
+ private final long maxQueueSizeInBytes;
+ private long bufferSizeInBytes;
+
+ public InMemoryRecordBufferingStrategy(final RecordWriter recordWriter,
+ final long maxQueueSizeInBytes) {
+ this(recordWriter, null, maxQueueSizeInBytes);
+ }
+
+ public InMemoryRecordBufferingStrategy(final RecordWriter recordWriter,
+ final CheckAndRemoveRecordWriter checkAndRemoveRecordWriter,
+ final long maxQueueSizeInBytes) {
+ this.recordWriter = recordWriter;
+ this.checkAndRemoveRecordWriter = checkAndRemoveRecordWriter;
+
+ this.maxQueueSizeInBytes = maxQueueSizeInBytes;
+ this.bufferSizeInBytes = 0;
+ this.recordSizeEstimator = new RecordSizeEstimator();
+ }
+
+ @Override
+ public Optional addRecord(final AirbyteStreamNameNamespacePair stream, final AirbyteMessage message) throws Exception {
+ Optional flushed = Optional.empty();
+
+ final long messageSizeInBytes = recordSizeEstimator.getEstimatedByteSize(message.getRecord());
+ if (bufferSizeInBytes + messageSizeInBytes > maxQueueSizeInBytes) {
+ flushAllBuffers();
+ flushed = Optional.of(BufferFlushType.FLUSH_ALL);
+ }
+
+ final List bufferedRecords = streamBuffer.computeIfAbsent(stream, k -> new ArrayList<>());
+ bufferedRecords.add(message.getRecord());
+ bufferSizeInBytes += messageSizeInBytes;
+
+ return flushed;
+ }
+
+ @Override
+ public void flushSingleBuffer(final AirbyteStreamNameNamespacePair stream, final SerializableBuffer buffer) throws Exception {
+ LOGGER.info("Flushing single stream {}: {} records", stream.getName(), streamBuffer.get(stream).size());
+ recordWriter.accept(stream, streamBuffer.get(stream));
+ LOGGER.info("Flushing completed for {}", stream.getName());
+ }
+
+ @Override
+ public void flushAllBuffers() throws Exception {
+ for (final Map.Entry> entry : streamBuffer.entrySet()) {
+ LOGGER.info("Flushing {}: {} records ({})", entry.getKey().getName(), entry.getValue().size(),
+ FileUtils.byteCountToDisplaySize(bufferSizeInBytes));
+ recordWriter.accept(entry.getKey(), entry.getValue());
+ if (checkAndRemoveRecordWriter != null) {
+ fileName = checkAndRemoveRecordWriter.apply(entry.getKey(), fileName);
+ }
+ LOGGER.info("Flushing completed for {}", entry.getKey().getName());
+ }
+ close();
+ clear();
+ bufferSizeInBytes = 0;
+ }
+
+ @Override
+ public void clear() {
+ streamBuffer = new HashMap<>();
+ }
+
+ @Override
+ public void close() throws Exception {}
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/SerializableBuffer.java b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/SerializableBuffer.java
new file mode 100644
index 0000000000000..2762ab055ecb9
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/SerializableBuffer.java
@@ -0,0 +1,90 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.record_buffer;
+
+import io.airbyte.protocol.models.v0.AirbyteRecordMessage;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * A {@link SerializableBuffer} is designed to be used as part of a
+ * {@link SerializedBufferingStrategy}.
+ *
+ *
+ * It encapsulates the actual implementation of a buffer: both the medium storage (usually defined
+ * as part of {@link BufferStorage}. and the format of the serialized data when it is written to the
+ * buffer.
+ *
+ *
+ *
+ * A {@link BaseSerializedBuffer} class is provided, and should be the expected class to derive from
+ * when implementing a new format of buffer. The storage aspects are normally provided through
+ * composition of {@link BufferStorage}.
+ *
+ *
+ */
+public interface SerializableBuffer extends AutoCloseable {
+
+ /**
+ * Adds a {@link AirbyteRecordMessage} to the buffer and returns the size of the message in bytes
+ *
+ * @param record {@link AirbyteRecordMessage} to be added to buffer
+ * @return number of bytes written to the buffer
+ */
+ long accept(AirbyteRecordMessage record) throws Exception;
+
+ /**
+ * Flush a buffer implementation.
+ */
+ void flush() throws Exception;
+
+ /**
+ * The buffer implementation should be keeping track of how many bytes it accumulated so far. If any
+ * flush events were triggered, the amount of bytes accumulated would also have been decreased
+ * accordingly. This method @return such statistics.
+ */
+ long getByteCount();
+
+ /**
+ * @return the filename representation of this buffer.
+ */
+ String getFilename() throws IOException;
+
+ /**
+ * @return a temporary representation as a file of this buffer.
+ */
+ File getFile() throws IOException;
+
+ /**
+ * @return the InputStream to read data back from this buffer once it is done adding messages to it.
+ */
+ InputStream getInputStream() throws FileNotFoundException;
+
+ /*
+ * Depending on the implementation of the storage, methods below defined reasonable thresholds
+ * associated with using this kind of buffer implementation.
+ *
+ * These could also be dynamically configured/tuned at runtime if needed (from user input for
+ * example?)
+ */
+
+ /**
+ * @return How much storage should be used overall by all buffers
+ */
+ long getMaxTotalBufferSizeInBytes();
+
+ /**
+ * @return How much storage should be used for a particular stream at a time before flushing it
+ */
+ long getMaxPerStreamBufferSizeInBytes();
+
+ /**
+ * @return How many concurrent buffers can be handled at once in parallel
+ */
+ int getMaxConcurrentStreamsInBuffer();
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/SerializedBufferingStrategy.java b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/SerializedBufferingStrategy.java
new file mode 100644
index 0000000000000..d69451440e031
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/destination/record_buffer/SerializedBufferingStrategy.java
@@ -0,0 +1,171 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.record_buffer;
+
+import io.airbyte.commons.string.Strings;
+import io.airbyte.protocol.models.v0.AirbyteMessage;
+import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair;
+import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Optional;
+import org.apache.commons.io.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Buffering Strategy used to convert {@link io.airbyte.protocol.models.AirbyteRecordMessage} into a
+ * stream of bytes to more readily save and transmit information
+ *
+ *
+ * This class is meant to be used in conjunction with {@link SerializableBuffer}
+ *
+ */
+public class SerializedBufferingStrategy implements BufferingStrategy {
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(SerializedBufferingStrategy.class);
+
+ private final BufferCreateFunction onCreateBuffer;
+ private final FlushBufferFunction onStreamFlush;
+
+ private Map allBuffers = new HashMap<>();
+ private long totalBufferSizeInBytes;
+ private final ConfiguredAirbyteCatalog catalog;
+
+ /**
+ * Creates instance of Serialized Buffering Strategy used to handle the logic of flushing buffer
+ * with an associated buffer type
+ *
+ * @param onCreateBuffer type of buffer used upon creation
+ * @param catalog collection of {@link io.airbyte.protocol.models.ConfiguredAirbyteStream}
+ * @param onStreamFlush buffer flush logic used throughout the streaming of messages
+ */
+ public SerializedBufferingStrategy(final BufferCreateFunction onCreateBuffer,
+ final ConfiguredAirbyteCatalog catalog,
+ final FlushBufferFunction onStreamFlush) {
+ this.onCreateBuffer = onCreateBuffer;
+ this.catalog = catalog;
+ this.onStreamFlush = onStreamFlush;
+ this.totalBufferSizeInBytes = 0;
+ }
+
+ /**
+ * Handles both adding records and when buffer is full to also flush
+ *
+ * @param stream stream associated with record
+ * @param message {@link AirbyteMessage} to buffer
+ * @return Optional which contains a {@link BufferFlushType} if a flush occurred, otherwise empty)
+ * @throws Exception
+ */
+ @Override
+ public Optional addRecord(final AirbyteStreamNameNamespacePair stream, final AirbyteMessage message) throws Exception {
+ Optional flushed = Optional.empty();
+
+ final SerializableBuffer buffer = getOrCreateBuffer(stream);
+ if (buffer == null) {
+ throw new RuntimeException(String.format("Failed to create/get buffer for stream %s.%s", stream.getNamespace(), stream.getName()));
+ }
+
+ final long actualMessageSizeInBytes = buffer.accept(message.getRecord());
+ totalBufferSizeInBytes += actualMessageSizeInBytes;
+ // Flushes buffer when either the buffer was completely filled or only a single stream was filled
+ if (totalBufferSizeInBytes >= buffer.getMaxTotalBufferSizeInBytes()
+ || allBuffers.size() >= buffer.getMaxConcurrentStreamsInBuffer()) {
+ flushAllBuffers();
+ flushed = Optional.of(BufferFlushType.FLUSH_ALL);
+ } else if (buffer.getByteCount() >= buffer.getMaxPerStreamBufferSizeInBytes()) {
+ flushSingleBuffer(stream, buffer);
+ /*
+ * Note: This branch is needed to indicate to the {@link DefaultDestStateLifeCycleManager} that an
+ * individual stream was flushed, there is no guarantee that it will flush records in the same order
+ * that state messages were received. The outcome here is that records get flushed but our updating
+ * of which state messages have been flushed falls behind.
+ *
+ * This is not ideal from a checkpoint point of view, because it means in the case where there is a
+ * failure, we will not be able to report that those records that were flushed and committed were
+ * committed because there corresponding state messages weren't marked as flushed. Thus, it weakens
+ * checkpointing, but it does not cause a correctness issue.
+ *
+ * In non-failure cases, using this conditional branch relies on the state messages getting flushed
+ * by some other means. That can be caused by the previous branch in this conditional. It is
+ * guaranteed by the fact that we always flush all state messages at the end of a sync.
+ */
+ flushed = Optional.of(BufferFlushType.FLUSH_SINGLE_STREAM);
+ }
+ return flushed;
+ }
+
+ /**
+ * Creates a new buffer for each stream if buffers do not already exist, else return already
+ * computed buffer
+ */
+ private SerializableBuffer getOrCreateBuffer(final AirbyteStreamNameNamespacePair stream) {
+ return allBuffers.computeIfAbsent(stream, k -> {
+ LOGGER.info("Starting a new buffer for stream {} (current state: {} in {} buffers)",
+ stream.getName(),
+ FileUtils.byteCountToDisplaySize(totalBufferSizeInBytes),
+ allBuffers.size());
+ try {
+ return onCreateBuffer.apply(stream, catalog);
+ } catch (final Exception e) {
+ LOGGER.error("Failed to create a new buffer for stream {}", stream.getName(), e);
+ throw new RuntimeException(e);
+ }
+ });
+ }
+
+ @Override
+ public void flushSingleBuffer(final AirbyteStreamNameNamespacePair stream, final SerializableBuffer buffer) throws Exception {
+ LOGGER.info("Flushing buffer of stream {} ({})", stream.getName(), FileUtils.byteCountToDisplaySize(buffer.getByteCount()));
+ onStreamFlush.accept(stream, buffer);
+ totalBufferSizeInBytes -= buffer.getByteCount();
+ allBuffers.remove(stream);
+ LOGGER.info("Flushing completed for {}", stream.getName());
+ }
+
+ @Override
+ public void flushAllBuffers() throws Exception {
+ LOGGER.info("Flushing all {} current buffers ({} in total)", allBuffers.size(), FileUtils.byteCountToDisplaySize(totalBufferSizeInBytes));
+ for (final Entry entry : allBuffers.entrySet()) {
+ final AirbyteStreamNameNamespacePair stream = entry.getKey();
+ final SerializableBuffer buffer = entry.getValue();
+ LOGGER.info("Flushing buffer of stream {} ({})", stream.getName(), FileUtils.byteCountToDisplaySize(buffer.getByteCount()));
+ onStreamFlush.accept(stream, buffer);
+ LOGGER.info("Flushing completed for {}", stream.getName());
+ }
+ close();
+ clear();
+ totalBufferSizeInBytes = 0;
+ }
+
+ @Override
+ public void clear() throws Exception {
+ LOGGER.debug("Reset all buffers");
+ allBuffers = new HashMap<>();
+ }
+
+ @Override
+ public void close() throws Exception {
+ final List exceptionsThrown = new ArrayList<>();
+ for (final Entry entry : allBuffers.entrySet()) {
+ try {
+ final AirbyteStreamNameNamespacePair stream = entry.getKey();
+ LOGGER.info("Closing buffer for stream {}", stream.getName());
+ final SerializableBuffer buffer = entry.getValue();
+ buffer.close();
+ } catch (final Exception e) {
+ exceptionsThrown.add(e);
+ LOGGER.error("Exception while closing stream buffer", e);
+ }
+ }
+ if (!exceptionsThrown.isEmpty()) {
+ throw new RuntimeException(String.format("Exceptions thrown while closing buffers: %s", Strings.join(exceptionsThrown, "\n")));
+ }
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/util/ApmTraceUtils.java b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/util/ApmTraceUtils.java
new file mode 100644
index 0000000000000..555c7d4dd6c86
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/util/ApmTraceUtils.java
@@ -0,0 +1,150 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.util;
+
+import datadog.trace.api.DDTags;
+import datadog.trace.api.interceptor.MutableSpan;
+import io.opentracing.Span;
+import io.opentracing.log.Fields;
+import io.opentracing.tag.Tags;
+import io.opentracing.util.GlobalTracer;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.Map;
+
+/**
+ * Collection of utility methods to help with performance tracing.
+ */
+public class ApmTraceUtils {
+
+ /**
+ * String format for the name of tags added to spans.
+ */
+ public static final String TAG_FORMAT = "airbyte.%s.%s";
+
+ /**
+ * Standard prefix for tags added to spans.
+ */
+ public static final String TAG_PREFIX = "metadata";
+
+ /**
+ * Adds all the provided tags to the currently active span, if one exists.
+ * All tags added via this method will use the default {@link #TAG_PREFIX} namespace.
+ *
+ * @param tags A map of tags to be added to the currently active span.
+ */
+ public static void addTagsToTrace(final Map tags) {
+ addTagsToTrace(tags, TAG_PREFIX);
+ }
+
+ /**
+ * Adds all provided tags to the currently active span, if one exists, under the provided tag name
+ * namespace.
+ *
+ * @param tags A map of tags to be added to the currently active span.
+ * @param tagPrefix The prefix to be added to each custom tag name.
+ */
+ public static void addTagsToTrace(final Map tags, final String tagPrefix) {
+ addTagsToTrace(GlobalTracer.get().activeSpan(), tags, tagPrefix);
+ }
+
+ /**
+ * Adds all the provided tags to the provided span, if one exists.
+ *
+ * @param span The {@link Span} that will be associated with the tags.
+ * @param tags A map of tags to be added to the currently active span.
+ * @param tagPrefix The prefix to be added to each custom tag name.
+ */
+ public static void addTagsToTrace(final Span span, final Map tags, final String tagPrefix) {
+ if (span != null) {
+ tags.entrySet().forEach(entry -> {
+ span.setTag(formatTag(entry.getKey(), tagPrefix), entry.getValue().toString());
+ });
+ }
+ }
+
+ /**
+ * Adds an exception to the currently active span, if one exists.
+ *
+ * @param t The {@link Throwable} to be added to the currently active span.
+ */
+ public static void addExceptionToTrace(final Throwable t) {
+ addExceptionToTrace(GlobalTracer.get().activeSpan(), t);
+ }
+
+ /**
+ * Adds an exception to the provided span, if one exists.
+ *
+ * @param span The {@link Span} that will be associated with the exception.
+ * @param t The {@link Throwable} to be added to the provided span.
+ */
+ public static void addExceptionToTrace(final Span span, final Throwable t) {
+ if (span != null) {
+ span.setTag(Tags.ERROR, true);
+ span.log(Map.of(Fields.ERROR_OBJECT, t));
+ }
+ }
+
+ /**
+ * Adds all the provided tags to the root span.
+ *
+ * @param tags A map of tags to be added to the root span.
+ */
+ public static void addTagsToRootSpan(final Map tags) {
+ final Span activeSpan = GlobalTracer.get().activeSpan();
+ if (activeSpan instanceof MutableSpan) {
+ final MutableSpan localRootSpan = ((MutableSpan) activeSpan).getLocalRootSpan();
+ tags.entrySet().forEach(entry -> {
+ localRootSpan.setTag(formatTag(entry.getKey(), TAG_PREFIX), entry.getValue().toString());
+ });
+ }
+ }
+
+ /**
+ * Adds an exception to the root span, if an active one exists.
+ *
+ * @param t The {@link Throwable} to be added to the provided span.
+ */
+ public static void recordErrorOnRootSpan(final Throwable t) {
+ final Span activeSpan = GlobalTracer.get().activeSpan();
+ if (activeSpan != null) {
+ activeSpan.setTag(Tags.ERROR, true);
+ activeSpan.log(Map.of(Fields.ERROR_OBJECT, t));
+ }
+ if (activeSpan instanceof MutableSpan) {
+ final MutableSpan localRootSpan = ((MutableSpan) activeSpan).getLocalRootSpan();
+ localRootSpan.setError(true);
+ localRootSpan.setTag(DDTags.ERROR_MSG, t.getMessage());
+ localRootSpan.setTag(DDTags.ERROR_TYPE, t.getClass().getName());
+ final StringWriter errorString = new StringWriter();
+ t.printStackTrace(new PrintWriter(errorString));
+ localRootSpan.setTag(DDTags.ERROR_STACK, errorString.toString());
+ }
+ }
+
+ /**
+ * Formats the tag key using {@link #TAG_FORMAT} provided by this utility, using the default tag
+ * prefix {@link #TAG_PREFIX}.
+ *
+ * @param tagKey The tag key to format.
+ * @return The formatted tag key.
+ */
+ public static String formatTag(final String tagKey) {
+ return formatTag(tagKey, TAG_PREFIX);
+ }
+
+ /**
+ * Formats the tag key using {@link #TAG_FORMAT} provided by this utility with the provided tag
+ * prefix.
+ *
+ * @param tagKey The tag key to format.
+ * @param tagPrefix The prefix to be added to each custom tag name.
+ * @return The formatted tag key.
+ */
+ public static String formatTag(final String tagKey, final String tagPrefix) {
+ return String.format(TAG_FORMAT, tagPrefix, tagKey);
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/util/ConnectorExceptionUtil.java b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/util/ConnectorExceptionUtil.java
new file mode 100644
index 0000000000000..65d6428fcdc31
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/util/ConnectorExceptionUtil.java
@@ -0,0 +1,91 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.util;
+
+import com.google.common.collect.ImmutableList;
+import io.airbyte.commons.exceptions.ConfigErrorException;
+import io.airbyte.commons.exceptions.ConnectionErrorException;
+import io.airbyte.integrations.base.errors.messages.ErrorMessage;
+import java.sql.SQLException;
+import java.sql.SQLSyntaxErrorException;
+import java.util.List;
+import java.util.Locale;
+import java.util.function.Predicate;
+
+/**
+ * Utility class defining methods for handling configuration exceptions in connectors.
+ */
+public class ConnectorExceptionUtil {
+
+ public static final String COMMON_EXCEPTION_MESSAGE_TEMPLATE = "Could not connect with provided configuration. Error: %s";
+ static final String RECOVERY_CONNECTION_ERROR_MESSAGE =
+ "We're having issues syncing from a Postgres replica that is configured as a hot standby server. " +
+ "Please see https://docs.airbyte.com/integrations/sources/postgres/#sync-data-from-postgres-hot-standby-server for options and workarounds";
+
+ public static final List HTTP_AUTHENTICATION_ERROR_CODES = ImmutableList.of(401, 403);
+ private static final List> configErrorPredicates =
+ List.of(getConfigErrorPredicate(), getConnectionErrorPredicate(),
+ isRecoveryConnectionExceptionPredicate(), isUnknownColumnInFieldListException());
+
+ public static boolean isConfigError(final Throwable e) {
+ return configErrorPredicates.stream().anyMatch(predicate -> predicate.test(e));
+ }
+
+ public static String getDisplayMessage(final Throwable e) {
+ if (e instanceof ConfigErrorException) {
+ return ((ConfigErrorException) e).getDisplayMessage();
+ } else if (e instanceof ConnectionErrorException) {
+ final ConnectionErrorException connEx = (ConnectionErrorException) e;
+ return ErrorMessage.getErrorMessage(connEx.getStateCode(), connEx.getErrorCode(), connEx.getExceptionMessage(), connEx);
+ } else if (isRecoveryConnectionExceptionPredicate().test(e)) {
+ return RECOVERY_CONNECTION_ERROR_MESSAGE;
+ } else if (isUnknownColumnInFieldListException().test(e)) {
+ return e.getMessage();
+ } else {
+ return String.format(COMMON_EXCEPTION_MESSAGE_TEMPLATE, e.getMessage() != null ? e.getMessage() : "");
+ }
+ }
+
+ /**
+ * Returns the first instance of an exception associated with a configuration error (if it exists).
+ * Otherwise, the original exception is returned.
+ */
+ public static Throwable getRootConfigError(final Exception e) {
+ Throwable current = e;
+ while (current != null) {
+ if (ConnectorExceptionUtil.isConfigError(current)) {
+ return current;
+ } else {
+ current = current.getCause();
+ }
+ }
+ return e;
+ }
+
+ private static Predicate getConfigErrorPredicate() {
+ return e -> e instanceof ConfigErrorException;
+ }
+
+ private static Predicate getConnectionErrorPredicate() {
+ return e -> e instanceof ConnectionErrorException;
+ }
+
+ private static Predicate isRecoveryConnectionExceptionPredicate() {
+ return e -> e instanceof SQLException && e.getMessage()
+ .toLowerCase(Locale.ROOT)
+ .contains("due to conflict with recovery");
+ }
+
+ private static Predicate isUnknownColumnInFieldListException() {
+ return e -> e instanceof SQLSyntaxErrorException
+ && e.getMessage()
+ .toLowerCase(Locale.ROOT)
+ .contains("unknown column")
+ && e.getMessage()
+ .toLowerCase(Locale.ROOT)
+ .contains("in 'field list'");
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/util/HostPortResolver.java b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/util/HostPortResolver.java
new file mode 100644
index 0000000000000..89eaa857a916e
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/util/HostPortResolver.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.util;
+
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.Objects;
+import org.testcontainers.containers.GenericContainer;
+
+public class HostPortResolver {
+
+ public static String resolveHost(GenericContainer container) {
+ return System.getProperty("os.name").toLowerCase().startsWith("mac")
+ ? getIpAddress(container)
+ : container.getHost();
+ }
+
+ public static int resolvePort(GenericContainer container) {
+ return System.getProperty("os.name").toLowerCase().startsWith("mac") ? (Integer) container.getExposedPorts().get(0)
+ : container.getFirstMappedPort();
+ }
+
+ public static String resolveIpAddress(GenericContainer container) {
+ return getIpAddress(container);
+ }
+
+ public static String encodeValue(final String value) {
+ if (value != null) {
+ return URLEncoder.encode(value, StandardCharsets.UTF_8);
+ }
+ return null;
+ }
+
+ private static String getIpAddress(GenericContainer container) {
+ return Objects.requireNonNull(container.getContainerInfo()
+ .getNetworkSettings()
+ .getNetworks()
+ .entrySet().stream()
+ .findFirst()
+ .get().getValue().getIpAddress());
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/util/PostgresSslConnectionUtils.java b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/util/PostgresSslConnectionUtils.java
new file mode 100644
index 0000000000000..d5ab7f8c7a70f
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/main/java/io/airbyte/integrations/util/PostgresSslConnectionUtils.java
@@ -0,0 +1,190 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.util;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PostgresSslConnectionUtils {
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(PostgresSslConnectionUtils.class);
+ private static final String CA_CERTIFICATE = "ca.crt";
+ private static final String CLIENT_CERTIFICATE = "client.crt";
+ private static final String CLIENT_KEY = "client.key";
+ private static final String CLIENT_ENCRYPTED_KEY = "client.pk8";
+
+ public static final String PARAM_MODE = "mode";
+ public static final String PARAM_SSL = "ssl";
+ public static final String PARAM_SSL_MODE = "ssl_mode";
+ public static final String PARAM_SSLMODE = "sslmode";
+ public static final String PARAM_CLIENT_KEY_PASSWORD = "client_key_password";
+ public static final String PARAM_CA_CERTIFICATE = "ca_certificate";
+ public static final String PARAM_CLIENT_CERTIFICATE = "client_certificate";
+ public static final String PARAM_CLIENT_KEY = "client_key";
+
+ public static final String VERIFY_CA = "verify-ca";
+ public static final String VERIFY_FULL = "verify-full";
+ public static final String DISABLE = "disable";
+ public static final String TRUE_STRING_VALUE = "true";
+ public static final String ENCRYPT_FILE_NAME = "encrypt";
+ public static final String FACTORY_VALUE = "org.postgresql.ssl.DefaultJavaSSLFactory";
+
+ public static Map obtainConnectionOptions(final JsonNode encryption) {
+ final Map additionalParameters = new HashMap<>();
+ if (!encryption.isNull()) {
+ final var method = encryption.get(PARAM_MODE).asText();
+ var keyStorePassword = checkOrCreatePassword(encryption);
+ switch (method) {
+ case VERIFY_CA -> {
+ additionalParameters.putAll(obtainConnectionCaOptions(encryption, method, keyStorePassword));
+ }
+ case VERIFY_FULL -> {
+ additionalParameters.putAll(obtainConnectionFullOptions(encryption, method, keyStorePassword));
+ }
+ default -> {
+ additionalParameters.put(PARAM_SSL, TRUE_STRING_VALUE);
+ additionalParameters.put(PARAM_SSLMODE, method);
+ }
+ }
+ }
+ return additionalParameters;
+ }
+
+ private static String checkOrCreatePassword(final JsonNode encryption) {
+ String sslPassword = encryption.has(PARAM_CLIENT_KEY_PASSWORD) ? encryption.get(PARAM_CLIENT_KEY_PASSWORD).asText() : "";
+ var keyStorePassword = RandomStringUtils.randomAlphanumeric(10);
+ if (sslPassword.isEmpty()) {
+ var file = new File(ENCRYPT_FILE_NAME);
+ if (file.exists()) {
+ keyStorePassword = readFile(file);
+ } else {
+ try {
+ createCertificateFile(ENCRYPT_FILE_NAME, keyStorePassword);
+ } catch (final IOException e) {
+ throw new RuntimeException("Failed to create encryption file ");
+ }
+ }
+ } else {
+ keyStorePassword = sslPassword;
+ }
+ return keyStorePassword;
+ }
+
+ private static String readFile(final File file) {
+ try {
+ BufferedReader reader = new BufferedReader(new FileReader(file, StandardCharsets.UTF_8));
+ String currentLine = reader.readLine();
+ reader.close();
+ return currentLine;
+ } catch (final IOException e) {
+ throw new RuntimeException("Failed to read file with encryption");
+ }
+ }
+
+ private static Map obtainConnectionFullOptions(final JsonNode encryption,
+ final String method,
+ final String clientKeyPassword) {
+ final Map additionalParameters = new HashMap<>();
+ try {
+ convertAndImportFullCertificate(encryption.get(PARAM_CA_CERTIFICATE).asText(),
+ encryption.get(PARAM_CLIENT_CERTIFICATE).asText(), encryption.get(PARAM_CLIENT_KEY).asText(), clientKeyPassword);
+ } catch (final IOException | InterruptedException e) {
+ throw new RuntimeException("Failed to import certificate into Java Keystore");
+ }
+ additionalParameters.put("ssl", TRUE_STRING_VALUE);
+ additionalParameters.put("sslmode", method);
+ additionalParameters.put("sslrootcert", CA_CERTIFICATE);
+ additionalParameters.put("sslcert", CLIENT_CERTIFICATE);
+ additionalParameters.put("sslkey", CLIENT_ENCRYPTED_KEY);
+ additionalParameters.put("sslfactory", FACTORY_VALUE);
+ return additionalParameters;
+ }
+
+ private static Map obtainConnectionCaOptions(final JsonNode encryption,
+ final String method,
+ final String clientKeyPassword) {
+ final Map additionalParameters = new HashMap<>();
+ try {
+ convertAndImportCaCertificate(encryption.get(PARAM_CA_CERTIFICATE).asText(), clientKeyPassword);
+ } catch (final IOException | InterruptedException e) {
+ throw new RuntimeException("Failed to import certificate into Java Keystore");
+ }
+ additionalParameters.put("ssl", TRUE_STRING_VALUE);
+ additionalParameters.put("sslmode", method);
+ additionalParameters.put("sslrootcert", CA_CERTIFICATE);
+ additionalParameters.put("sslfactory", FACTORY_VALUE);
+ return additionalParameters;
+ }
+
+ private static void convertAndImportFullCertificate(final String caCertificate,
+ final String clientCertificate,
+ final String clientKey,
+ final String clientKeyPassword)
+ throws IOException, InterruptedException {
+ final Runtime run = Runtime.getRuntime();
+ createCaCertificate(caCertificate, clientKeyPassword, run);
+ createCertificateFile(CLIENT_CERTIFICATE, clientCertificate);
+ createCertificateFile(CLIENT_KEY, clientKey);
+ // add client certificate to the custom keystore
+ runProcess("keytool -alias client-certificate -keystore customkeystore"
+ + " -import -file " + CLIENT_CERTIFICATE + " -storepass " + clientKeyPassword + " -noprompt", run);
+ // convert client.key to client.pk8 based on the documentation
+ runProcess("openssl pkcs8 -topk8 -inform PEM -in " + CLIENT_KEY + " -outform DER -out "
+ + CLIENT_ENCRYPTED_KEY + " -nocrypt", run);
+ runProcess("rm " + CLIENT_KEY, run);
+
+ updateTrustStoreSystemProperty(clientKeyPassword);
+ }
+
+ private static void convertAndImportCaCertificate(final String caCertificate,
+ final String clientKeyPassword)
+ throws IOException, InterruptedException {
+ final Runtime run = Runtime.getRuntime();
+ createCaCertificate(caCertificate, clientKeyPassword, run);
+ updateTrustStoreSystemProperty(clientKeyPassword);
+ }
+
+ private static void createCaCertificate(final String caCertificate,
+ final String clientKeyPassword,
+ final Runtime run)
+ throws IOException, InterruptedException {
+ createCertificateFile(CA_CERTIFICATE, caCertificate);
+ // add CA certificate to the custom keystore
+ runProcess("keytool -import -alias rds-root -keystore customkeystore"
+ + " -file " + CA_CERTIFICATE + " -storepass " + clientKeyPassword + " -noprompt", run);
+ }
+
+ private static void updateTrustStoreSystemProperty(final String clientKeyPassword) {
+ String result = System.getProperty("user.dir") + "/customkeystore";
+ System.setProperty("javax.net.ssl.trustStore", result);
+ System.setProperty("javax.net.ssl.trustStorePassword", clientKeyPassword);
+ }
+
+ private static void createCertificateFile(String fileName, String fileValue) throws IOException {
+ try (final PrintWriter out = new PrintWriter(fileName, StandardCharsets.UTF_8)) {
+ out.print(fileValue);
+ }
+ }
+
+ private static void runProcess(final String cmd, final Runtime run) throws IOException, InterruptedException {
+ final Process pr = run.exec(cmd);
+ if (!pr.waitFor(30, TimeUnit.SECONDS)) {
+ pr.destroy();
+ throw new RuntimeException("Timeout while executing: " + cmd);
+ }
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/main/resources/AirbyteLogMessageTemplate.json b/airbyte-integrations/bases/base-java-async/src/main/resources/AirbyteLogMessageTemplate.json
new file mode 100644
index 0000000000000..ea1e0f9f7b402
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/main/resources/AirbyteLogMessageTemplate.json
@@ -0,0 +1,21 @@
+{
+ "type": "LOG",
+ "log": {
+ "level": {
+ "$resolver": "level",
+ "field": "name"
+ },
+ "message": {
+ "$resolver": "pattern",
+ "pattern": "%level %C{1.}(%M):%L %m",
+ "stringified": true
+ },
+ "stack_trace": {
+ "$resolver": "exception",
+ "field": "stackTrace",
+ "stackTrace": {
+ "stringified": true
+ }
+ }
+ }
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/main/resources/bastion/Dockerfile b/airbyte-integrations/bases/base-java-async/src/main/resources/bastion/Dockerfile
new file mode 100644
index 0000000000000..e50bfde6aed9d
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/main/resources/bastion/Dockerfile
@@ -0,0 +1,25 @@
+FROM ubuntu:18.04
+
+RUN apt-get update && apt-get install -y openssh-server
+RUN apt-get install -y apt-utils
+RUN mkdir /var/run/sshd
+RUN sed -i 's/PermitRootLogin prohibit-password/PermitRootLogin yes/' /etc/ssh/sshd_config
+RUN sed -ri 's/UsePAM yes/#UsePAM yes/g' /etc/ssh/sshd_config
+
+RUN useradd -m -s /bin/bash sshuser
+RUN echo "sshuser:secret" | chpasswd
+
+RUN mkdir /var/bastion
+RUN ssh-keygen -m PEM -t rsa -b 4096 -C "test-container-bastion" -P "" -f /var/bastion/id_rsa -q
+RUN install -D /var/bastion/id_rsa.pub /home/sshuser/.ssh/authorized_keys
+
+RUN chown -R sshuser:sshuser /home/sshuser/.ssh
+RUN chmod 600 /home/sshuser/.ssh/authorized_keys
+
+RUN mkdir /root/.ssh
+
+RUN apt-get clean && \
+ rm -rf /var/lib/apt/lists/* /tmp/* /var/tmp/*
+EXPOSE 22
+
+CMD ["/usr/sbin/sshd", "-D"]
diff --git a/airbyte-integrations/bases/base-java-async/src/main/resources/log4j2.xml b/airbyte-integrations/bases/base-java-async/src/main/resources/log4j2.xml
new file mode 100644
index 0000000000000..81e76194de838
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/main/resources/log4j2.xml
@@ -0,0 +1,14 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/airbyte-integrations/bases/base-java-async/src/main/resources/ssh-tunnel-spec.json b/airbyte-integrations/bases/base-java-async/src/main/resources/ssh-tunnel-spec.json
new file mode 100644
index 0000000000000..4597f533a0341
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/main/resources/ssh-tunnel-spec.json
@@ -0,0 +1,114 @@
+{
+ "type": "object",
+ "title": "SSH Tunnel Method",
+ "description": "Whether to initiate an SSH tunnel before connecting to the database, and if so, which kind of authentication to use.",
+ "oneOf": [
+ {
+ "title": "No Tunnel",
+ "required": ["tunnel_method"],
+ "properties": {
+ "tunnel_method": {
+ "description": "No ssh tunnel needed to connect to database",
+ "type": "string",
+ "const": "NO_TUNNEL",
+ "order": 0
+ }
+ }
+ },
+ {
+ "title": "SSH Key Authentication",
+ "required": [
+ "tunnel_method",
+ "tunnel_host",
+ "tunnel_port",
+ "tunnel_user",
+ "ssh_key"
+ ],
+ "properties": {
+ "tunnel_method": {
+ "description": "Connect through a jump server tunnel host using username and ssh key",
+ "type": "string",
+ "const": "SSH_KEY_AUTH",
+ "order": 0
+ },
+ "tunnel_host": {
+ "title": "SSH Tunnel Jump Server Host",
+ "description": "Hostname of the jump server host that allows inbound ssh tunnel.",
+ "type": "string",
+ "order": 1
+ },
+ "tunnel_port": {
+ "title": "SSH Connection Port",
+ "description": "Port on the proxy/jump server that accepts inbound ssh connections.",
+ "type": "integer",
+ "minimum": 0,
+ "maximum": 65536,
+ "default": 22,
+ "examples": ["22"],
+ "order": 2
+ },
+ "tunnel_user": {
+ "title": "SSH Login Username",
+ "description": "OS-level username for logging into the jump server host.",
+ "type": "string",
+ "order": 3
+ },
+ "ssh_key": {
+ "title": "SSH Private Key",
+ "description": "OS-level user account ssh key credentials in RSA PEM format ( created with ssh-keygen -t rsa -m PEM -f myuser_rsa )",
+ "type": "string",
+ "airbyte_secret": true,
+ "multiline": true,
+ "order": 4
+ }
+ }
+ },
+ {
+ "title": "Password Authentication",
+ "required": [
+ "tunnel_method",
+ "tunnel_host",
+ "tunnel_port",
+ "tunnel_user",
+ "tunnel_user_password"
+ ],
+ "properties": {
+ "tunnel_method": {
+ "description": "Connect through a jump server tunnel host using username and password authentication",
+ "type": "string",
+ "const": "SSH_PASSWORD_AUTH",
+ "order": 0
+ },
+ "tunnel_host": {
+ "title": "SSH Tunnel Jump Server Host",
+ "description": "Hostname of the jump server host that allows inbound ssh tunnel.",
+ "type": "string",
+ "order": 1
+ },
+ "tunnel_port": {
+ "title": "SSH Connection Port",
+ "description": "Port on the proxy/jump server that accepts inbound ssh connections.",
+ "type": "integer",
+ "minimum": 0,
+ "maximum": 65536,
+ "default": 22,
+ "examples": ["22"],
+ "order": 2
+ },
+ "tunnel_user": {
+ "title": "SSH Login Username",
+ "description": "OS-level username for logging into the jump server host",
+ "type": "string",
+ "order": 3
+ },
+ "tunnel_user_password": {
+ "title": "Password",
+ "description": "OS-level password for logging into the jump server host",
+ "type": "string",
+ "airbyte_secret": true,
+ "order": 4
+ }
+ }
+ }
+ ]
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/AirbyteExceptionHandlerTest.java b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/AirbyteExceptionHandlerTest.java
new file mode 100644
index 0000000000000..8729bca1f8d98
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/AirbyteExceptionHandlerTest.java
@@ -0,0 +1,73 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.base;
+
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.spy;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import io.airbyte.commons.json.Jsons;
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+import java.nio.charset.StandardCharsets;
+import lombok.SneakyThrows;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+import org.slf4j.LoggerFactory;
+
+public class AirbyteExceptionHandlerTest {
+
+ PrintStream originalOut = System.out;
+ private volatile ByteArrayOutputStream outContent = new ByteArrayOutputStream();
+
+ @Before
+ public void setUpOut() {
+ System.setOut(new PrintStream(outContent, true, StandardCharsets.UTF_8));
+ }
+
+ @Test
+ void testTraceMessageEmission() throws Exception {
+ // mocking terminate() method in AirbyteExceptionHandler, so we don't kill the JVM
+ AirbyteExceptionHandler airbyteExceptionHandler = spy(new AirbyteExceptionHandler());
+ doNothing().when(airbyteExceptionHandler).terminate();
+
+ // have to spawn a new thread to test the uncaught exception handling,
+ // because junit catches any exceptions in main thread, i.e. they're not 'uncaught'
+ Thread thread = new Thread() {
+
+ @SneakyThrows
+ public void run() {
+ setUpOut();
+ final IntegrationRunner runner = Mockito.mock(IntegrationRunner.class);
+ doThrow(new RuntimeException("error")).when(runner).run(new String[] {"write"});
+ runner.run(new String[] {"write"});
+ }
+
+ };
+ thread.setUncaughtExceptionHandler(airbyteExceptionHandler);
+ thread.start();
+ thread.join();
+ System.out.flush();
+ revertOut();
+
+ // now we turn the std out from the thread into json and check it's the expected TRACE message
+ JsonNode traceMsgJson = Jsons.deserialize(outContent.toString(StandardCharsets.UTF_8));
+ LoggerFactory.getLogger(AirbyteExceptionHandlerTest.class).debug(traceMsgJson.toString());
+ Assertions.assertEquals("TRACE", traceMsgJson.get("type").asText());
+ Assertions.assertEquals("ERROR", traceMsgJson.get("trace").get("type").asText());
+ Assertions.assertEquals(AirbyteExceptionHandler.logMessage, traceMsgJson.get("trace").get("error").get("message").asText());
+ Assertions.assertEquals("system_error", traceMsgJson.get("trace").get("error").get("failure_type").asText());
+ }
+
+ @After
+ public void revertOut() {
+ System.setOut(originalOut);
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/AirbyteLogMessageTemplateTest.java b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/AirbyteLogMessageTemplateTest.java
new file mode 100644
index 0000000000000..6862221f3d8ed
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/AirbyteLogMessageTemplateTest.java
@@ -0,0 +1,110 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.base;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import io.airbyte.commons.json.Jsons;
+import io.airbyte.protocol.models.AirbyteLogMessage;
+import io.airbyte.protocol.models.AirbyteMessage;
+import io.airbyte.protocol.models.AirbyteMessage.Type;
+import java.io.ByteArrayOutputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.Optional;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.appender.OutputStreamAppender;
+import org.apache.logging.log4j.core.config.Configuration;
+import org.apache.logging.log4j.core.config.LoggerConfig;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.platform.commons.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AirbyteLogMessageTemplateTest {
+
+ private static final ByteArrayOutputStream outputContent = new ByteArrayOutputStream();
+ private static final Logger LOGGER = LoggerFactory.getLogger(AirbyteLogMessageTemplateTest.class);
+ public static final String OUTPUT_STREAM_APPENDER = "OutputStreamAppender";
+ public static final String CONSOLE_JSON_APPENDER = "ConsoleJSONAppender";
+ private static OutputStreamAppender outputStreamAppender;
+ private static LoggerConfig rootLoggerConfig;
+
+ @BeforeAll
+ static void init() {
+ // We are creating a log appender with the same output pattern
+ // as the console json appender defined in this project's log4j2.xml file.
+ // We then attach this log appender with the LOGGER instance so that we can validate the logs
+ // produced by code and assert that it matches the expected format.
+ final LoggerContext loggerContext = (LoggerContext) LogManager.getContext(false);
+ final Configuration configuration = loggerContext.getConfiguration();
+ rootLoggerConfig = configuration.getLoggerConfig("");
+
+ outputStreamAppender = OutputStreamAppender.createAppender(
+ rootLoggerConfig.getAppenders().get(CONSOLE_JSON_APPENDER).getLayout(),
+ null, outputContent, OUTPUT_STREAM_APPENDER, false, true);
+ outputStreamAppender.start();
+
+ rootLoggerConfig.addAppender(outputStreamAppender, Level.ALL, null);
+ }
+
+ @BeforeEach
+ void setup() {
+ outputContent.reset();
+ }
+
+ @AfterAll
+ static void cleanUp() {
+ outputStreamAppender.stop();
+ rootLoggerConfig.removeAppender(OUTPUT_STREAM_APPENDER);
+ }
+
+ @Test
+ public void testAirbyteLogMessageFormat() throws java.io.IOException {
+ LOGGER.info("hello");
+
+ outputContent.flush();
+ final String logMessage = outputContent.toString(StandardCharsets.UTF_8);
+ final AirbyteMessage airbyteMessage = validateLogIsAirbyteMessage(logMessage);
+ final AirbyteLogMessage airbyteLogMessage = validateAirbyteMessageIsLog(airbyteMessage);
+
+ final String connectorLogMessage = airbyteLogMessage.getMessage();
+ // validate that the message inside AirbyteLogMessage matches the pattern.
+ // pattern to check for is: LOG_LEVEL className(methodName):LineNumber logMessage
+ final String connectorLogMessageRegex =
+ "^INFO [\\w+.]*.AirbyteLogMessageTemplateTest\\(testAirbyteLogMessageFormat\\):\\d+ hello$";
+ final Pattern pattern = Pattern.compile(connectorLogMessageRegex);
+
+ final Matcher matcher = pattern.matcher(connectorLogMessage);
+ assertTrue(matcher.matches(), connectorLogMessage);
+ }
+
+ private AirbyteMessage validateLogIsAirbyteMessage(final String logMessage) {
+ final Optional jsonLine = Jsons.tryDeserialize(logMessage);
+ assertFalse(jsonLine.isEmpty());
+
+ final Optional m = Jsons.tryObject(jsonLine.get(), AirbyteMessage.class);
+ assertFalse(m.isEmpty());
+ return m.get();
+ }
+
+ private AirbyteLogMessage validateAirbyteMessageIsLog(final AirbyteMessage airbyteMessage) {
+ assertEquals(Type.LOG, airbyteMessage.getType());
+ assertNotNull(airbyteMessage.getLog());
+ assertFalse(StringUtils.isBlank(airbyteMessage.getLog().getMessage()));
+ return airbyteMessage.getLog();
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/AirbyteTraceMessageUtilityTest.java b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/AirbyteTraceMessageUtilityTest.java
new file mode 100644
index 0000000000000..c5f7db19131a4
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/AirbyteTraceMessageUtilityTest.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.base;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import io.airbyte.commons.json.Jsons;
+import io.airbyte.protocol.models.v0.AirbyteErrorTraceMessage.FailureType;
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+import java.nio.charset.StandardCharsets;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+public class AirbyteTraceMessageUtilityTest {
+
+ PrintStream originalOut = System.out;
+ private final ByteArrayOutputStream outContent = new ByteArrayOutputStream();
+
+ @BeforeEach
+ public void setUpOut() {
+ System.setOut(new PrintStream(outContent, true, StandardCharsets.UTF_8));
+ }
+
+ private void assertJsonNodeIsTraceMessage(JsonNode jsonNode) {
+ // todo: this check could be better by actually trying to convert the JsonNode to an
+ // AirbyteTraceMessage instance
+ Assertions.assertEquals("TRACE", jsonNode.get("type").asText());
+ Assertions.assertNotNull(jsonNode.get("trace"));
+ }
+
+ @Test
+ void testEmitSystemErrorTrace() {
+ AirbyteTraceMessageUtility.emitSystemErrorTrace(Mockito.mock(RuntimeException.class), "this is a system error");
+ JsonNode outJson = Jsons.deserialize(outContent.toString(StandardCharsets.UTF_8));
+ assertJsonNodeIsTraceMessage(outJson);
+ Assertions.assertEquals("system_error", outJson.get("trace").get("error").get("failure_type").asText());
+ }
+
+ @Test
+ void testEmitConfigErrorTrace() {
+ AirbyteTraceMessageUtility.emitConfigErrorTrace(Mockito.mock(RuntimeException.class), "this is a config error");
+ JsonNode outJson = Jsons.deserialize(outContent.toString(StandardCharsets.UTF_8));
+ assertJsonNodeIsTraceMessage(outJson);
+ Assertions.assertEquals("config_error", outJson.get("trace").get("error").get("failure_type").asText());
+ }
+
+ @Test
+ void testEmitErrorTrace() {
+ AirbyteTraceMessageUtility.emitErrorTrace(Mockito.mock(RuntimeException.class), "this is an error", FailureType.SYSTEM_ERROR);
+ assertJsonNodeIsTraceMessage(Jsons.deserialize(outContent.toString(StandardCharsets.UTF_8)));
+ }
+
+ @Test
+ void testCorrectStacktraceFormat() {
+ try {
+ int x = 1 / 0;
+ } catch (Exception e) {
+ AirbyteTraceMessageUtility.emitSystemErrorTrace(e, "you exploded the universe");
+ }
+ JsonNode outJson = Jsons.deserialize(outContent.toString(StandardCharsets.UTF_8));
+ Assertions.assertTrue(outJson.get("trace").get("error").get("stack_trace").asText().contains("\n\tat"));
+ }
+
+ @AfterEach
+ public void revertOut() {
+ System.setOut(originalOut);
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/FailureTrackingAirbyteMessageConsumerTest.java b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/FailureTrackingAirbyteMessageConsumerTest.java
new file mode 100644
index 0000000000000..dba9eb0483a41
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/FailureTrackingAirbyteMessageConsumerTest.java
@@ -0,0 +1,86 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.base;
+
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import io.airbyte.protocol.models.v0.AirbyteMessage;
+import io.airbyte.protocol.models.v0.AirbyteMessage.Type;
+import org.junit.jupiter.api.Test;
+
+class FailureTrackingAirbyteMessageConsumerTest {
+
+ @Test
+ void testStartNoFailure() throws Exception {
+ final TestConsumer consumer = spy(new TestConsumer());
+ consumer.start();
+ consumer.close();
+
+ verify(consumer).close(false);
+ }
+
+ @Test
+ void testStartWithFailure() throws Exception {
+ final TestConsumer consumer = spy(new TestConsumer());
+ doThrow(new RuntimeException()).when(consumer).startTracked();
+
+ // verify the exception still gets thrown.
+ assertThrows(RuntimeException.class, consumer::start);
+ consumer.close();
+
+ verify(consumer).close(true);
+ }
+
+ @Test
+ void testAcceptNoFailure() throws Exception {
+ final TestConsumer consumer = spy(new TestConsumer());
+
+ final AirbyteMessage msg = mock(AirbyteMessage.class);
+ consumer.accept(msg);
+ consumer.close();
+
+ verify(consumer).close(false);
+ }
+
+ @Test
+ void testAcceptWithFailure() throws Exception {
+ final TestConsumer consumer = spy(new TestConsumer());
+ final AirbyteMessage msg = mock(AirbyteMessage.class);
+ when(msg.getType()).thenReturn(Type.RECORD);
+ doThrow(new RuntimeException()).when(consumer).acceptTracked(any());
+
+ // verify the exception still gets thrown.
+ assertThrows(RuntimeException.class, () -> consumer.accept(msg));
+ consumer.close();
+
+ verify(consumer).close(true);
+ }
+
+ static class TestConsumer extends FailureTrackingAirbyteMessageConsumer {
+
+ @Override
+ protected void startTracked() {
+
+ }
+
+ @Override
+ protected void acceptTracked(final AirbyteMessage s) {
+
+ }
+
+ @Override
+ protected void close(final boolean hasFailed) {
+
+ }
+
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/IntegrationCliParserTest.java b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/IntegrationCliParserTest.java
new file mode 100644
index 0000000000000..384e13347fdeb
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/IntegrationCliParserTest.java
@@ -0,0 +1,67 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.base;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.nio.file.Path;
+import org.junit.jupiter.api.Test;
+
+class IntegrationCliParserTest {
+
+ private static final String CONFIG_FILENAME = "config.json";
+ private static final String CATALOG_FILENAME = "catalog.json";
+ private static final String STATE_FILENAME = "state.json";
+
+ @Test
+ void testSpec() {
+ final String[] args = new String[] {"--spec"};
+ final IntegrationConfig actual = new IntegrationCliParser().parse(args);
+ assertEquals(IntegrationConfig.spec(), actual);
+ }
+
+ @Test
+ void testCheck() {
+ final String[] args = new String[] {"--check", "--config", CONFIG_FILENAME};
+ final IntegrationConfig actual = new IntegrationCliParser().parse(args);
+ assertEquals(IntegrationConfig.check(Path.of(CONFIG_FILENAME)), actual);
+ }
+
+ @Test
+ void testDiscover() {
+ final String[] args = new String[] {"--discover", "--config", CONFIG_FILENAME};
+ final IntegrationConfig actual = new IntegrationCliParser().parse(args);
+ assertEquals(IntegrationConfig.discover(Path.of(CONFIG_FILENAME)), actual);
+ }
+
+ @Test
+ void testWrite() {
+ final String[] args = new String[] {"--write", "--config", CONFIG_FILENAME, "--catalog", CATALOG_FILENAME};
+ final IntegrationConfig actual = new IntegrationCliParser().parse(args);
+ assertEquals(IntegrationConfig.write(Path.of(CONFIG_FILENAME), Path.of(CATALOG_FILENAME)), actual);
+ }
+
+ @Test
+ void testReadWithoutState() {
+ final String[] args = new String[] {"--read", "--config", CONFIG_FILENAME, "--catalog", CATALOG_FILENAME};
+ final IntegrationConfig actual = new IntegrationCliParser().parse(args);
+ assertEquals(IntegrationConfig.read(Path.of(CONFIG_FILENAME), Path.of(CATALOG_FILENAME), null), actual);
+ }
+
+ @Test
+ void testReadWithState() {
+ final String[] args = new String[] {"--read", "--config", CONFIG_FILENAME, "--catalog", CATALOG_FILENAME, "--state", STATE_FILENAME};
+ final IntegrationConfig actual = new IntegrationCliParser().parse(args);
+ assertEquals(IntegrationConfig.read(Path.of(CONFIG_FILENAME), Path.of(CATALOG_FILENAME), Path.of(STATE_FILENAME)), actual);
+ }
+
+ @Test
+ void testFailsOnUnknownArg() {
+ final String[] args = new String[] {"--check", "--config", CONFIG_FILENAME, "--random", "garbage"};
+ assertThrows(IllegalArgumentException.class, () -> new IntegrationCliParser().parse(args));
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/IntegrationConfigTest.java b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/IntegrationConfigTest.java
new file mode 100644
index 0000000000000..926fca719906b
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/IntegrationConfigTest.java
@@ -0,0 +1,87 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.base;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.nio.file.Path;
+import java.util.Optional;
+import org.junit.jupiter.api.Test;
+
+class IntegrationConfigTest {
+
+ private static final Path CONFIG_PATH = Path.of("config.json");
+ private static final Path CATALOG_PATH = Path.of("catalog.json");
+ private static final Path STATE_PATH = Path.of("state.json");
+
+ @Test
+ void testSpec() {
+ final IntegrationConfig config = IntegrationConfig.spec();
+ assertEquals(Command.SPEC, config.getCommand());
+ assertThrows(IllegalStateException.class, config::getConfigPath);
+ assertThrows(IllegalStateException.class, config::getCatalogPath);
+ assertThrows(IllegalStateException.class, config::getStatePath);
+ }
+
+ @Test
+ void testCheck() {
+ assertThrows(NullPointerException.class, () -> IntegrationConfig.check(null));
+
+ final IntegrationConfig config = IntegrationConfig.check(CONFIG_PATH);
+ assertEquals(Command.CHECK, config.getCommand());
+ assertEquals(CONFIG_PATH, config.getConfigPath());
+ assertThrows(IllegalStateException.class, config::getCatalogPath);
+ assertThrows(IllegalStateException.class, config::getStatePath);
+ }
+
+ @Test
+ void testDiscover() {
+ assertThrows(NullPointerException.class, () -> IntegrationConfig.discover(null));
+
+ final IntegrationConfig config = IntegrationConfig.discover(CONFIG_PATH);
+ assertEquals(Command.DISCOVER, config.getCommand());
+ assertEquals(CONFIG_PATH, config.getConfigPath());
+ assertThrows(IllegalStateException.class, config::getCatalogPath);
+ assertThrows(IllegalStateException.class, config::getStatePath);
+ }
+
+ @Test
+ void testWrite() {
+ assertThrows(NullPointerException.class, () -> IntegrationConfig.write(null, CATALOG_PATH));
+ assertThrows(NullPointerException.class, () -> IntegrationConfig.write(CONFIG_PATH, null));
+
+ final IntegrationConfig config = IntegrationConfig.write(CONFIG_PATH, CATALOG_PATH);
+ assertEquals(Command.WRITE, config.getCommand());
+ assertEquals(CONFIG_PATH, config.getConfigPath());
+ assertEquals(CATALOG_PATH, config.getCatalogPath());
+ assertThrows(IllegalStateException.class, config::getStatePath);
+ }
+
+ @Test
+ void testReadWithState() {
+ assertThrows(NullPointerException.class, () -> IntegrationConfig.read(null, CATALOG_PATH, STATE_PATH));
+ assertThrows(NullPointerException.class, () -> IntegrationConfig.read(CONFIG_PATH, null, STATE_PATH));
+
+ final IntegrationConfig config = IntegrationConfig.read(CONFIG_PATH, CATALOG_PATH, STATE_PATH);
+ assertEquals(Command.READ, config.getCommand());
+ assertEquals(CONFIG_PATH, config.getConfigPath());
+ assertEquals(CATALOG_PATH, config.getCatalogPath());
+ assertEquals(Optional.of(STATE_PATH), config.getStatePath());
+ }
+
+ @Test
+ void testReadWithoutState() {
+ assertThrows(NullPointerException.class, () -> IntegrationConfig.read(null, CATALOG_PATH, null));
+ assertThrows(NullPointerException.class, () -> IntegrationConfig.read(CONFIG_PATH, null, null));
+
+ final IntegrationConfig config = IntegrationConfig.read(CONFIG_PATH, CATALOG_PATH, null);
+ assertEquals(Command.READ, config.getCommand());
+ assertEquals(CONFIG_PATH, config.getConfigPath());
+ assertEquals(CATALOG_PATH, config.getCatalogPath());
+ assertEquals(Optional.empty(), config.getStatePath());
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/IntegrationRunnerTest.java b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/IntegrationRunnerTest.java
new file mode 100644
index 0000000000000..866bf8e07aa17
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/IntegrationRunnerTest.java
@@ -0,0 +1,491 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.base;
+
+import static io.airbyte.integrations.util.ConnectorExceptionUtil.COMMON_EXCEPTION_MESSAGE_TEMPLATE;
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+import static org.assertj.core.api.AssertionsForClassTypes.catchThrowable;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.inOrder;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import io.airbyte.commons.exceptions.ConfigErrorException;
+import io.airbyte.commons.io.IOs;
+import io.airbyte.commons.json.Jsons;
+import io.airbyte.commons.util.AutoCloseableIterators;
+import io.airbyte.commons.util.MoreIterators;
+import io.airbyte.protocol.models.v0.AirbyteCatalog;
+import io.airbyte.protocol.models.v0.AirbyteConnectionStatus;
+import io.airbyte.protocol.models.v0.AirbyteConnectionStatus.Status;
+import io.airbyte.protocol.models.v0.AirbyteMessage;
+import io.airbyte.protocol.models.v0.AirbyteMessage.Type;
+import io.airbyte.protocol.models.v0.AirbyteRecordMessage;
+import io.airbyte.protocol.models.v0.AirbyteStateMessage;
+import io.airbyte.protocol.models.v0.AirbyteStream;
+import io.airbyte.protocol.models.v0.CatalogHelpers;
+import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog;
+import io.airbyte.protocol.models.v0.ConnectorSpecification;
+import io.airbyte.validation.json.JsonSchemaValidator;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.commons.lang3.ThreadUtils;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.InOrder;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class IntegrationRunnerTest {
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(IntegrationRunnerTest.class);
+
+ private static final String CONFIG_FILE_NAME = "config.json";
+ private static final String CONFIGURED_CATALOG_FILE_NAME = "configured_catalog.json";
+ private static final String STATE_FILE_NAME = "state.json";
+
+ private static final String[] ARGS = new String[] {"args"};
+
+ private static final String CONFIG_STRING = "{ \"username\": \"airbyte\" }";
+ private static final JsonNode CONFIG = Jsons.deserialize(CONFIG_STRING);
+ private static final String STREAM_NAME = "users";
+ private static final Long EMITTED_AT = Instant.now().toEpochMilli();
+ private static final Path TEST_ROOT = Path.of("/tmp/airbyte_tests");
+
+ private static final AirbyteCatalog CATALOG = new AirbyteCatalog().withStreams(Lists.newArrayList(new AirbyteStream().withName(STREAM_NAME)));
+ private static final ConfiguredAirbyteCatalog CONFIGURED_CATALOG = CatalogHelpers.toDefaultConfiguredCatalog(CATALOG);
+ private static final JsonNode STATE = Jsons.jsonNode(ImmutableMap.of("checkpoint", "05/08/1945"));
+
+ private IntegrationCliParser cliParser;
+ private Consumer stdoutConsumer;
+ private Destination destination;
+ private Source source;
+ private Path configPath;
+ private Path configuredCatalogPath;
+ private Path statePath;
+
+ @SuppressWarnings("unchecked")
+ @BeforeEach
+ void setup() throws IOException {
+ cliParser = mock(IntegrationCliParser.class);
+ stdoutConsumer = Mockito.mock(Consumer.class);
+ destination = mock(Destination.class);
+ source = mock(Source.class);
+ final Path configDir = Files.createTempDirectory(Files.createDirectories(TEST_ROOT), "test");
+
+ configPath = IOs.writeFile(configDir, CONFIG_FILE_NAME, CONFIG_STRING);
+ configuredCatalogPath = IOs.writeFile(configDir, CONFIGURED_CATALOG_FILE_NAME, Jsons.serialize(CONFIGURED_CATALOG));
+ statePath = IOs.writeFile(configDir, STATE_FILE_NAME, Jsons.serialize(STATE));
+
+ final String testName = Thread.currentThread().getName();
+ ThreadUtils.getAllThreads()
+ .stream()
+ .filter(runningThread -> !runningThread.isDaemon())
+ .forEach(runningThread -> runningThread.setName(testName));
+ }
+
+ @Test
+ void testSpecSource() throws Exception {
+ final IntegrationConfig intConfig = IntegrationConfig.spec();
+ final ConnectorSpecification output = new ConnectorSpecification().withDocumentationUrl(new URI("https://docs.airbyte.io/"));
+
+ when(cliParser.parse(ARGS)).thenReturn(intConfig);
+ when(source.spec()).thenReturn(output);
+
+ new IntegrationRunner(cliParser, stdoutConsumer, null, source).run(ARGS);
+
+ verify(source).spec();
+ verify(stdoutConsumer).accept(new AirbyteMessage().withType(Type.SPEC).withSpec(output));
+ }
+
+ @Test
+ void testSpecDestination() throws Exception {
+ final IntegrationConfig intConfig = IntegrationConfig.spec();
+ final ConnectorSpecification output = new ConnectorSpecification().withDocumentationUrl(new URI("https://docs.airbyte.io/"));
+
+ when(cliParser.parse(ARGS)).thenReturn(intConfig);
+ when(destination.spec()).thenReturn(output);
+
+ new IntegrationRunner(cliParser, stdoutConsumer, destination, null).run(ARGS);
+
+ verify(destination).spec();
+ verify(stdoutConsumer).accept(new AirbyteMessage().withType(Type.SPEC).withSpec(output));
+ }
+
+ @Test
+ void testCheckSource() throws Exception {
+ final IntegrationConfig intConfig = IntegrationConfig.check(configPath);
+ final AirbyteConnectionStatus output = new AirbyteConnectionStatus().withStatus(Status.FAILED).withMessage("it failed");
+
+ when(cliParser.parse(ARGS)).thenReturn(intConfig);
+ when(source.check(CONFIG)).thenReturn(output);
+
+ final ConnectorSpecification expectedConnSpec = mock(ConnectorSpecification.class);
+ when(source.spec()).thenReturn(expectedConnSpec);
+ when(expectedConnSpec.getConnectionSpecification()).thenReturn(CONFIG);
+ final JsonSchemaValidator jsonSchemaValidator = mock(JsonSchemaValidator.class);
+ new IntegrationRunner(cliParser, stdoutConsumer, null, source, jsonSchemaValidator).run(ARGS);
+
+ verify(source).check(CONFIG);
+ verify(stdoutConsumer).accept(new AirbyteMessage().withType(Type.CONNECTION_STATUS).withConnectionStatus(output));
+ verify(jsonSchemaValidator).validate(any(), any());
+ }
+
+ @Test
+ void testCheckDestination() throws Exception {
+ final IntegrationConfig intConfig = IntegrationConfig.check(configPath);
+ final AirbyteConnectionStatus output = new AirbyteConnectionStatus().withStatus(Status.FAILED).withMessage("it failed");
+
+ when(cliParser.parse(ARGS)).thenReturn(intConfig);
+ when(destination.check(CONFIG)).thenReturn(output);
+
+ final ConnectorSpecification expectedConnSpec = mock(ConnectorSpecification.class);
+ when(destination.spec()).thenReturn(expectedConnSpec);
+ when(expectedConnSpec.getConnectionSpecification()).thenReturn(CONFIG);
+
+ final JsonSchemaValidator jsonSchemaValidator = mock(JsonSchemaValidator.class);
+
+ new IntegrationRunner(cliParser, stdoutConsumer, destination, null, jsonSchemaValidator).run(ARGS);
+
+ verify(destination).check(CONFIG);
+ verify(stdoutConsumer).accept(new AirbyteMessage().withType(Type.CONNECTION_STATUS).withConnectionStatus(output));
+ verify(jsonSchemaValidator).validate(any(), any());
+ }
+
+ @Test
+ void testDiscover() throws Exception {
+ final IntegrationConfig intConfig = IntegrationConfig.discover(configPath);
+ final AirbyteCatalog output = new AirbyteCatalog()
+ .withStreams(Lists.newArrayList(new AirbyteStream().withName("oceans")));
+
+ when(cliParser.parse(ARGS)).thenReturn(intConfig);
+ when(source.discover(CONFIG)).thenReturn(output);
+
+ final ConnectorSpecification expectedConnSpec = mock(ConnectorSpecification.class);
+ when(source.spec()).thenReturn(expectedConnSpec);
+ when(expectedConnSpec.getConnectionSpecification()).thenReturn(CONFIG);
+
+ final JsonSchemaValidator jsonSchemaValidator = mock(JsonSchemaValidator.class);
+ new IntegrationRunner(cliParser, stdoutConsumer, null, source, jsonSchemaValidator).run(ARGS);
+
+ verify(source).discover(CONFIG);
+ verify(stdoutConsumer).accept(new AirbyteMessage().withType(Type.CATALOG).withCatalog(output));
+ verify(jsonSchemaValidator).validate(any(), any());
+ }
+
+ @Test
+ void testRead() throws Exception {
+ final IntegrationConfig intConfig = IntegrationConfig.read(configPath, configuredCatalogPath,
+ statePath);
+ final AirbyteMessage message1 = new AirbyteMessage().withType(Type.RECORD)
+ .withRecord(new AirbyteRecordMessage().withData(Jsons.jsonNode(ImmutableMap.of("names", "byron"))));
+ final AirbyteMessage message2 = new AirbyteMessage().withType(Type.RECORD).withRecord(new AirbyteRecordMessage()
+ .withData(Jsons.jsonNode(ImmutableMap.of("names", "reginald"))));
+
+ when(cliParser.parse(ARGS)).thenReturn(intConfig);
+ when(source.read(CONFIG, CONFIGURED_CATALOG, STATE))
+ .thenReturn(AutoCloseableIterators.fromIterator(MoreIterators.of(message1, message2)));
+
+ final ConnectorSpecification expectedConnSpec = mock(ConnectorSpecification.class);
+ when(source.spec()).thenReturn(expectedConnSpec);
+ when(expectedConnSpec.getConnectionSpecification()).thenReturn(CONFIG);
+
+ final JsonSchemaValidator jsonSchemaValidator = mock(JsonSchemaValidator.class);
+ new IntegrationRunner(cliParser, stdoutConsumer, null, source, jsonSchemaValidator).run(ARGS);
+
+ verify(source).read(CONFIG, CONFIGURED_CATALOG, STATE);
+ verify(stdoutConsumer).accept(message1);
+ verify(stdoutConsumer).accept(message2);
+ verify(jsonSchemaValidator).validate(any(), any());
+ }
+
+ @Test
+ void testReadException() throws Exception {
+ final IntegrationConfig intConfig = IntegrationConfig.read(configPath, configuredCatalogPath,
+ statePath);
+ final ConfigErrorException configErrorException = new ConfigErrorException("Invalid configuration");
+
+ when(cliParser.parse(ARGS)).thenReturn(intConfig);
+ when(source.read(CONFIG, CONFIGURED_CATALOG, STATE)).thenThrow(configErrorException);
+
+ final ConnectorSpecification expectedConnSpec = mock(ConnectorSpecification.class);
+ when(source.spec()).thenReturn(expectedConnSpec);
+ when(expectedConnSpec.getConnectionSpecification()).thenReturn(CONFIG);
+
+ final JsonSchemaValidator jsonSchemaValidator = mock(JsonSchemaValidator.class);
+ final Throwable throwable = catchThrowable(() -> new IntegrationRunner(cliParser, stdoutConsumer, null, source, jsonSchemaValidator).run(ARGS));
+
+ assertThat(throwable).isInstanceOf(ConfigErrorException.class);
+ verify(source).read(CONFIG, CONFIGURED_CATALOG, STATE);
+ }
+
+ @Test
+ void testCheckNestedException() throws Exception {
+ final IntegrationConfig intConfig = IntegrationConfig.check(configPath);
+ final AirbyteConnectionStatus output = new AirbyteConnectionStatus().withStatus(Status.FAILED).withMessage("Invalid configuration");
+ final ConfigErrorException configErrorException = new ConfigErrorException("Invalid configuration");
+ final RuntimeException runtimeException = new RuntimeException(new RuntimeException(configErrorException));
+
+ when(cliParser.parse(ARGS)).thenReturn(intConfig);
+ when(source.check(CONFIG)).thenThrow(runtimeException);
+
+ final ConnectorSpecification expectedConnSpec = mock(ConnectorSpecification.class);
+ when(source.spec()).thenReturn(expectedConnSpec);
+ when(expectedConnSpec.getConnectionSpecification()).thenReturn(CONFIG);
+ final JsonSchemaValidator jsonSchemaValidator = mock(JsonSchemaValidator.class);
+ new IntegrationRunner(cliParser, stdoutConsumer, null, source, jsonSchemaValidator).run(ARGS);
+
+ verify(source).check(CONFIG);
+ verify(stdoutConsumer).accept(new AirbyteMessage().withType(Type.CONNECTION_STATUS).withConnectionStatus(output));
+ verify(jsonSchemaValidator).validate(any(), any());
+ }
+
+ @Test
+ void testCheckRuntimeException() throws Exception {
+ final IntegrationConfig intConfig = IntegrationConfig.check(configPath);
+ final AirbyteConnectionStatus output =
+ new AirbyteConnectionStatus().withStatus(Status.FAILED).withMessage(String.format(COMMON_EXCEPTION_MESSAGE_TEMPLATE, "Runtime Error"));
+ final RuntimeException runtimeException = new RuntimeException("Runtime Error");
+
+ when(cliParser.parse(ARGS)).thenReturn(intConfig);
+ when(source.check(CONFIG)).thenThrow(runtimeException);
+
+ final ConnectorSpecification expectedConnSpec = mock(ConnectorSpecification.class);
+ when(source.spec()).thenReturn(expectedConnSpec);
+ when(expectedConnSpec.getConnectionSpecification()).thenReturn(CONFIG);
+ final JsonSchemaValidator jsonSchemaValidator = mock(JsonSchemaValidator.class);
+ new IntegrationRunner(cliParser, stdoutConsumer, null, source, jsonSchemaValidator).run(ARGS);
+
+ verify(source).check(CONFIG);
+ verify(stdoutConsumer).accept(new AirbyteMessage().withType(Type.CONNECTION_STATUS).withConnectionStatus(output));
+ verify(jsonSchemaValidator).validate(any(), any());
+ }
+
+ @Test
+ void testWrite() throws Exception {
+ final IntegrationConfig intConfig = IntegrationConfig.write(configPath, configuredCatalogPath);
+ final AirbyteMessageConsumer airbyteMessageConsumerMock = mock(AirbyteMessageConsumer.class);
+ when(cliParser.parse(ARGS)).thenReturn(intConfig);
+ when(destination.getConsumer(CONFIG, CONFIGURED_CATALOG, stdoutConsumer)).thenReturn(airbyteMessageConsumerMock);
+
+ final ConnectorSpecification expectedConnSpec = mock(ConnectorSpecification.class);
+ when(destination.spec()).thenReturn(expectedConnSpec);
+ when(expectedConnSpec.getConnectionSpecification()).thenReturn(CONFIG);
+
+ final JsonSchemaValidator jsonSchemaValidator = mock(JsonSchemaValidator.class);
+
+ final IntegrationRunner runner = spy(new IntegrationRunner(cliParser, stdoutConsumer, destination, null, jsonSchemaValidator));
+ runner.run(ARGS);
+
+ verify(destination).getConsumer(CONFIG, CONFIGURED_CATALOG, stdoutConsumer);
+ verify(jsonSchemaValidator).validate(any(), any());
+ }
+
+ @Test
+ void testDestinationConsumerLifecycleSuccess() throws Exception {
+ final AirbyteMessage message1 = new AirbyteMessage()
+ .withType(AirbyteMessage.Type.RECORD)
+ .withRecord(new AirbyteRecordMessage()
+ .withData(Jsons.deserialize("{ \"color\": \"blue\" }"))
+ .withStream(STREAM_NAME)
+ .withEmittedAt(EMITTED_AT));
+ final AirbyteMessage message2 = new AirbyteMessage()
+ .withType(AirbyteMessage.Type.RECORD)
+ .withRecord(new AirbyteRecordMessage()
+ .withData(Jsons.deserialize("{ \"color\": \"yellow\" }"))
+ .withStream(STREAM_NAME)
+ .withEmittedAt(EMITTED_AT));
+ final AirbyteMessage stateMessage = new AirbyteMessage()
+ .withType(Type.STATE)
+ .withState(new AirbyteStateMessage()
+ .withData(Jsons.deserialize("{ \"checkpoint\": \"1\" }")));
+ System.setIn(new ByteArrayInputStream((Jsons.serialize(message1) + "\n"
+ + Jsons.serialize(message2) + "\n"
+ + Jsons.serialize(stateMessage)).getBytes(StandardCharsets.UTF_8)));
+
+ try (final AirbyteMessageConsumer airbyteMessageConsumerMock = mock(AirbyteMessageConsumer.class)) {
+ IntegrationRunner.consumeWriteStream(airbyteMessageConsumerMock);
+ final InOrder inOrder = inOrder(airbyteMessageConsumerMock);
+ inOrder.verify(airbyteMessageConsumerMock).accept(message1);
+ inOrder.verify(airbyteMessageConsumerMock).accept(message2);
+ inOrder.verify(airbyteMessageConsumerMock).accept(stateMessage);
+ }
+ }
+
+ @Test
+ void testDestinationConsumerLifecycleFailure() throws Exception {
+ final AirbyteMessage message1 = new AirbyteMessage()
+ .withType(AirbyteMessage.Type.RECORD)
+ .withRecord(new AirbyteRecordMessage()
+ .withData(Jsons.deserialize("{ \"color\": \"blue\" }"))
+ .withStream(STREAM_NAME)
+ .withEmittedAt(EMITTED_AT));
+ final AirbyteMessage message2 = new AirbyteMessage()
+ .withType(AirbyteMessage.Type.RECORD)
+ .withRecord(new AirbyteRecordMessage()
+ .withData(Jsons.deserialize("{ \"color\": \"yellow\" }"))
+ .withStream(STREAM_NAME)
+ .withEmittedAt(EMITTED_AT));
+ System.setIn(new ByteArrayInputStream((Jsons.serialize(message1) + "\n" + Jsons.serialize(message2)).getBytes(StandardCharsets.UTF_8)));
+
+ try (final AirbyteMessageConsumer airbyteMessageConsumerMock = mock(AirbyteMessageConsumer.class)) {
+ doThrow(new IOException("error")).when(airbyteMessageConsumerMock).accept(message1);
+ assertThrows(IOException.class, () -> IntegrationRunner.consumeWriteStream(airbyteMessageConsumerMock));
+ final InOrder inOrder = inOrder(airbyteMessageConsumerMock);
+ inOrder.verify(airbyteMessageConsumerMock).accept(message1);
+ inOrder.verifyNoMoreInteractions();
+ }
+ }
+
+ @Test
+ void testInterruptOrphanThreadFailure() {
+ final String testName = Thread.currentThread().getName();
+ final List caughtExceptions = new ArrayList<>();
+ startSleepingThread(caughtExceptions, false);
+ assertThrows(IOException.class, () -> IntegrationRunner.watchForOrphanThreads(
+ () -> {
+ throw new IOException("random error");
+ },
+ Assertions::fail,
+ 3, TimeUnit.SECONDS,
+ 10, TimeUnit.SECONDS));
+ try {
+ TimeUnit.SECONDS.sleep(15);
+ } catch (final Exception e) {
+ throw new RuntimeException(e);
+ }
+ final List runningThreads = ThreadUtils.getAllThreads().stream()
+ .filter(runningThread -> !runningThread.isDaemon() && !runningThread.getName().equals(testName))
+ .collect(Collectors.toList());
+ // all threads should be interrupted
+ assertEquals(List.of(), runningThreads);
+ assertEquals(1, caughtExceptions.size());
+ }
+
+ @Test
+ void testNoInterruptOrphanThreadFailure() {
+ final String testName = Thread.currentThread().getName();
+ final List caughtExceptions = new ArrayList<>();
+ final AtomicBoolean exitCalled = new AtomicBoolean(false);
+ startSleepingThread(caughtExceptions, true);
+ assertThrows(IOException.class, () -> IntegrationRunner.watchForOrphanThreads(
+ () -> {
+ throw new IOException("random error");
+ },
+ () -> exitCalled.set(true),
+ 3, TimeUnit.SECONDS,
+ 10, TimeUnit.SECONDS));
+ try {
+ TimeUnit.SECONDS.sleep(15);
+ } catch (final Exception e) {
+ throw new RuntimeException(e);
+ }
+ final List runningThreads = ThreadUtils.getAllThreads().stream()
+ .filter(runningThread -> !runningThread.isDaemon() && !runningThread.getName().equals(testName))
+ .collect(Collectors.toList());
+ // a thread that refuses to be interrupted should remain
+ assertEquals(1, runningThreads.size());
+ assertEquals(1, caughtExceptions.size());
+ assertTrue(exitCalled.get());
+ }
+
+ private void startSleepingThread(final List caughtExceptions, final boolean ignoreInterrupt) {
+ final ExecutorService executorService = Executors.newFixedThreadPool(1);
+ executorService.submit(() -> {
+ for (int tries = 0; tries < 3; tries++) {
+ try {
+ TimeUnit.MINUTES.sleep(5);
+ } catch (final Exception e) {
+ LOGGER.info("Caught Exception", e);
+ caughtExceptions.add(e);
+ if (!ignoreInterrupt) {
+ executorService.shutdownNow();
+ break;
+ }
+ }
+ }
+ });
+ }
+
+ @Test
+ void testParseConnectorImage() {
+ assertEquals("unknown", IntegrationRunner.parseConnectorVersion(null));
+ assertEquals("unknown", IntegrationRunner.parseConnectorVersion(""));
+ assertEquals("1.0.1-alpha", IntegrationRunner.parseConnectorVersion("airbyte/destination-test:1.0.1-alpha"));
+ assertEquals("dev", IntegrationRunner.parseConnectorVersion("airbyte/destination-test:dev"));
+ assertEquals("1.0.1-alpha", IntegrationRunner.parseConnectorVersion("destination-test:1.0.1-alpha"));
+ assertEquals("1.0.1-alpha", IntegrationRunner.parseConnectorVersion(":1.0.1-alpha"));
+ }
+
+ @Test
+ void testConsumptionOfInvalidStateMessage() {
+ final String invalidStateMessage = """
+ {
+ "type" : "STATE",
+ "state" : {
+ "type": "NOT_RECOGNIZED",
+ "global": {
+ "streamStates": {
+ "foo" : "bar"
+ }
+ }
+ }
+ }
+ """;
+
+ Assertions.assertThrows(IllegalStateException.class, () -> {
+ try (final AirbyteMessageConsumer consumer = mock(AirbyteMessageConsumer.class)) {
+ IntegrationRunner.consumeMessage(consumer, invalidStateMessage);
+ }
+ });
+ }
+
+ @Test
+ void testConsumptionOfInvalidNonStateMessage() {
+ final String invalidNonStateMessage = """
+ {
+ "type" : "NOT_RECOGNIZED",
+ "record" : {
+ "namespace": "namespace",
+ "stream": "stream",
+ "emittedAt": 123456789
+ }
+ }
+ """;
+
+ Assertions.assertDoesNotThrow(() -> {
+ try (final AirbyteMessageConsumer consumer = mock(AirbyteMessageConsumer.class)) {
+ IntegrationRunner.consumeMessage(consumer, invalidNonStateMessage);
+ verify(consumer, times(0)).accept(any(AirbyteMessage.class));
+ }
+ });
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/NameTransformerTest.java b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/NameTransformerTest.java
new file mode 100644
index 0000000000000..10e7794713119
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/NameTransformerTest.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.base;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import io.airbyte.integrations.destination.NamingConventionTransformer;
+import io.airbyte.integrations.destination.StandardNameTransformer;
+import org.junit.jupiter.api.Test;
+
+class NameTransformerTest {
+
+ @Test
+ void testStandardSQLNaming() {
+ final NamingConventionTransformer namingResolver = new StandardNameTransformer();
+ assertEquals("identifier_name", namingResolver.getIdentifier("identifier_name"));
+ assertEquals("iDenTiFieR_name", namingResolver.getIdentifier("iDenTiFieR_name"));
+ assertEquals("__identifier_name", namingResolver.getIdentifier("__identifier_name"));
+ assertEquals("IDENTIFIER_NAME", namingResolver.getIdentifier("IDENTIFIER_NAME"));
+ assertEquals("123identifier_name", namingResolver.getIdentifier("123identifier_name"));
+ assertEquals("i0d0e0n0t0i0f0i0e0r0n0a0m0e", namingResolver.getIdentifier("i0d0e0n0t0i0f0i0e0r0n0a0m0e"));
+ assertEquals("_identifier_name", namingResolver.getIdentifier(",identifier+name"));
+ assertEquals("identifier_name", namingResolver.getIdentifier("identifiêr name"));
+ assertEquals("a_unicode_name__", namingResolver.getIdentifier("a_unicode_name_文"));
+ assertEquals("identifier__name__", namingResolver.getIdentifier("identifier__name__"));
+ assertEquals("identifier_name_weee", namingResolver.getIdentifier("identifier-name.weee"));
+ assertEquals("_identifier_name_", namingResolver.getIdentifier("\"identifier name\""));
+ assertEquals("identifier_name", namingResolver.getIdentifier("identifier name"));
+ assertEquals("identifier_", namingResolver.getIdentifier("identifier%"));
+ assertEquals("_identifier_", namingResolver.getIdentifier("`identifier`"));
+
+ assertEquals("_airbyte_raw_identifier_name", namingResolver.getRawTableName("identifier_name"));
+ }
+
+ // Temporarily disabling the behavior of the StandardNameTransformer, see (issue #1785)
+ // @Test
+ void testExtendedSQLNaming() {
+ final NamingConventionTransformer namingResolver = new StandardNameTransformer();
+ assertEquals("identifier_name", namingResolver.getIdentifier("identifier_name"));
+ assertEquals("iDenTiFieR_name", namingResolver.getIdentifier("iDenTiFieR_name"));
+ assertEquals("__identifier_name", namingResolver.getIdentifier("__identifier_name"));
+ assertEquals("IDENTIFIER_NAME", namingResolver.getIdentifier("IDENTIFIER_NAME"));
+ assertEquals("\"123identifier_name\"", namingResolver.getIdentifier("123identifier_name"));
+ assertEquals("i0d0e0n0t0i0f0i0e0r0n0a0m0e", namingResolver.getIdentifier("i0d0e0n0t0i0f0i0e0r0n0a0m0e"));
+ assertEquals("\",identifier+name\"", namingResolver.getIdentifier(",identifier+name"));
+ assertEquals("\"identifiêr name\"", namingResolver.getIdentifier("identifiêr name"));
+ assertEquals("\"a_unicode_name_文\"", namingResolver.getIdentifier("a_unicode_name_文"));
+ assertEquals("identifier__name__", namingResolver.getIdentifier("identifier__name__"));
+ assertEquals("\"identifier-name.weee\"", namingResolver.getIdentifier("identifier-name.weee"));
+ assertEquals("\"\"identifier name\"\"", namingResolver.getIdentifier("\"identifier name\""));
+ assertEquals("\"identifier name\"", namingResolver.getIdentifier("identifier name"));
+ assertEquals("\"identifier%\"", namingResolver.getIdentifier("identifier%"));
+ assertEquals("\"`identifier`\"", namingResolver.getIdentifier("`identifier`"));
+
+ assertEquals("_airbyte_raw_identifier_name", namingResolver.getRawTableName("identifier_name"));
+ assertEquals("\"_airbyte_raw_identifiêr name\"", namingResolver.getRawTableName("identifiêr name"));
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/normalization/NormalizationLogParserTest.java b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/normalization/NormalizationLogParserTest.java
new file mode 100644
index 0000000000000..44c9dc74f5856
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/normalization/NormalizationLogParserTest.java
@@ -0,0 +1,122 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.base.normalization;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import io.airbyte.integrations.destination.normalization.NormalizationLogParser;
+import io.airbyte.protocol.models.AirbyteErrorTraceMessage;
+import io.airbyte.protocol.models.AirbyteErrorTraceMessage.FailureType;
+import io.airbyte.protocol.models.AirbyteLogMessage;
+import io.airbyte.protocol.models.AirbyteLogMessage.Level;
+import io.airbyte.protocol.models.AirbyteMessage;
+import io.airbyte.protocol.models.AirbyteMessage.Type;
+import io.airbyte.protocol.models.AirbyteTraceMessage;
+import java.io.BufferedReader;
+import java.io.ByteArrayInputStream;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+class NormalizationLogParserTest {
+
+ private NormalizationLogParser parser;
+
+ @BeforeEach
+ void setup() {
+ parser = new NormalizationLogParser();
+ }
+
+ @Test
+ void testWrapNonJsonLogs() {
+ runTest(
+ """
+ foo
+ bar
+ [error] oh no
+ asdf
+ [error] qwer
+ """,
+ List.of(
+ logMessage(Level.INFO, "foo"),
+ logMessage(Level.INFO, "bar"),
+ logMessage(Level.INFO, "[error] oh no"),
+ logMessage(Level.INFO, "asdf"),
+ logMessage(Level.INFO, "[error] qwer")),
+ List.of(
+ "[error] oh no",
+ "[error] qwer"));
+ }
+
+ @Test
+ void testWrapJsonLogs() {
+ runTest(
+ """
+ {"code": "A001", "data": {"v": "=1.0.9"}, "invocation_id": "ed2017da-965d-406b-8fa1-07fb7c19fd14", "level": "info", "log_version": 1, "msg": "Running with dbt=1.0.9", "node_info": {}, "pid": 55, "thread_name": "MainThread", "ts": "2023-04-11T16:08:54.781886Z", "type": "log_line"}
+ {"code": "A001", "data": {"v": "=1.0.9"}, "invocation_id": "ed2017da-965d-406b-8fa1-07fb7c19fd14", "level": "error", "log_version": 1, "msg": "oh no", "node_info": {}, "pid": 55, "thread_name": "MainThread", "ts": "2023-04-11T16:08:54.781886Z", "type": "log_line"}
+ {"type": "TRACE", "trace": {"type": "ERROR", "emitted_at": 1.681766805198E12, "error": {"failure_type": "system_error", "message": "uh oh", "stack_trace": "normalization blew up", "internal_message": "normalization blew up with more detail"}}}
+ """,
+ List.of(
+ logMessage(Level.INFO, "Running with dbt=1.0.9"),
+ logMessage(Level.ERROR, "oh no"),
+ new AirbyteMessage()
+ .withType(Type.TRACE)
+ .withTrace(new AirbyteTraceMessage()
+ .withType(AirbyteTraceMessage.Type.ERROR)
+ .withEmittedAt(1.681766805198E12)
+ .withError(new AirbyteErrorTraceMessage()
+ .withFailureType(FailureType.SYSTEM_ERROR)
+ .withMessage("uh oh")
+ .withStackTrace("normalization blew up")
+ .withInternalMessage("normalization blew up with more detail")))),
+ List.of(
+ "oh no"));
+ }
+
+ @Test
+ void testWeirdLogs() {
+ runTest(
+ """
+ null
+ "null"
+ {"msg": "message with no level", "type": "log_line"}
+ {"level": "info", "type": "log_line"}
+ {"level": "error", "type": "log_line"}
+ """,
+ List.of(
+ logMessage(Level.INFO, "null"),
+ logMessage(Level.INFO, "\"null\""),
+ logMessage(Level.INFO, "{\n \"msg\" : \"message with no level\",\n \"type\" : \"log_line\"\n}"),
+ logMessage(Level.INFO, ""),
+ logMessage(Level.ERROR, "")),
+ List.of(
+ ""));
+ }
+
+ private void runTest(String rawLogs, List expectedMessages, List expectedDbtErrors) {
+ final List messages = parser.create(new BufferedReader(
+ new InputStreamReader(
+ new ByteArrayInputStream(
+ rawLogs.getBytes(StandardCharsets.UTF_8)),
+ StandardCharsets.UTF_8)))
+ .toList();
+
+ assertEquals(
+ expectedMessages,
+ messages);
+ assertEquals(expectedDbtErrors, parser.getDbtErrors());
+ }
+
+ private AirbyteMessage logMessage(Level level, String message) {
+ return new AirbyteMessage()
+ .withType(Type.LOG)
+ .withLog(new AirbyteLogMessage()
+ .withLevel(level)
+ .withMessage(message));
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/ssh/SshTunnelTest.java b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/ssh/SshTunnelTest.java
new file mode 100644
index 0000000000000..8f5f1a003ecc8
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/base/ssh/SshTunnelTest.java
@@ -0,0 +1,216 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.base.ssh;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.airbyte.commons.json.Jsons;
+import io.airbyte.integrations.base.ssh.SshTunnel.TunnelMethod;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyPair;
+import java.security.PrivateKey;
+import java.security.PublicKey;
+import java.util.Arrays;
+import org.apache.sshd.client.SshClient;
+import org.apache.sshd.client.session.ClientSession;
+import org.apache.sshd.common.util.security.SecurityUtils;
+import org.apache.sshd.common.util.security.eddsa.EdDSASecurityProviderRegistrar;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+class SshTunnelTest {
+
+ private static final String SSH_ED25519_PRIVATE_KEY = "-----BEGIN OPENSSH PRIVATE KEY-----\\n"
+ + "b3BlbnNzaC1rZXktdjEAAAAABG5vbmUAAAAEbm9uZQAAAAAAAAABAAAAMwAAAAtzc2gtZW\\n"
+ + "QyNTUxOQAAACDbBP+5jmEtjh1JvhzVQsvvTC2IQrX6P68XzrV7ZbnGsQAAAKBgtw9/YLcP\\n"
+ + "fwAAAAtzc2gtZWQyNTUxOQAAACDbBP+5jmEtjh1JvhzVQsvvTC2IQrX6P68XzrV7ZbnGsQ\\n"
+ + "AAAEAaKYn22N1O78HfdG22C7hcG2HiezKMzlq4JTdgYG1DstsE/7mOYS2OHUm+HNVCy+9M\\n"
+ + "LYhCtfo/rxfOtXtlucaxAAAAHHRmbG9yZXNfZHQwMUB0ZmxvcmVzX2R0MDEtUEMB\\n"
+ + "-----END OPENSSH PRIVATE KEY-----";
+ private static final String SSH_RSA_PRIVATE_KEY = "-----BEGIN OPENSSH PRIVATE KEY-----\\n"
+ + "b3BlbnNzaC1rZXktdjEAAAAABG5vbmUAAAAEbm9uZQAAAAAAAAABAAABlwAAAAdzc2gtcn\\n"
+ + "NhAAAAAwEAAQAAAYEAuFjfTMS6BrgoxaQe9i83y6CdGH3xJIwc1Wy+11ibWAFcQ6khX/x0\\n"
+ + "M+JnJaSCs/hxiDE4afHscP3HzVQC699IgKwyAPaG0ZG+bLhxWAm4E79P7Yssj7imhTqr0A\\n"
+ + "DZDO23CCOagHvfdg1svnBhk1ih14GMGKRFCS27CLgholIOeogOyH7b3Jaqy9LtICiE054e\\n"
+ + "jwdaZdwWU08kxMO4ItdxNasCPC5uQiaXIzWFysG0mLk7WWc8WyuQHneQFl3Qu6p/rWJz4i\\n"
+ + "seea5CBL5s1DIyCyo/jgN5/oOWOciPUl49mDLleCzYTDnWqX43NK9A87unNeuA95Fk9akH\\n"
+ + "8QH4hKBCzpHhsh4U3Ys/l9Q5NmnyBrtFWBY2n13ZftNA/Ms+Hsh6V3eyJW0rIFY2/UM4XA\\n"
+ + "YyD6MEOlvFAQjxC6EbqfkrC6FQgH3I2wAtIDqEk2j79vfIIDdzp8otWjIQsApX55j+kKio\\n"
+ + "sY8YTXb9sLWuEdpSd/AN3iQ8HwIceyTulaKn7rTBAAAFkMwDTyPMA08jAAAAB3NzaC1yc2\\n"
+ + "EAAAGBALhY30zEuga4KMWkHvYvN8ugnRh98SSMHNVsvtdYm1gBXEOpIV/8dDPiZyWkgrP4\\n"
+ + "cYgxOGnx7HD9x81UAuvfSICsMgD2htGRvmy4cVgJuBO/T+2LLI+4poU6q9AA2Qzttwgjmo\\n"
+ + "B733YNbL5wYZNYodeBjBikRQktuwi4IaJSDnqIDsh+29yWqsvS7SAohNOeHo8HWmXcFlNP\\n"
+ + "JMTDuCLXcTWrAjwubkImlyM1hcrBtJi5O1lnPFsrkB53kBZd0Luqf61ic+IrHnmuQgS+bN\\n"
+ + "QyMgsqP44Def6DljnIj1JePZgy5Xgs2Ew51ql+NzSvQPO7pzXrgPeRZPWpB/EB+ISgQs6R\\n"
+ + "4bIeFN2LP5fUOTZp8ga7RVgWNp9d2X7TQPzLPh7Ield3siVtKyBWNv1DOFwGMg+jBDpbxQ\\n"
+ + "EI8QuhG6n5KwuhUIB9yNsALSA6hJNo+/b3yCA3c6fKLVoyELAKV+eY/pCoqLGPGE12/bC1\\n"
+ + "rhHaUnfwDd4kPB8CHHsk7pWip+60wQAAAAMBAAEAAAGAXw+dHpY3o21lwP0v5h1VNVD+kX\\n"
+ + "moVwNVfw0ToDKV8JzK+i0GA9xIA9VVAUlDCREtYmCXSbKyDVYgqRYQZ5d9aLTjGDIINZtl\\n"
+ + "SeUWtaJVZQF7cvAYq4g5fmxR2vIE+zC9+Jl7e5PlGJg1okKLXpMO6fVoy/AxlVkaoJVq6q\\n"
+ + "xLwQ3WKbeZIrgjHPYIx1N9oy5fbbwJ9oq2jIE8YabXlkfonhcwEN6UhtIlj8dy1apruXGT\\n"
+ + "VDfzHMRrDfrzt0TrdUqmqgo/istP89sggtkJ8uuPtkBFHTjao8MiBsshy1iDVbIno9gDbJ\\n"
+ + "JgYyunmSgEjEZpp09+mkgwfZO3/RDLRPF1SRAGBNy27CH8/bh9gAVRhAPi0GLclNi292Ya\\n"
+ + "NrGvjMcRlYAsWL3mZ9aTbv0j7Qi8qdWth+rZ+tBmNToUVVl5iLxifgo0kjiXAehZB1LaQV\\n"
+ + "yuMXlXOGmt9V2/DPACA9getQJQONxrLAcgHdjMiuwD8r7d+m/kE4+cOTakOlzrfrwBAAAA\\n"
+ + "wQCVTQTvuyBW3JemMPtRLifQqdwMGRPokm5nTn+JSJQvg+dNpL7hC0k3IesKs63gxuuHoq\\n"
+ + "4q1xkMmCMvihT8oVlxrezEjsO/QMCxe6Sr9eMfHAjrdPeHsPaf9oOgG9vEEH9dEilHpnlb\\n"
+ + "97Vyl9EHm1iahONM1gWdXkPjIfnQzYPvSLZPtBBSI0XBjCTifMnCRgd3s2bdm7kh+7XA+C\\n"
+ + "rX62WfPIJKL+OhMIf+ED4HBJTd/vU34Vk73yvqHzqel0ZQnRoAAADBAOGSm6TNBptV7S5P\\n"
+ + "wT3BhGYSm35/7nCFTilyfy5/8EUmifUFittRIvgDoXBWeZEwvqIiQ55iX9mYmNmb0KbPCw\\n"
+ + "cqN/BtXWItAvyTDZ6PeI2m2aUj+rW2R3ZXEsBjgaNRtbPyMKQ69xtKRvHtNZNfgjpRQ4is\\n"
+ + "lbufhAK1YbUxrlfKaBGOcGyR7DNmUUUN6nptQbpOr1HQc5DOH17HIDnRPs44HIws3/apww\\n"
+ + "RBIjjy6GQNfJ/Ge8N4pxGoLl1qKO8xoQAAAMEA0Tat/E5mSsgjCgmFja/jOZJcrzZHwrPT\\n"
+ + "3NEbuAMQ/L3atKEINypmpJfjIAvNljKJwSUDMEWvs8qj8cSGCrtkcAv1YSm697TL2oC9HU\\n"
+ + "CFoOJAkH1X2CGTgHlR9it3j4aRJ3dXdL2k7aeoGXObfRWqBNPj0LOOZs64RA6scGAzo6MR\\n"
+ + "5WlcOxfV1wZuaM0fOd+PBmIlFEE7Uf6AY/UahBAxaFV2+twgK9GCDcu1t4Ye9wZ9kZ4Nal\\n"
+ + "0fkKD4uN4DRO8hAAAAFm10dWhhaUBrYnAxLWxocC1hMTQ1MzMBAgME\\n"
+ + "-----END OPENSSH PRIVATE KEY-----";
+ private static final String HOST_PORT_CONFIG =
+ "{\"ssl\":true,\"host\":\"fakehost.com\",\"port\":5432,\"schema\":\"public\",\"database\":\"postgres\",\"password\":\"\",\"username\":\"postgres\",\"tunnel_method\":{\"ssh_key\":\""
+ + "%s"
+ + "\",\"tunnel_host\":\"faketunnel.com\",\"tunnel_port\":22,\"tunnel_user\":\"ec2-user\",\"tunnel_method\":\"SSH_KEY_AUTH\"}}";
+
+ private static final String URL_CONFIG_WITH_PORT =
+ "{\"ssl\":true,\"endpoint\":\"http://fakehost.com:9090/service\",\"password\":\"\",\"username\":\"restuser\",\"tunnel_method\":{\"ssh_key\":\""
+ + "%s"
+ + "\",\"tunnel_host\":\"faketunnel.com\",\"tunnel_port\":22,\"tunnel_user\":\"ec2-user\",\"tunnel_method\":\"SSH_KEY_AUTH\"}}";
+
+ private static final String URL_CONFIG_NO_PORT =
+ "{\"ssl\":true,\"endpoint\":\"http://fakehost.com/service\",\"password\":\"\",\"username\":\"restuser\",\"tunnel_method\":{\"ssh_key\":\""
+ + "%s"
+ + "\",\"tunnel_host\":\"faketunnel.com\",\"tunnel_port\":22,\"tunnel_user\":\"ec2-user\",\"tunnel_method\":\"SSH_KEY_AUTH\"}}";
+
+ /**
+ * This test verifies that OpenSsh correctly replaces values in connector configuration in a spec
+ * with host/port config and in a spec with endpoint URL config
+ *
+ * @param configString
+ * @throws Exception
+ */
+ @ParameterizedTest
+ @ValueSource(strings = {HOST_PORT_CONFIG, URL_CONFIG_WITH_PORT, URL_CONFIG_NO_PORT})
+ public void testConfigInTunnel(final String configString) throws Exception {
+ final JsonNode config = (new ObjectMapper()).readTree(String.format(configString, SSH_RSA_PRIVATE_KEY));
+ String endPointURL = Jsons.getStringOrNull(config, "endpoint");
+ final SshTunnel sshTunnel = new SshTunnel(
+ config,
+ endPointURL == null ? Arrays.asList(new String[] {"host"}) : null,
+ endPointURL == null ? Arrays.asList(new String[] {"port"}) : null,
+ endPointURL == null ? null : "endpoint",
+ endPointURL,
+ TunnelMethod.SSH_KEY_AUTH,
+ "faketunnel.com",
+ 22,
+ "tunnelUser",
+ SSH_RSA_PRIVATE_KEY,
+ "tunnelUserPassword",
+ endPointURL == null ? "fakeHost.com" : null,
+ endPointURL == null ? 5432 : 0) {
+
+ @Override
+ ClientSession openTunnel(final SshClient client) {
+ tunnelLocalPort = 8080;
+ return null; // Prevent tunnel from attempting to connect
+ }
+
+ };
+
+ final JsonNode configInTunnel = sshTunnel.getConfigInTunnel();
+ if (endPointURL == null) {
+ assertTrue(configInTunnel.has("port"));
+ assertTrue(configInTunnel.has("host"));
+ assertFalse(configInTunnel.has("endpoint"));
+ assertEquals(8080, configInTunnel.get("port").asInt());
+ assertEquals("127.0.0.1", configInTunnel.get("host").asText());
+ } else {
+ assertFalse(configInTunnel.has("port"));
+ assertFalse(configInTunnel.has("host"));
+ assertTrue(configInTunnel.has("endpoint"));
+ assertEquals("http://127.0.0.1:8080/service", configInTunnel.get("endpoint").asText());
+ }
+ }
+
+ /**
+ * This test verifies that SshTunnel correctly extracts private key pairs from keys formatted as
+ * EdDSA and OpenSSH
+ *
+ * @param privateKey
+ * @throws Exception
+ */
+ @ParameterizedTest
+ @ValueSource(strings = {SSH_ED25519_PRIVATE_KEY, SSH_RSA_PRIVATE_KEY})
+ public void getKeyPair(final String privateKey) throws Exception {
+ final JsonNode config = (new ObjectMapper()).readTree(String.format(HOST_PORT_CONFIG, privateKey));
+ final SshTunnel sshTunnel = new SshTunnel(
+ config,
+ Arrays.asList(new String[] {"host"}),
+ Arrays.asList(new String[] {"port"}),
+ null,
+ null,
+ TunnelMethod.SSH_KEY_AUTH,
+ "faketunnel.com",
+ 22,
+ "tunnelUser",
+ privateKey,
+ "tunnelUserPassword",
+ "fakeHost.com",
+ 5432) {
+
+ @Override
+ ClientSession openTunnel(final SshClient client) {
+ return null; // Prevent tunnel from attempting to connect
+ }
+
+ };
+
+ final KeyPair authKeyPair = sshTunnel.getPrivateKeyPair();
+ assertNotNull(authKeyPair);// actually, all is good if there is no exception on previous line
+ }
+
+ /**
+ * This test verifies that 'net.i2p.crypto:eddsa' is present and EdDSA is supported. If
+ * net.i2p.crypto:eddsa will be removed from project, then will be thrown: generator not correctly
+ * initialized
+ *
+ * @throws Exception
+ */
+ @Test
+ public void edDsaIsSupported() throws Exception {
+ final var keygen = SecurityUtils.getKeyPairGenerator("EdDSA");
+ final String message = "hello world";
+ final KeyPair keyPair = keygen.generateKeyPair();
+
+ final byte[] signedMessage = sign(keyPair.getPrivate(), message);
+
+ assertTrue(new EdDSASecurityProviderRegistrar().isSupported());
+ assertTrue(verify(keyPair.getPublic(), signedMessage, message));
+ }
+
+ private byte[] sign(final PrivateKey privateKey, final String message) throws Exception {
+ final var signature = SecurityUtils.getSignature("NONEwithEdDSA");
+ signature.initSign(privateKey);
+
+ signature.update(message.getBytes(StandardCharsets.UTF_8));
+
+ return signature.sign();
+ }
+
+ private boolean verify(final PublicKey publicKey, final byte[] signed, final String message)
+ throws Exception {
+ final var signature = SecurityUtils.getSignature("NONEwithEdDSA");
+ signature.initVerify(publicKey);
+
+ signature.update(message.getBytes(StandardCharsets.UTF_8));
+
+ return signature.verify(signed);
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/destination/buffered_stream_consumer/BufferedStreamConsumerTest.java b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/destination/buffered_stream_consumer/BufferedStreamConsumerTest.java
new file mode 100644
index 0000000000000..11ef6402e14ec
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/destination/buffered_stream_consumer/BufferedStreamConsumerTest.java
@@ -0,0 +1,417 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.buffered_stream_consumer;
+
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoInteractions;
+import static org.mockito.Mockito.when;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import io.airbyte.commons.functional.CheckedFunction;
+import io.airbyte.commons.json.Jsons;
+import io.airbyte.integrations.destination.record_buffer.InMemoryRecordBufferingStrategy;
+import io.airbyte.protocol.models.Field;
+import io.airbyte.protocol.models.JsonSchemaType;
+import io.airbyte.protocol.models.v0.AirbyteMessage;
+import io.airbyte.protocol.models.v0.AirbyteMessage.Type;
+import io.airbyte.protocol.models.v0.AirbyteRecordMessage;
+import io.airbyte.protocol.models.v0.AirbyteStateMessage;
+import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair;
+import io.airbyte.protocol.models.v0.CatalogHelpers;
+import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog;
+import java.time.Duration;
+import java.time.Instant;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.commons.lang.RandomStringUtils;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+public class BufferedStreamConsumerTest {
+
+ private static final String SCHEMA_NAME = "public";
+ private static final String STREAM_NAME = "id_and_name";
+ private static final String STREAM_NAME2 = STREAM_NAME + 2;
+ private static final int PERIODIC_BUFFER_FREQUENCY = 5;
+ private static final ConfiguredAirbyteCatalog CATALOG = new ConfiguredAirbyteCatalog().withStreams(List.of(
+ CatalogHelpers.createConfiguredAirbyteStream(
+ STREAM_NAME,
+ SCHEMA_NAME,
+ Field.of("id", JsonSchemaType.NUMBER),
+ Field.of("name", JsonSchemaType.STRING)),
+ CatalogHelpers.createConfiguredAirbyteStream(
+ STREAM_NAME2,
+ SCHEMA_NAME,
+ Field.of("id", JsonSchemaType.NUMBER),
+ Field.of("name", JsonSchemaType.STRING))));
+
+ private static final AirbyteMessage STATE_MESSAGE1 = new AirbyteMessage()
+ .withType(Type.STATE)
+ .withState(new AirbyteStateMessage().withData(Jsons.jsonNode(ImmutableMap.of("state_message_id", 1))));
+ private static final AirbyteMessage STATE_MESSAGE2 = new AirbyteMessage()
+ .withType(Type.STATE)
+ .withState(new AirbyteStateMessage().withData(Jsons.jsonNode(ImmutableMap.of("state_message_id", 2))));
+
+ private BufferedStreamConsumer consumer;
+ private OnStartFunction onStart;
+ private RecordWriter recordWriter;
+ private OnCloseFunction onClose;
+ private CheckedFunction isValidRecord;
+ private Consumer outputRecordCollector;
+
+ @SuppressWarnings("unchecked")
+ @BeforeEach
+ void setup() throws Exception {
+ onStart = mock(OnStartFunction.class);
+ recordWriter = mock(RecordWriter.class);
+ onClose = mock(OnCloseFunction.class);
+ isValidRecord = mock(CheckedFunction.class);
+ outputRecordCollector = mock(Consumer.class);
+ consumer = new BufferedStreamConsumer(
+ outputRecordCollector,
+ onStart,
+ new InMemoryRecordBufferingStrategy(recordWriter, 1_000),
+ onClose,
+ CATALOG,
+ isValidRecord);
+
+ when(isValidRecord.apply(any())).thenReturn(true);
+ }
+
+ @Test
+ void test1StreamWith1State() throws Exception {
+ final List expectedRecords = generateRecords(1_000);
+
+ consumer.start();
+ consumeRecords(consumer, expectedRecords);
+ consumer.accept(STATE_MESSAGE1);
+ consumer.close();
+
+ verifyStartAndClose();
+
+ verifyRecords(STREAM_NAME, SCHEMA_NAME, expectedRecords);
+
+ verify(outputRecordCollector).accept(STATE_MESSAGE1);
+ }
+
+ @Test
+ void test1StreamWith2State() throws Exception {
+ final List expectedRecords = generateRecords(1_000);
+
+ consumer.start();
+ consumeRecords(consumer, expectedRecords);
+ consumer.accept(STATE_MESSAGE1);
+ consumer.accept(STATE_MESSAGE2);
+ consumer.close();
+
+ verifyStartAndClose();
+
+ verifyRecords(STREAM_NAME, SCHEMA_NAME, expectedRecords);
+
+ verify(outputRecordCollector, times(1)).accept(STATE_MESSAGE2);
+ }
+
+ @Test
+ void test1StreamWith0State() throws Exception {
+ final List expectedRecords = generateRecords(1_000);
+
+ consumer.start();
+ consumeRecords(consumer, expectedRecords);
+ consumer.close();
+
+ verifyStartAndClose();
+
+ verifyRecords(STREAM_NAME, SCHEMA_NAME, expectedRecords);
+ }
+
+ @Test
+ void test1StreamWithStateAndThenMoreRecordsBiggerThanBuffer() throws Exception {
+ final List expectedRecordsBatch1 = generateRecords(1_000);
+ final List expectedRecordsBatch2 = generateRecords(1_000);
+
+ consumer.start();
+ consumeRecords(consumer, expectedRecordsBatch1);
+ consumer.accept(STATE_MESSAGE1);
+ consumeRecords(consumer, expectedRecordsBatch2);
+ consumer.close();
+
+ verifyStartAndClose();
+
+ verifyRecords(STREAM_NAME, SCHEMA_NAME, expectedRecordsBatch1);
+ verifyRecords(STREAM_NAME, SCHEMA_NAME, expectedRecordsBatch2);
+
+ verify(outputRecordCollector).accept(STATE_MESSAGE1);
+ }
+
+ @Test
+ void test1StreamWithStateAndThenMoreRecordsSmallerThanBuffer() throws Exception {
+ final List expectedRecordsBatch1 = generateRecords(1_000);
+ final List expectedRecordsBatch2 = generateRecords(1_000);
+
+ // consumer with big enough buffered that we see both batches are flushed in one go.
+ final BufferedStreamConsumer consumer = new BufferedStreamConsumer(
+ outputRecordCollector,
+ onStart,
+ new InMemoryRecordBufferingStrategy(recordWriter, 10_000),
+ onClose,
+ CATALOG,
+ isValidRecord);
+
+ consumer.start();
+ consumeRecords(consumer, expectedRecordsBatch1);
+ consumer.accept(STATE_MESSAGE1);
+ consumeRecords(consumer, expectedRecordsBatch2);
+ consumer.close();
+
+ verifyStartAndClose();
+
+ final List expectedRecords = Lists.newArrayList(expectedRecordsBatch1, expectedRecordsBatch2)
+ .stream()
+ .flatMap(Collection::stream)
+ .collect(Collectors.toList());
+ verifyRecords(STREAM_NAME, SCHEMA_NAME, expectedRecords);
+
+ verify(outputRecordCollector).accept(STATE_MESSAGE1);
+ }
+
+ @Test
+ void testExceptionAfterOneStateMessage() throws Exception {
+ final List expectedRecordsBatch1 = generateRecords(1_000);
+ final List expectedRecordsBatch2 = generateRecords(1_000);
+ final List expectedRecordsBatch3 = generateRecords(1_000);
+
+ consumer.start();
+ consumeRecords(consumer, expectedRecordsBatch1);
+ consumer.accept(STATE_MESSAGE1);
+ consumeRecords(consumer, expectedRecordsBatch2);
+ when(isValidRecord.apply(any())).thenThrow(new IllegalStateException("induced exception"));
+ assertThrows(IllegalStateException.class, () -> consumer.accept(expectedRecordsBatch3.get(0)));
+ consumer.close();
+
+ verifyStartAndCloseFailure();
+
+ verifyRecords(STREAM_NAME, SCHEMA_NAME, expectedRecordsBatch1);
+
+ verify(outputRecordCollector).accept(STATE_MESSAGE1);
+ }
+
+ @Test
+ void testExceptionAfterNoStateMessages() throws Exception {
+ final List expectedRecordsBatch1 = generateRecords(1_000);
+ final List expectedRecordsBatch2 = generateRecords(1_000);
+ final List expectedRecordsBatch3 = generateRecords(1_000);
+
+ consumer.start();
+ consumeRecords(consumer, expectedRecordsBatch1);
+ consumeRecords(consumer, expectedRecordsBatch2);
+ when(isValidRecord.apply(any())).thenThrow(new IllegalStateException("induced exception"));
+ assertThrows(IllegalStateException.class, () -> consumer.accept(expectedRecordsBatch3.get(0)));
+ consumer.close();
+
+ verifyStartAndCloseFailure();
+
+ verifyRecords(STREAM_NAME, SCHEMA_NAME, expectedRecordsBatch1);
+
+ verifyNoInteractions(outputRecordCollector);
+ }
+
+ @Test
+ void testExceptionDuringOnClose() throws Exception {
+ doThrow(new IllegalStateException("induced exception")).when(onClose).accept(false);
+
+ final List expectedRecordsBatch1 = generateRecords(1_000);
+ final List expectedRecordsBatch2 = generateRecords(1_000);
+
+ consumer.start();
+ consumeRecords(consumer, expectedRecordsBatch1);
+ consumer.accept(STATE_MESSAGE1);
+ consumeRecords(consumer, expectedRecordsBatch2);
+ assertThrows(IllegalStateException.class, () -> consumer.close(), "Expected an error to be thrown on close");
+
+ verifyStartAndClose();
+
+ verifyRecords(STREAM_NAME, SCHEMA_NAME, expectedRecordsBatch1);
+
+ verify(outputRecordCollector).accept(STATE_MESSAGE1);
+ }
+
+ @Test
+ void test2StreamWith1State() throws Exception {
+ final List expectedRecordsStream1 = generateRecords(1_000);
+ final List expectedRecordsStream2 = expectedRecordsStream1
+ .stream()
+ .map(Jsons::clone)
+ .peek(m -> m.getRecord().withStream(STREAM_NAME2))
+ .collect(Collectors.toList());
+
+ consumer.start();
+ consumeRecords(consumer, expectedRecordsStream1);
+ consumer.accept(STATE_MESSAGE1);
+ consumeRecords(consumer, expectedRecordsStream2);
+ consumer.close();
+
+ verifyStartAndClose();
+
+ verifyRecords(STREAM_NAME, SCHEMA_NAME, expectedRecordsStream1);
+ verifyRecords(STREAM_NAME2, SCHEMA_NAME, expectedRecordsStream2);
+
+ verify(outputRecordCollector).accept(STATE_MESSAGE1);
+ }
+
+ @Test
+ void test2StreamWith2State() throws Exception {
+ final List expectedRecordsStream1 = generateRecords(1_000);
+ final List expectedRecordsStream2 = expectedRecordsStream1
+ .stream()
+ .map(Jsons::clone)
+ .peek(m -> m.getRecord().withStream(STREAM_NAME2))
+ .collect(Collectors.toList());
+
+ consumer.start();
+ consumeRecords(consumer, expectedRecordsStream1);
+ consumer.accept(STATE_MESSAGE1);
+ consumeRecords(consumer, expectedRecordsStream2);
+ consumer.accept(STATE_MESSAGE2);
+ consumer.close();
+
+ verifyStartAndClose();
+
+ verifyRecords(STREAM_NAME, SCHEMA_NAME, expectedRecordsStream1);
+ verifyRecords(STREAM_NAME2, SCHEMA_NAME, expectedRecordsStream2);
+
+ verify(outputRecordCollector, times(1)).accept(STATE_MESSAGE2);
+ }
+
+ // Periodic Buffer Flush Tests
+ @Test
+ void testSlowStreamReturnsState() throws Exception {
+ // generate records less than the default maxQueueSizeInBytes to confirm periodic flushing occurs
+ final List expectedRecordsStream1 = generateRecords(500L);
+ final List expectedRecordsStream1Batch2 = generateRecords(200L);
+
+ // Overrides flush frequency for testing purposes to 5 seconds
+ final BufferedStreamConsumer flushConsumer = getConsumerWithFlushFrequency();
+ flushConsumer.start();
+ consumeRecords(flushConsumer, expectedRecordsStream1);
+ flushConsumer.accept(STATE_MESSAGE1);
+ // NOTE: Sleeps process for 5 seconds, if tests are slow this can be updated to reduce slowdowns
+ TimeUnit.SECONDS.sleep(PERIODIC_BUFFER_FREQUENCY);
+ consumeRecords(flushConsumer, expectedRecordsStream1Batch2);
+ flushConsumer.close();
+
+ verifyStartAndClose();
+ // expects the records to be grouped because periodicBufferFlush occurs at the end of acceptTracked
+ verifyRecords(STREAM_NAME, SCHEMA_NAME,
+ Stream.concat(expectedRecordsStream1.stream(), expectedRecordsStream1Batch2.stream()).collect(Collectors.toList()));
+ verify(outputRecordCollector).accept(STATE_MESSAGE1);
+ }
+
+ @Test
+ void testSlowStreamReturnsMultipleStates() throws Exception {
+ // generate records less than the default maxQueueSizeInBytes to confirm periodic flushing occurs
+ final List expectedRecordsStream1 = generateRecords(500L);
+ final List expectedRecordsStream1Batch2 = generateRecords(200L);
+ // creates records equal to size that triggers buffer flush
+ final List expectedRecordsStream1Batch3 = generateRecords(1_000L);
+
+ // Overrides flush frequency for testing purposes to 5 seconds
+ final BufferedStreamConsumer flushConsumer = getConsumerWithFlushFrequency();
+ flushConsumer.start();
+ consumeRecords(flushConsumer, expectedRecordsStream1);
+ flushConsumer.accept(STATE_MESSAGE1);
+ // NOTE: Sleeps process for 5 seconds, if tests are slow this can be updated to reduce slowdowns
+ TimeUnit.SECONDS.sleep(PERIODIC_BUFFER_FREQUENCY);
+ consumeRecords(flushConsumer, expectedRecordsStream1Batch2);
+ consumeRecords(flushConsumer, expectedRecordsStream1Batch3);
+ flushConsumer.accept(STATE_MESSAGE2);
+ flushConsumer.close();
+
+ verifyStartAndClose();
+ // expects the records to be grouped because periodicBufferFlush occurs at the end of acceptTracked
+ verifyRecords(STREAM_NAME, SCHEMA_NAME,
+ Stream.concat(expectedRecordsStream1.stream(), expectedRecordsStream1Batch2.stream()).collect(Collectors.toList()));
+ verifyRecords(STREAM_NAME, SCHEMA_NAME, expectedRecordsStream1Batch3);
+ // expects two STATE messages returned since one will be flushed after periodic flushing occurs
+ // and the other after buffer has been filled
+ verify(outputRecordCollector).accept(STATE_MESSAGE1);
+ verify(outputRecordCollector).accept(STATE_MESSAGE2);
+ }
+
+ private BufferedStreamConsumer getConsumerWithFlushFrequency() {
+ final BufferedStreamConsumer flushFrequencyConsumer = new BufferedStreamConsumer(
+ outputRecordCollector,
+ onStart,
+ new InMemoryRecordBufferingStrategy(recordWriter, 10_000),
+ onClose,
+ CATALOG,
+ isValidRecord,
+ Duration.ofSeconds(PERIODIC_BUFFER_FREQUENCY));
+ return flushFrequencyConsumer;
+ }
+
+ private void verifyStartAndClose() throws Exception {
+ verify(onStart).call();
+ verify(onClose).accept(false);
+ }
+
+ /** Indicates that a failure occurred while consuming AirbyteMessages */
+ private void verifyStartAndCloseFailure() throws Exception {
+ verify(onStart).call();
+ verify(onClose).accept(true);
+ }
+
+ private static void consumeRecords(final BufferedStreamConsumer consumer, final Collection records) {
+ records.forEach(m -> {
+ try {
+ consumer.accept(m);
+ } catch (final Exception e) {
+ throw new RuntimeException(e);
+ }
+ });
+ }
+
+ // NOTE: Generates records at chunks of 160 bytes
+ private static List generateRecords(final long targetSizeInBytes) {
+ final List output = Lists.newArrayList();
+ long bytesCounter = 0;
+ for (int i = 0;; i++) {
+ final JsonNode payload =
+ Jsons.jsonNode(ImmutableMap.of("id", RandomStringUtils.randomAlphabetic(7), "name", "human " + String.format("%8d", i)));
+ final long sizeInBytes = RecordSizeEstimator.getStringByteSize(payload);
+ bytesCounter += sizeInBytes;
+ final AirbyteMessage airbyteMessage = new AirbyteMessage()
+ .withType(Type.RECORD)
+ .withRecord(new AirbyteRecordMessage()
+ .withStream(STREAM_NAME)
+ .withNamespace(SCHEMA_NAME)
+ .withEmittedAt(Instant.now().toEpochMilli())
+ .withData(payload));
+ if (bytesCounter > targetSizeInBytes) {
+ break;
+ } else {
+ output.add(airbyteMessage);
+ }
+ }
+ return output;
+ }
+
+ private void verifyRecords(final String streamName, final String namespace, final Collection expectedRecords) throws Exception {
+ verify(recordWriter).accept(
+ new AirbyteStreamNameNamespacePair(streamName, namespace),
+ expectedRecords.stream().map(AirbyteMessage::getRecord).collect(Collectors.toList()));
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/destination/buffered_stream_consumer/RecordSizeEstimatorTest.java b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/destination/buffered_stream_consumer/RecordSizeEstimatorTest.java
new file mode 100644
index 0000000000000..478398d12aa13
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/destination/buffered_stream_consumer/RecordSizeEstimatorTest.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.buffered_stream_consumer;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import io.airbyte.commons.json.Jsons;
+import io.airbyte.protocol.models.v0.AirbyteRecordMessage;
+import org.junit.jupiter.api.Test;
+
+class RecordSizeEstimatorTest {
+
+ private static final JsonNode DATA_0 = Jsons.deserialize("{}");
+ private static final JsonNode DATA_1 = Jsons.deserialize("{ \"field1\": true }");
+ private static final JsonNode DATA_2 = Jsons.deserialize("{ \"field1\": 10000 }");
+ private static final long DATA_0_SIZE = RecordSizeEstimator.getStringByteSize(DATA_0);
+ private static final long DATA_1_SIZE = RecordSizeEstimator.getStringByteSize(DATA_1);
+ private static final long DATA_2_SIZE = RecordSizeEstimator.getStringByteSize(DATA_2);
+
+ @Test
+ public void testPeriodicSampling() {
+ // the estimate performs a size sampling every 3 records
+ final RecordSizeEstimator sizeEstimator = new RecordSizeEstimator(3);
+ final String stream = "stream";
+ final AirbyteRecordMessage record0 = new AirbyteRecordMessage().withStream(stream).withData(DATA_0);
+ final AirbyteRecordMessage record1 = new AirbyteRecordMessage().withStream(stream).withData(DATA_1);
+ final AirbyteRecordMessage record2 = new AirbyteRecordMessage().withStream(stream).withData(DATA_2);
+
+ // sample record message 1
+ final long firstEstimation = DATA_1_SIZE;
+ assertEquals(firstEstimation, sizeEstimator.getEstimatedByteSize(record1));
+ // next two calls return the first sampling result
+ assertEquals(firstEstimation, sizeEstimator.getEstimatedByteSize(record0));
+ assertEquals(firstEstimation, sizeEstimator.getEstimatedByteSize(record0));
+
+ // sample record message 2
+ final long secondEstimation = firstEstimation / 2 + DATA_2_SIZE / 2;
+ assertEquals(secondEstimation, sizeEstimator.getEstimatedByteSize(record2));
+ // next two calls return the second sampling result
+ assertEquals(secondEstimation, sizeEstimator.getEstimatedByteSize(record0));
+ assertEquals(secondEstimation, sizeEstimator.getEstimatedByteSize(record0));
+
+ // sample record message 1
+ final long thirdEstimation = secondEstimation / 2 + DATA_1_SIZE / 2;
+ assertEquals(thirdEstimation, sizeEstimator.getEstimatedByteSize(record1));
+ // next two calls return the first sampling result
+ assertEquals(thirdEstimation, sizeEstimator.getEstimatedByteSize(record0));
+ assertEquals(thirdEstimation, sizeEstimator.getEstimatedByteSize(record0));
+ }
+
+ @Test
+ public void testDifferentEstimationPerStream() {
+ final RecordSizeEstimator sizeEstimator = new RecordSizeEstimator();
+ final AirbyteRecordMessage record0 = new AirbyteRecordMessage().withStream("stream1").withData(DATA_0);
+ final AirbyteRecordMessage record1 = new AirbyteRecordMessage().withStream("stream2").withData(DATA_1);
+ final AirbyteRecordMessage record2 = new AirbyteRecordMessage().withStream("stream3").withData(DATA_2);
+ assertEquals(DATA_0_SIZE, sizeEstimator.getEstimatedByteSize(record0));
+ assertEquals(DATA_1_SIZE, sizeEstimator.getEstimatedByteSize(record1));
+ assertEquals(DATA_2_SIZE, sizeEstimator.getEstimatedByteSize(record2));
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/destination/dest_state_lifecycle_manager/DefaultDestStateLifecycleManagerTest.java b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/destination/dest_state_lifecycle_manager/DefaultDestStateLifecycleManagerTest.java
new file mode 100644
index 0000000000000..afa85a50ae785
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/destination/dest_state_lifecycle_manager/DefaultDestStateLifecycleManagerTest.java
@@ -0,0 +1,124 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.dest_state_lifecycle_manager;
+
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+import io.airbyte.protocol.models.v0.AirbyteMessage;
+import io.airbyte.protocol.models.v0.AirbyteMessage.Type;
+import io.airbyte.protocol.models.v0.AirbyteStateMessage;
+import io.airbyte.protocol.models.v0.AirbyteStateMessage.AirbyteStateType;
+import io.airbyte.protocol.models.v0.AirbyteStreamState;
+import io.airbyte.protocol.models.v0.StreamDescriptor;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+class DefaultDestStateLifecycleManagerTest {
+
+ private static final AirbyteMessage UNSET_TYPE_MESSAGE = new AirbyteMessage()
+ .withType(Type.STATE)
+ .withState(new AirbyteStateMessage());
+ private static final AirbyteMessage LEGACY_MESSAGE = new AirbyteMessage()
+ .withType(Type.STATE)
+ .withState(new AirbyteStateMessage().withType(AirbyteStateType.LEGACY));
+ private static final AirbyteMessage GLOBAL_MESSAGE = new AirbyteMessage()
+ .withType(Type.STATE)
+ .withState(new AirbyteStateMessage().withType(AirbyteStateType.GLOBAL));
+ private static final AirbyteMessage STREAM_MESSAGE = new AirbyteMessage()
+ .withType(Type.STATE)
+ .withState(new AirbyteStateMessage()
+ .withType(AirbyteStateType.STREAM)
+ .withStream(new AirbyteStreamState().withStreamDescriptor(new StreamDescriptor().withName("users"))));
+
+ private DestStateLifecycleManager mgr1;
+ private DestStateLifecycleManager singleStateMgr;
+ private DestStateLifecycleManager streamMgr;
+
+ @BeforeEach
+ void setup() {
+ singleStateMgr = mock(DestStateLifecycleManager.class);
+ streamMgr = mock(DestStateLifecycleManager.class);
+ mgr1 = new DefaultDestStateLifecycleManager(singleStateMgr, streamMgr);
+ }
+
+ @Test
+ void testFailsOnIncompatibleStates() {
+ final DefaultDestStateLifecycleManager manager1 = new DefaultDestStateLifecycleManager(singleStateMgr, streamMgr);
+ manager1.addState(UNSET_TYPE_MESSAGE);
+ manager1.addState(UNSET_TYPE_MESSAGE);
+ manager1.addState(LEGACY_MESSAGE);
+ assertThrows(IllegalArgumentException.class, () -> manager1.addState(GLOBAL_MESSAGE));
+ assertThrows(IllegalArgumentException.class, () -> manager1.addState(STREAM_MESSAGE));
+
+ final DefaultDestStateLifecycleManager manager2 = new DefaultDestStateLifecycleManager(singleStateMgr, streamMgr);
+ manager2.addState(LEGACY_MESSAGE);
+ manager2.addState(LEGACY_MESSAGE);
+ manager2.addState(UNSET_TYPE_MESSAGE);
+ assertThrows(IllegalArgumentException.class, () -> manager2.addState(GLOBAL_MESSAGE));
+ assertThrows(IllegalArgumentException.class, () -> manager2.addState(STREAM_MESSAGE));
+
+ final DefaultDestStateLifecycleManager manager3 = new DefaultDestStateLifecycleManager(singleStateMgr, streamMgr);
+ manager3.addState(GLOBAL_MESSAGE);
+ manager3.addState(GLOBAL_MESSAGE);
+ assertThrows(IllegalArgumentException.class, () -> manager3.addState(UNSET_TYPE_MESSAGE));
+ assertThrows(IllegalArgumentException.class, () -> manager3.addState(LEGACY_MESSAGE));
+ assertThrows(IllegalArgumentException.class, () -> manager3.addState(STREAM_MESSAGE));
+
+ final DefaultDestStateLifecycleManager manager4 = new DefaultDestStateLifecycleManager(singleStateMgr, streamMgr);
+ manager4.addState(STREAM_MESSAGE);
+ manager4.addState(STREAM_MESSAGE);
+ assertThrows(IllegalArgumentException.class, () -> manager4.addState(UNSET_TYPE_MESSAGE));
+ assertThrows(IllegalArgumentException.class, () -> manager4.addState(LEGACY_MESSAGE));
+ assertThrows(IllegalArgumentException.class, () -> manager4.addState(GLOBAL_MESSAGE));
+ }
+
+ @Test
+ void testDelegatesLegacyMessages() {
+ mgr1.addState(UNSET_TYPE_MESSAGE);
+ mgr1.addState(LEGACY_MESSAGE);
+ mgr1.markPendingAsFlushed();
+ mgr1.markFlushedAsCommitted();
+ mgr1.listFlushed();
+ mgr1.listCommitted();
+ verify(singleStateMgr).addState(UNSET_TYPE_MESSAGE);
+ verify(singleStateMgr).addState(LEGACY_MESSAGE);
+ verify(singleStateMgr).markPendingAsFlushed();
+ verify(singleStateMgr).markFlushedAsCommitted();
+ verify(singleStateMgr).listFlushed();
+ verify(singleStateMgr).listCommitted();
+ }
+
+ @Test
+ void testDelegatesGlobalMessages() {
+ mgr1.addState(GLOBAL_MESSAGE);
+ mgr1.markPendingAsFlushed();
+ mgr1.markFlushedAsCommitted();
+ mgr1.listFlushed();
+ mgr1.listCommitted();
+ verify(singleStateMgr).addState(GLOBAL_MESSAGE);
+ verify(singleStateMgr).markPendingAsFlushed();
+ verify(singleStateMgr).markFlushedAsCommitted();
+ verify(singleStateMgr).listFlushed();
+ verify(singleStateMgr).listCommitted();
+ }
+
+ @Test
+ void testDelegatesStreamMessages() {
+ mgr1.addState(STREAM_MESSAGE);
+ mgr1.markPendingAsFlushed();
+ mgr1.markFlushedAsCommitted();
+ mgr1.listFlushed();
+ mgr1.listCommitted();
+
+ verify(streamMgr).addState(STREAM_MESSAGE);
+ verify(streamMgr).markPendingAsFlushed();
+ verify(streamMgr).markFlushedAsCommitted();
+ verify(streamMgr).listFlushed();
+ verify(streamMgr).listCommitted();
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/destination/dest_state_lifecycle_manager/DestSingleStateLifecycleManagerTest.java b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/destination/dest_state_lifecycle_manager/DestSingleStateLifecycleManagerTest.java
new file mode 100644
index 0000000000000..c70b415cdcc3c
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/destination/dest_state_lifecycle_manager/DestSingleStateLifecycleManagerTest.java
@@ -0,0 +1,141 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.dest_state_lifecycle_manager;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import io.airbyte.commons.json.Jsons;
+import io.airbyte.protocol.models.v0.AirbyteMessage;
+import io.airbyte.protocol.models.v0.AirbyteMessage.Type;
+import io.airbyte.protocol.models.v0.AirbyteStateMessage;
+import io.airbyte.protocol.models.v0.AirbyteStateMessage.AirbyteStateType;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+class DestSingleStateLifecycleManagerTest {
+
+ private static final AirbyteMessage MESSAGE1 = new AirbyteMessage()
+ .withType(Type.STATE)
+ .withState(new AirbyteStateMessage().withType(AirbyteStateType.GLOBAL).withData(Jsons.jsonNode("a")));
+ private static final AirbyteMessage MESSAGE2 = new AirbyteMessage()
+ .withType(Type.STATE)
+ .withState(new AirbyteStateMessage().withType(AirbyteStateType.GLOBAL).withData(Jsons.jsonNode("b")));
+
+ private DestSingleStateLifecycleManager mgr;
+
+ @BeforeEach
+ void setup() {
+ mgr = new DestSingleStateLifecycleManager();
+ }
+
+ /**
+ * Demonstrates expected lifecycle of a state object for documentation purposes. Subsequent test get
+ * into the details.
+ */
+ @Test
+ void testBasicLifeCycle() {
+ // starts with no state.
+ assertTrue(mgr.listPending().isEmpty());
+ assertTrue(mgr.listFlushed().isEmpty());
+ assertTrue(mgr.listCommitted().isEmpty());
+
+ mgr.addState(MESSAGE1);
+ // new state supersedes previous ones. we should only see MESSAGE2 from here on out.
+ mgr.addState(MESSAGE2);
+
+ // after adding a state, it is in pending only.
+ assertEquals(MESSAGE2, mgr.listPending().poll());
+ assertTrue(mgr.listFlushed().isEmpty());
+ assertTrue(mgr.listCommitted().isEmpty());
+
+ mgr.markPendingAsFlushed();
+
+ // after flushing the state it is in flushed only.
+ assertTrue(mgr.listPending().isEmpty());
+ assertEquals(MESSAGE2, mgr.listFlushed().poll());
+ assertTrue(mgr.listCommitted().isEmpty());
+
+ // after committing the state it is in committed only.
+ mgr.markFlushedAsCommitted();
+
+ assertTrue(mgr.listPending().isEmpty());
+ assertTrue(mgr.listFlushed().isEmpty());
+ assertEquals(MESSAGE2, mgr.listCommitted().poll());
+ }
+
+ @Test
+ void testPending() {
+ mgr.addState(MESSAGE1);
+ mgr.addState(MESSAGE2);
+
+ // verify the LAST message is returned.
+ assertEquals(MESSAGE2, mgr.listPending().poll());
+ assertTrue(mgr.listFlushed().isEmpty());
+ assertTrue(mgr.listCommitted().isEmpty());
+ }
+
+ @Test
+ void testFlushed() {
+ mgr.addState(MESSAGE1);
+ mgr.addState(MESSAGE2);
+ mgr.markPendingAsFlushed();
+
+ assertTrue(mgr.listPending().isEmpty());
+ assertEquals(MESSAGE2, mgr.listFlushed().poll());
+ assertTrue(mgr.listCommitted().isEmpty());
+
+ // verify that multiple calls to markPendingAsFlushed overwrite old states
+ mgr.addState(MESSAGE1);
+ mgr.markPendingAsFlushed();
+ mgr.markPendingAsFlushed();
+
+ assertTrue(mgr.listPending().isEmpty());
+ assertEquals(MESSAGE1, mgr.listFlushed().poll());
+ assertTrue(mgr.listCommitted().isEmpty());
+ }
+
+ @Test
+ void testCommitted() {
+ mgr.addState(MESSAGE1);
+ mgr.addState(MESSAGE2);
+ mgr.markPendingAsFlushed();
+ mgr.markFlushedAsCommitted();
+
+ assertTrue(mgr.listPending().isEmpty());
+ assertTrue(mgr.listFlushed().isEmpty());
+ assertEquals(MESSAGE2, mgr.listCommitted().poll());
+
+ // verify that multiple calls to markFlushedAsCommitted overwrite old states
+ mgr.addState(MESSAGE1);
+ mgr.markPendingAsFlushed();
+ mgr.markFlushedAsCommitted();
+ mgr.markFlushedAsCommitted();
+
+ assertTrue(mgr.listPending().isEmpty());
+ assertTrue(mgr.listFlushed().isEmpty());
+ assertEquals(MESSAGE1, mgr.listCommitted().poll());
+ }
+
+ /*
+ * This change follows the same changes in DestStreamStateLifecycleManager where the goal is to
+ * confirm that `markPendingAsCommitted` combines what was previous `markPendingAsFlushed` and
+ * `markFlushedAsCommitted`
+ *
+ * The reason for this method is due to destination checkpointing will no longer hold into a state
+ * as "Flushed" but immediately commit records to the destination's final table
+ */
+ @Test
+ void testMarkPendingAsCommitted() {
+ mgr.addState(MESSAGE1);
+ mgr.addState(MESSAGE2);
+ mgr.markPendingAsCommitted();
+
+ assertTrue(mgr.listPending().isEmpty());
+ assertTrue(mgr.listFlushed().isEmpty());
+ assertEquals(MESSAGE2, mgr.listCommitted().poll());
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/destination/dest_state_lifecycle_manager/DestStreamStateLifecycleManagerTest.java b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/destination/dest_state_lifecycle_manager/DestStreamStateLifecycleManagerTest.java
new file mode 100644
index 0000000000000..b24350b969c56
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/destination/dest_state_lifecycle_manager/DestStreamStateLifecycleManagerTest.java
@@ -0,0 +1,162 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.dest_state_lifecycle_manager;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+import io.airbyte.commons.json.Jsons;
+import io.airbyte.protocol.models.v0.AirbyteMessage;
+import io.airbyte.protocol.models.v0.AirbyteMessage.Type;
+import io.airbyte.protocol.models.v0.AirbyteStateMessage;
+import io.airbyte.protocol.models.v0.AirbyteStateMessage.AirbyteStateType;
+import io.airbyte.protocol.models.v0.AirbyteStreamState;
+import io.airbyte.protocol.models.v0.StreamDescriptor;
+import java.util.LinkedList;
+import java.util.List;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+class DestStreamStateLifecycleManagerTest {
+
+ private static final AirbyteMessage STREAM1_MESSAGE1 = new AirbyteMessage()
+ .withType(Type.STATE)
+ .withState(new AirbyteStateMessage()
+ .withType(AirbyteStateType.STREAM)
+ .withStream(new AirbyteStreamState().withStreamDescriptor(new StreamDescriptor().withName("apples")).withStreamState(Jsons.jsonNode("a"))));
+ private static final AirbyteMessage STREAM1_MESSAGE2 = new AirbyteMessage()
+ .withType(Type.STATE)
+ .withState(new AirbyteStateMessage()
+ .withType(AirbyteStateType.STREAM)
+ .withStream(new AirbyteStreamState().withStreamDescriptor(new StreamDescriptor().withName("apples")).withStreamState(Jsons.jsonNode("b"))));
+ private static final AirbyteMessage STREAM2_MESSAGE1 = new AirbyteMessage()
+ .withType(Type.STATE)
+ .withState(new AirbyteStateMessage()
+ .withType(AirbyteStateType.STREAM)
+ .withStream(
+ new AirbyteStreamState().withStreamDescriptor(new StreamDescriptor().withName("bananas")).withStreamState(Jsons.jsonNode("10"))));
+
+ private DestStreamStateLifecycleManager mgr;
+
+ @BeforeEach
+ void setup() {
+ mgr = new DestStreamStateLifecycleManager();
+ }
+
+ /**
+ * Demonstrates expected lifecycle of a state object for documentation purposes. Subsequent test get
+ * into the details.
+ */
+ @Test
+ void testBasicLifeCycle() {
+ // starts with no state.
+ assertTrue(mgr.listPending().isEmpty());
+ assertTrue(mgr.listFlushed().isEmpty());
+ assertTrue(mgr.listCommitted().isEmpty());
+
+ mgr.addState(STREAM1_MESSAGE1);
+ // new state supersedes previous ones. we should only see MESSAGE2 for STREAM1 from here on out.
+ mgr.addState(STREAM1_MESSAGE2);
+ // different stream, thus does not interact with messages from STREAM1.
+ mgr.addState(STREAM2_MESSAGE1);
+
+ // after adding a state, it is in pending only.
+ assertEquals(new LinkedList<>(List.of(STREAM1_MESSAGE2, STREAM2_MESSAGE1)), mgr.listPending());
+ assertTrue(mgr.listFlushed().isEmpty());
+ assertTrue(mgr.listCommitted().isEmpty());
+
+ mgr.markPendingAsFlushed();
+
+ // after flushing the state it is in flushed only.
+ assertTrue(mgr.listPending().isEmpty());
+ assertEquals(new LinkedList<>(List.of(STREAM1_MESSAGE2, STREAM2_MESSAGE1)), mgr.listFlushed());
+ assertTrue(mgr.listCommitted().isEmpty());
+
+ // after committing the state it is in committed only.
+ mgr.markFlushedAsCommitted();
+
+ assertTrue(mgr.listPending().isEmpty());
+ assertTrue(mgr.listFlushed().isEmpty());
+ assertEquals(new LinkedList<>(List.of(STREAM1_MESSAGE2, STREAM2_MESSAGE1)), mgr.listCommitted());
+ }
+
+ @Test
+ void testPending() {
+ mgr.addState(STREAM1_MESSAGE1);
+ mgr.addState(STREAM1_MESSAGE2);
+ mgr.addState(STREAM2_MESSAGE1);
+
+ // verify the LAST message is returned.
+ assertEquals(new LinkedList<>(List.of(STREAM1_MESSAGE2, STREAM2_MESSAGE1)), mgr.listPending());
+ assertTrue(mgr.listFlushed().isEmpty());
+ assertTrue(mgr.listCommitted().isEmpty());
+ }
+
+ /*
+ * TODO: remove this test after all destination connectors have updated to reflect destination
+ * checkpointing changes where flush/commit will be bundled into the same operation
+ */
+ @Deprecated
+ @Test
+ void testFlushed() {
+ mgr.addState(STREAM1_MESSAGE1);
+ mgr.addState(STREAM1_MESSAGE2);
+ mgr.addState(STREAM2_MESSAGE1);
+ mgr.markPendingAsFlushed();
+
+ assertTrue(mgr.listPending().isEmpty());
+ assertEquals(new LinkedList<>(List.of(STREAM1_MESSAGE2, STREAM2_MESSAGE1)), mgr.listFlushed());
+ assertTrue(mgr.listCommitted().isEmpty());
+
+ // verify that multiple calls to markPendingAsFlushed overwrite old states
+ mgr.addState(STREAM1_MESSAGE1);
+ mgr.markPendingAsFlushed();
+ mgr.markPendingAsFlushed();
+
+ assertTrue(mgr.listPending().isEmpty());
+ assertEquals(new LinkedList<>(List.of(STREAM1_MESSAGE1, STREAM2_MESSAGE1)), mgr.listFlushed());
+ assertTrue(mgr.listCommitted().isEmpty());
+ }
+
+ @Test
+ void testCommitted() {
+ mgr.addState(STREAM1_MESSAGE1);
+ mgr.addState(STREAM1_MESSAGE2);
+ mgr.addState(STREAM2_MESSAGE1);
+ mgr.markPendingAsFlushed();
+ mgr.markFlushedAsCommitted();
+
+ assertTrue(mgr.listPending().isEmpty());
+ assertTrue(mgr.listFlushed().isEmpty());
+ assertEquals(new LinkedList<>(List.of(STREAM1_MESSAGE2, STREAM2_MESSAGE1)), mgr.listCommitted());
+
+ // verify that multiple calls to markFlushedAsCommitted overwrite old states
+ mgr.addState(STREAM1_MESSAGE1);
+ mgr.markPendingAsFlushed();
+ mgr.markFlushedAsCommitted();
+ mgr.markFlushedAsCommitted();
+
+ assertTrue(mgr.listPending().isEmpty());
+ assertTrue(mgr.listFlushed().isEmpty());
+ assertEquals(new LinkedList<>(List.of(STREAM1_MESSAGE1, STREAM2_MESSAGE1)), mgr.listCommitted());
+ }
+
+ /*
+ * This section is to test for logic that is isolated to changes with respect to destination
+ * checkpointing where it captures flush and commit are bundled into a transaction so
+ *
+ * buffer -(flush buffer)-> staging area -(copy into {staging_file})-> destination raw table
+ */
+ @Test
+ void testPendingAsCommitted() {
+ mgr.addState(STREAM1_MESSAGE1);
+ mgr.markPendingAsCommitted();
+
+ // verifies that we've skipped "Flushed" without needing to call `markPendingAsFlushed()` and
+ // `markFlushedAsCommitted`
+ assertTrue(mgr.listPending().isEmpty());
+ assertEquals(new LinkedList<>(List.of(STREAM1_MESSAGE1)), mgr.listCommitted());
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/destination/record_buffer/InMemoryRecordBufferingStrategyTest.java b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/destination/record_buffer/InMemoryRecordBufferingStrategyTest.java
new file mode 100644
index 0000000000000..69ede03c8b8a5
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/destination/record_buffer/InMemoryRecordBufferingStrategyTest.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.record_buffer;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import io.airbyte.commons.json.Jsons;
+import io.airbyte.integrations.destination.buffered_stream_consumer.RecordWriter;
+import io.airbyte.protocol.models.v0.AirbyteMessage;
+import io.airbyte.protocol.models.v0.AirbyteRecordMessage;
+import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair;
+import java.util.List;
+import java.util.Optional;
+import org.junit.jupiter.api.Test;
+
+public class InMemoryRecordBufferingStrategyTest {
+
+ private static final JsonNode MESSAGE_DATA = Jsons.deserialize("{ \"field1\": 10000 }");
+ // MESSAGE_DATA should be 64 bytes long, size the buffer such as it can contain at least 2 message
+ // instances
+ private static final int MAX_QUEUE_SIZE_IN_BYTES = 130;
+
+ @SuppressWarnings("unchecked")
+ private final RecordWriter recordWriter = mock(RecordWriter.class);
+
+ @Test
+ public void testBuffering() throws Exception {
+ final InMemoryRecordBufferingStrategy buffering = new InMemoryRecordBufferingStrategy(recordWriter, MAX_QUEUE_SIZE_IN_BYTES);
+ final AirbyteStreamNameNamespacePair stream1 = new AirbyteStreamNameNamespacePair("stream1", "namespace");
+ final AirbyteStreamNameNamespacePair stream2 = new AirbyteStreamNameNamespacePair("stream2", null);
+ final AirbyteMessage message1 = generateMessage(stream1);
+ final AirbyteMessage message2 = generateMessage(stream2);
+ final AirbyteMessage message3 = generateMessage(stream2);
+ final AirbyteMessage message4 = generateMessage(stream2);
+
+ assertFalse(buffering.addRecord(stream1, message1).isPresent());
+ assertFalse(buffering.addRecord(stream2, message2).isPresent());
+ // Buffer still has room
+ final Optional flushType = buffering.addRecord(stream2, message3);
+ // Keeps track of this #addRecord since we're expecting a buffer flush & that the flushType
+ // value will indicate that all buffers were flushed
+ assertTrue(flushType.isPresent());
+ assertEquals(flushType.get(), BufferFlushType.FLUSH_ALL);
+ // Buffer limit reach, flushing all messages so far before adding the new incoming one
+ verify(recordWriter, times(1)).accept(stream1, List.of(message1.getRecord()));
+ verify(recordWriter, times(1)).accept(stream2, List.of(message2.getRecord()));
+
+ buffering.addRecord(stream2, message4);
+
+ // force flush to terminate test
+ buffering.flushAllBuffers();
+ verify(recordWriter, times(1)).accept(stream2, List.of(message3.getRecord(), message4.getRecord()));
+ }
+
+ private static AirbyteMessage generateMessage(final AirbyteStreamNameNamespacePair stream) {
+ return new AirbyteMessage().withRecord(new AirbyteRecordMessage()
+ .withStream(stream.getName())
+ .withNamespace(stream.getNamespace())
+ .withData(MESSAGE_DATA));
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/destination/record_buffer/SerializedBufferingStrategyTest.java b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/destination/record_buffer/SerializedBufferingStrategyTest.java
new file mode 100644
index 0000000000000..b38953c3c25b7
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/destination/record_buffer/SerializedBufferingStrategyTest.java
@@ -0,0 +1,220 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.record_buffer;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import io.airbyte.commons.json.Jsons;
+import io.airbyte.protocol.models.v0.AirbyteMessage;
+import io.airbyte.protocol.models.v0.AirbyteRecordMessage;
+import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair;
+import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog;
+import java.util.Optional;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+public class SerializedBufferingStrategyTest {
+
+ private static final JsonNode MESSAGE_DATA = Jsons.deserialize("{ \"field1\": 10000 }");
+ private static final String STREAM_1 = "stream1";
+ private static final String STREAM_2 = "stream2";
+ private static final String STREAM_3 = "stream3";
+ private static final String STREAM_4 = "stream4";
+
+ // we set the limit to hold at most 4 messages of 10b total
+ private static final long MAX_TOTAL_BUFFER_SIZE_BYTES = 42L;
+ // we set the limit to hold at most 2 messages of 10b per stream
+ private static final long MAX_PER_STREAM_BUFFER_SIZE_BYTES = 21L;
+
+ private final ConfiguredAirbyteCatalog catalog = mock(ConfiguredAirbyteCatalog.class);
+ @SuppressWarnings("unchecked")
+ private final FlushBufferFunction perStreamFlushHook = mock(FlushBufferFunction.class);
+
+ private final SerializableBuffer recordWriter1 = mock(SerializableBuffer.class);
+ private final SerializableBuffer recordWriter2 = mock(SerializableBuffer.class);
+ private final SerializableBuffer recordWriter3 = mock(SerializableBuffer.class);
+ private final SerializableBuffer recordWriter4 = mock(SerializableBuffer.class);
+
+ @BeforeEach
+ public void setup() throws Exception {
+ setupMock(recordWriter1);
+ setupMock(recordWriter2);
+ setupMock(recordWriter3);
+ setupMock(recordWriter4);
+ }
+
+ private void setupMock(final SerializableBuffer mockObject) throws Exception {
+ when(mockObject.accept(any())).thenReturn(10L);
+ when(mockObject.getByteCount()).thenReturn(10L);
+ when(mockObject.getMaxTotalBufferSizeInBytes()).thenReturn(MAX_TOTAL_BUFFER_SIZE_BYTES);
+ when(mockObject.getMaxPerStreamBufferSizeInBytes()).thenReturn(MAX_PER_STREAM_BUFFER_SIZE_BYTES);
+ when(mockObject.getMaxConcurrentStreamsInBuffer()).thenReturn(4);
+ }
+
+ @Test
+ public void testPerStreamThresholdFlush() throws Exception {
+ final SerializedBufferingStrategy buffering = new SerializedBufferingStrategy(onCreateBufferFunction(), catalog, perStreamFlushHook);
+ final AirbyteStreamNameNamespacePair stream1 = new AirbyteStreamNameNamespacePair(STREAM_1, "namespace");
+ final AirbyteStreamNameNamespacePair stream2 = new AirbyteStreamNameNamespacePair(STREAM_2, null);
+ // To test per stream threshold, we are sending multiple test messages on a single stream
+ final AirbyteMessage message1 = generateMessage(stream1);
+ final AirbyteMessage message2 = generateMessage(stream2);
+ final AirbyteMessage message3 = generateMessage(stream2);
+ final AirbyteMessage message4 = generateMessage(stream2);
+ final AirbyteMessage message5 = generateMessage(stream2);
+
+ when(recordWriter1.getByteCount()).thenReturn(10L); // one record in recordWriter1
+ assertFalse(buffering.addRecord(stream1, message1).isPresent());
+ when(recordWriter2.getByteCount()).thenReturn(10L); // one record in recordWriter2
+ assertFalse(buffering.addRecord(stream2, message2).isPresent());
+
+ // Total and per stream Buffers still have room
+ verify(perStreamFlushHook, times(0)).accept(stream1, recordWriter1);
+ verify(perStreamFlushHook, times(0)).accept(stream2, recordWriter2);
+
+ when(recordWriter2.getByteCount()).thenReturn(20L); // second record in recordWriter2
+ assertFalse(buffering.addRecord(stream2, message3).isPresent());
+ when(recordWriter2.getByteCount()).thenReturn(30L); // third record in recordWriter2
+
+ // Buffer reaches limit so a buffer flush occurs returning a buffer flush type of single stream
+ final Optional flushType = buffering.addRecord(stream2, message4);
+ assertTrue(flushType.isPresent());
+ assertEquals(flushType.get(), BufferFlushType.FLUSH_SINGLE_STREAM);
+
+ // The buffer limit is now reached for stream2, flushing that single stream only
+ verify(perStreamFlushHook, times(0)).accept(stream1, recordWriter1);
+ verify(perStreamFlushHook, times(1)).accept(stream2, recordWriter2);
+
+ when(recordWriter2.getByteCount()).thenReturn(10L); // back to one record in recordWriter2
+ assertFalse(buffering.addRecord(stream2, message5).isPresent());
+
+ // force flush to terminate test
+ buffering.flushAllBuffers();
+ verify(perStreamFlushHook, times(1)).accept(stream1, recordWriter1);
+ verify(perStreamFlushHook, times(2)).accept(stream2, recordWriter2);
+ }
+
+ @Test
+ public void testTotalStreamThresholdFlush() throws Exception {
+ final SerializedBufferingStrategy buffering = new SerializedBufferingStrategy(onCreateBufferFunction(), catalog, perStreamFlushHook);
+ final AirbyteStreamNameNamespacePair stream1 = new AirbyteStreamNameNamespacePair(STREAM_1, "namespace");
+ final AirbyteStreamNameNamespacePair stream2 = new AirbyteStreamNameNamespacePair(STREAM_2, "namespace");
+ final AirbyteStreamNameNamespacePair stream3 = new AirbyteStreamNameNamespacePair(STREAM_3, "namespace");
+ // To test total stream threshold, we are sending test messages to multiple streams without reaching
+ // per stream limits
+ final AirbyteMessage message1 = generateMessage(stream1);
+ final AirbyteMessage message2 = generateMessage(stream2);
+ final AirbyteMessage message3 = generateMessage(stream3);
+ final AirbyteMessage message4 = generateMessage(stream1);
+ final AirbyteMessage message5 = generateMessage(stream2);
+ final AirbyteMessage message6 = generateMessage(stream3);
+
+ assertFalse(buffering.addRecord(stream1, message1).isPresent());
+ assertFalse(buffering.addRecord(stream2, message2).isPresent());
+ // Total and per stream Buffers still have room
+ verify(perStreamFlushHook, times(0)).accept(stream1, recordWriter1);
+ verify(perStreamFlushHook, times(0)).accept(stream2, recordWriter2);
+ verify(perStreamFlushHook, times(0)).accept(stream3, recordWriter3);
+
+ assertFalse(buffering.addRecord(stream3, message3).isPresent());
+ when(recordWriter1.getByteCount()).thenReturn(20L); // second record in recordWriter1
+ assertFalse(buffering.addRecord(stream1, message4).isPresent());
+ when(recordWriter2.getByteCount()).thenReturn(20L); // second record in recordWriter2
+
+ // In response to checkpointing, will need to know what type of buffer flush occurred to mark
+ // AirbyteStateMessage as committed depending on DestDefaultStateLifecycleManager
+ final Optional flushType = buffering.addRecord(stream2, message5);
+ assertTrue(flushType.isPresent());
+ assertEquals(flushType.get(), BufferFlushType.FLUSH_ALL);
+
+ // Buffer limit reached for total streams, flushing all streams
+ verify(perStreamFlushHook, times(1)).accept(stream1, recordWriter1);
+ verify(perStreamFlushHook, times(1)).accept(stream2, recordWriter2);
+ verify(perStreamFlushHook, times(1)).accept(stream3, recordWriter3);
+
+ assertFalse(buffering.addRecord(stream3, message6).isPresent());
+ // force flush to terminate test
+ buffering.flushAllBuffers();
+ verify(perStreamFlushHook, times(1)).accept(stream1, recordWriter1);
+ verify(perStreamFlushHook, times(1)).accept(stream2, recordWriter2);
+ verify(perStreamFlushHook, times(2)).accept(stream3, recordWriter3);
+ }
+
+ @Test
+ public void testConcurrentStreamThresholdFlush() throws Exception {
+ final SerializedBufferingStrategy buffering = new SerializedBufferingStrategy(onCreateBufferFunction(), catalog, perStreamFlushHook);
+ final AirbyteStreamNameNamespacePair stream1 = new AirbyteStreamNameNamespacePair(STREAM_1, "namespace1");
+ final AirbyteStreamNameNamespacePair stream2 = new AirbyteStreamNameNamespacePair(STREAM_2, "namespace2");
+ final AirbyteStreamNameNamespacePair stream3 = new AirbyteStreamNameNamespacePair(STREAM_3, null);
+ final AirbyteStreamNameNamespacePair stream4 = new AirbyteStreamNameNamespacePair(STREAM_4, null);
+ // To test concurrent stream threshold, we are sending test messages to multiple streams
+ final AirbyteMessage message1 = generateMessage(stream1);
+ final AirbyteMessage message2 = generateMessage(stream2);
+ final AirbyteMessage message3 = generateMessage(stream3);
+ final AirbyteMessage message4 = generateMessage(stream4);
+ final AirbyteMessage message5 = generateMessage(stream1);
+
+ assertFalse(buffering.addRecord(stream1, message1).isPresent());
+ assertFalse(buffering.addRecord(stream2, message2).isPresent());
+ assertFalse(buffering.addRecord(stream3, message3).isPresent());
+ // Total and per stream Buffers still have room
+ verify(perStreamFlushHook, times(0)).accept(stream1, recordWriter1);
+ verify(perStreamFlushHook, times(0)).accept(stream2, recordWriter2);
+ verify(perStreamFlushHook, times(0)).accept(stream3, recordWriter3);
+
+ // Since the concurrent stream threshold has been exceeded, all buffer streams are flush
+ final Optional flushType = buffering.addRecord(stream4, message4);
+ assertTrue(flushType.isPresent());
+ assertEquals(flushType.get(), BufferFlushType.FLUSH_ALL);
+
+ // Buffer limit reached for concurrent streams, flushing all streams
+ verify(perStreamFlushHook, times(1)).accept(stream1, recordWriter1);
+ verify(perStreamFlushHook, times(1)).accept(stream2, recordWriter2);
+ verify(perStreamFlushHook, times(1)).accept(stream3, recordWriter3);
+ verify(perStreamFlushHook, times(1)).accept(stream4, recordWriter4);
+
+ assertFalse(buffering.addRecord(stream1, message5).isPresent());
+ // force flush to terminate test
+ buffering.flushAllBuffers();
+ verify(perStreamFlushHook, times(2)).accept(stream1, recordWriter1);
+ verify(perStreamFlushHook, times(1)).accept(stream2, recordWriter2);
+ verify(perStreamFlushHook, times(1)).accept(stream3, recordWriter3);
+ verify(perStreamFlushHook, times(1)).accept(stream4, recordWriter4);
+ }
+
+ @Test
+ public void testCreateBufferFailure() {
+ final SerializedBufferingStrategy buffering = new SerializedBufferingStrategy(onCreateBufferFunction(), catalog, perStreamFlushHook);
+ final AirbyteStreamNameNamespacePair stream = new AirbyteStreamNameNamespacePair("unknown_stream", "namespace1");
+ assertThrows(RuntimeException.class, () -> buffering.addRecord(stream, generateMessage(stream)));
+ }
+
+ private static AirbyteMessage generateMessage(final AirbyteStreamNameNamespacePair stream) {
+ return new AirbyteMessage().withRecord(new AirbyteRecordMessage()
+ .withStream(stream.getName())
+ .withNamespace(stream.getNamespace())
+ .withData(MESSAGE_DATA));
+ }
+
+ private BufferCreateFunction onCreateBufferFunction() {
+ return (stream, catalog) -> switch (stream.getName()) {
+ case STREAM_1 -> recordWriter1;
+ case STREAM_2 -> recordWriter2;
+ case STREAM_3 -> recordWriter3;
+ case STREAM_4 -> recordWriter4;
+ default -> null;
+ };
+ }
+
+}
diff --git a/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/util/ConnectorExceptionUtilTest.java b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/util/ConnectorExceptionUtilTest.java
new file mode 100644
index 0000000000000..5371299b4cc61
--- /dev/null
+++ b/airbyte-integrations/bases/base-java-async/src/test/java/io/airbyte/integrations/util/ConnectorExceptionUtilTest.java
@@ -0,0 +1,136 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.util;
+
+import static io.airbyte.integrations.util.ConnectorExceptionUtil.COMMON_EXCEPTION_MESSAGE_TEMPLATE;
+import static io.airbyte.integrations.util.ConnectorExceptionUtil.RECOVERY_CONNECTION_ERROR_MESSAGE;
+import static org.junit.jupiter.api.Assertions.*;
+
+import io.airbyte.commons.exceptions.ConfigErrorException;
+import io.airbyte.commons.exceptions.ConnectionErrorException;
+import java.sql.SQLException;
+import java.sql.SQLSyntaxErrorException;
+import org.junit.jupiter.api.Test;
+
+class ConnectorExceptionUtilTest {
+
+ public static final String CONFIG_EXCEPTION_MESSAGE = "test message";
+ public static final String RECOVERY_EXCEPTION_MESSAGE = "FATAL: terminating connection due to conflict with recovery";
+ public static final String COMMON_EXCEPTION_MESSAGE = "something happens with connection";
+ public static final String CONNECTION_ERROR_MESSAGE_TEMPLATE = "State code: %s; Error code: %s; Message: %s";
+ public static final String UNKNOWN_COLUMN_SQL_EXCEPTION_MESSAGE = "Unknown column 'table.column' in 'field list'";
+
+ @Test()
+ void isConfigErrorForConfigException() {
+ ConfigErrorException configErrorException = new ConfigErrorException(CONFIG_EXCEPTION_MESSAGE);
+ assertTrue(ConnectorExceptionUtil.isConfigError(configErrorException));
+
+ }
+
+ @Test
+ void isConfigErrorForConnectionException() {
+ ConnectionErrorException connectionErrorException = new ConnectionErrorException(CONFIG_EXCEPTION_MESSAGE);
+ assertTrue(ConnectorExceptionUtil.isConfigError(connectionErrorException));
+ }
+
+ @Test
+ void isConfigErrorForRecoveryPSQLException() {
+ SQLException recoveryPSQLException = new SQLException(RECOVERY_EXCEPTION_MESSAGE);
+ assertTrue(ConnectorExceptionUtil.isConfigError(recoveryPSQLException));
+ }
+
+ @Test
+ void isConfigErrorForUnknownColumnSQLSyntaxErrorException() {
+ SQLSyntaxErrorException unknownColumnSQLSyntaxErrorException = new SQLSyntaxErrorException(UNKNOWN_COLUMN_SQL_EXCEPTION_MESSAGE);
+ assertTrue(ConnectorExceptionUtil.isConfigError(unknownColumnSQLSyntaxErrorException));
+ }
+
+ @Test
+ void isConfigErrorForCommonSQLException() {
+ SQLException recoveryPSQLException = new SQLException(COMMON_EXCEPTION_MESSAGE);
+ assertFalse(ConnectorExceptionUtil.isConfigError(recoveryPSQLException));
+ }
+
+ @Test
+ void isConfigErrorForCommonException() {
+ assertFalse(ConnectorExceptionUtil.isConfigError(new Exception()));
+ }
+
+ @Test
+ void getDisplayMessageForConfigException() {
+ ConfigErrorException configErrorException = new ConfigErrorException(CONFIG_EXCEPTION_MESSAGE);
+ String actualDisplayMessage = ConnectorExceptionUtil.getDisplayMessage(configErrorException);
+ assertEquals(CONFIG_EXCEPTION_MESSAGE, actualDisplayMessage);
+ }
+
+ @Test
+ void getDisplayMessageForConnectionError() {
+ String testCode = "test code";
+ int errorCode = -1;
+ ConnectionErrorException connectionErrorException = new ConnectionErrorException(testCode, errorCode, CONFIG_EXCEPTION_MESSAGE, new Exception());
+ String actualDisplayMessage = ConnectorExceptionUtil.getDisplayMessage(connectionErrorException);
+ assertEquals(String.format(CONNECTION_ERROR_MESSAGE_TEMPLATE, testCode, errorCode, CONFIG_EXCEPTION_MESSAGE), actualDisplayMessage);
+ }
+
+ @Test
+ void getDisplayMessageForRecoveryException() {
+ SQLException recoveryException = new SQLException(RECOVERY_EXCEPTION_MESSAGE);
+ String actualDisplayMessage = ConnectorExceptionUtil.getDisplayMessage(recoveryException);
+ assertEquals(RECOVERY_CONNECTION_ERROR_MESSAGE, actualDisplayMessage);
+ }
+
+ @Test
+ void getDisplayMessageForUnknownSQLErrorException() {
+ SQLSyntaxErrorException unknownColumnSQLSyntaxErrorException = new SQLSyntaxErrorException(UNKNOWN_COLUMN_SQL_EXCEPTION_MESSAGE);
+ String actualDisplayMessage = ConnectorExceptionUtil.getDisplayMessage(unknownColumnSQLSyntaxErrorException);
+ assertEquals(UNKNOWN_COLUMN_SQL_EXCEPTION_MESSAGE, actualDisplayMessage);
+ }
+
+ @Test
+ void getDisplayMessageForCommonException() {
+ Exception exception = new SQLException(COMMON_EXCEPTION_MESSAGE);
+ String actualDisplayMessage = ConnectorExceptionUtil.getDisplayMessage(exception);
+ assertEquals(String.format(COMMON_EXCEPTION_MESSAGE_TEMPLATE, COMMON_EXCEPTION_MESSAGE), actualDisplayMessage);
+ }
+
+ @Test
+ void getRootConfigErrorFromConfigException() {
+ ConfigErrorException configErrorException = new ConfigErrorException(CONFIG_EXCEPTION_MESSAGE);
+ Exception exception = new Exception(COMMON_EXCEPTION_MESSAGE, configErrorException);
+
+ Throwable actualRootConfigError = ConnectorExceptionUtil.getRootConfigError(exception);
+ assertEquals(configErrorException, actualRootConfigError);
+ }
+
+ @Test
+ void getRootConfigErrorFromRecoverySQLException() {
+ SQLException recoveryException = new SQLException(RECOVERY_EXCEPTION_MESSAGE);
+ RuntimeException runtimeException = new RuntimeException(COMMON_EXCEPTION_MESSAGE, recoveryException);
+ Exception exception = new Exception(runtimeException);
+
+ Throwable actualRootConfigError = ConnectorExceptionUtil.getRootConfigError(exception);
+ assertEquals(recoveryException, actualRootConfigError);
+ }
+
+ @Test
+ void getRootConfigErrorFromUnknownSQLErrorException() {
+ SQLException unknownSQLErrorException = new SQLSyntaxErrorException(UNKNOWN_COLUMN_SQL_EXCEPTION_MESSAGE);
+ RuntimeException runtimeException = new RuntimeException(COMMON_EXCEPTION_MESSAGE, unknownSQLErrorException);
+ Exception exception = new Exception(runtimeException);
+
+ Throwable actualRootConfigError = ConnectorExceptionUtil.getRootConfigError(exception);
+ assertEquals(unknownSQLErrorException, actualRootConfigError);
+ }
+
+ @Test
+ void getRootConfigErrorFromNonConfigException() {
+ SQLException configErrorException = new SQLException(CONFIG_EXCEPTION_MESSAGE);
+ Exception exception = new Exception(COMMON_EXCEPTION_MESSAGE, configErrorException);
+
+ Throwable actualRootConfigError = ConnectorExceptionUtil.getRootConfigError(exception);
+ assertEquals(exception, actualRootConfigError);
+ }
+
+}
diff --git a/airbyte-integrations/bases/bases-destination-jdbc-async/build.gradle b/airbyte-integrations/bases/bases-destination-jdbc-async/build.gradle
new file mode 100644
index 0000000000000..57c98181a3e96
--- /dev/null
+++ b/airbyte-integrations/bases/bases-destination-jdbc-async/build.gradle
@@ -0,0 +1,34 @@
+plugins {
+ id 'application'
+ id 'airbyte-docker'
+ id 'airbyte-integration-test-java'
+}
+
+dependencies {
+ implementation 'com.google.cloud:google-cloud-storage:1.113.16'
+ implementation 'com.google.auth:google-auth-library-oauth2-http:0.25.5'
+
+ implementation project(':airbyte-db:db-lib')
+ implementation project(':airbyte-integrations:bases:base-java')
+ implementation project(':airbyte-integrations:bases:base-java-s3')
+ implementation libs.airbyte.protocol
+
+ implementation 'org.apache.commons:commons-lang3:3.11'
+ implementation 'org.apache.commons:commons-csv:1.4'
+ implementation 'com.github.alexmojaki:s3-stream-upload:2.2.2'
+ implementation 'com.fasterxml.jackson.core:jackson-databind'
+ implementation 'com.azure:azure-storage-blob:12.12.0'
+
+// A small utility library for working with units of digital information
+// https://github.com/aesy/datasize
+ implementation "io.aesy:datasize:1.0.0"
+
+ testImplementation libs.connectors.testcontainers.postgresql
+ testImplementation "org.mockito:mockito-inline:4.1.0"
+
+ integrationTestJavaImplementation project(':airbyte-integrations:bases:standard-destination-test')
+ integrationTestJavaImplementation libs.connectors.testcontainers.postgresql
+
+ implementation files(project(':airbyte-integrations:bases:base-java').airbyteDocker.outputs)
+ integrationTestJavaImplementation files(project(':airbyte-integrations:bases:base-normalization').airbyteDocker.outputs)
+}
diff --git a/airbyte-integrations/bases/bases-destination-jdbc-async/readme.md b/airbyte-integrations/bases/bases-destination-jdbc-async/readme.md
new file mode 100644
index 0000000000000..90924191b4be8
--- /dev/null
+++ b/airbyte-integrations/bases/bases-destination-jdbc-async/readme.md
@@ -0,0 +1,7 @@
+# JDBC Destination
+
+We are not planning to expose this destination in the UI yet. It serves as a base upon which we can build all of our other JDBC-compliant destinations.
+
+The reasons we are not exposing this destination by itself are:
+1. It is not terribly user-friendly (jdbc urls are hard for a human to parse)
+1. Each JDBC-compliant db, we need to make sure the appropriate drivers are installed on the image. We don't want to frontload installing all possible drivers, and instead would like to be more methodical. Instead for each JDBC-compliant destination, we will extend this one and then install only the necessary JDBC drivers on that destination's image.
diff --git a/airbyte-integrations/bases/bases-destination-jdbc-async/src/main/java/io/airbyte/integrations/destination/jdbc/AbstractJdbcDestination.java b/airbyte-integrations/bases/bases-destination-jdbc-async/src/main/java/io/airbyte/integrations/destination/jdbc/AbstractJdbcDestination.java
new file mode 100644
index 0000000000000..28079f9f50ea5
--- /dev/null
+++ b/airbyte-integrations/bases/bases-destination-jdbc-async/src/main/java/io/airbyte/integrations/destination/jdbc/AbstractJdbcDestination.java
@@ -0,0 +1,208 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.jdbc;
+
+import static io.airbyte.integrations.base.errors.messages.ErrorMessage.getErrorMessage;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import io.airbyte.commons.exceptions.ConnectionErrorException;
+import io.airbyte.commons.json.Jsons;
+import io.airbyte.commons.map.MoreMaps;
+import io.airbyte.db.factory.DataSourceFactory;
+import io.airbyte.db.jdbc.DefaultJdbcDatabase;
+import io.airbyte.db.jdbc.JdbcDatabase;
+import io.airbyte.db.jdbc.JdbcUtils;
+import io.airbyte.integrations.BaseConnector;
+import io.airbyte.integrations.base.AirbyteMessageConsumer;
+import io.airbyte.integrations.base.AirbyteTraceMessageUtility;
+import io.airbyte.integrations.base.Destination;
+import io.airbyte.integrations.destination.NamingConventionTransformer;
+import io.airbyte.protocol.models.v0.AirbyteConnectionStatus;
+import io.airbyte.protocol.models.v0.AirbyteConnectionStatus.Status;
+import io.airbyte.protocol.models.v0.AirbyteMessage;
+import io.airbyte.protocol.models.v0.AirbyteRecordMessage;
+import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.function.Consumer;
+import javax.sql.DataSource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractJdbcDestination extends BaseConnector implements Destination {
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(AbstractJdbcDestination.class);
+
+ private final String driverClass;
+ private final NamingConventionTransformer namingResolver;
+ private final SqlOperations sqlOperations;
+
+ protected NamingConventionTransformer getNamingResolver() {
+ return namingResolver;
+ }
+
+ protected SqlOperations getSqlOperations() {
+ return sqlOperations;
+ }
+
+ public AbstractJdbcDestination(final String driverClass,
+ final NamingConventionTransformer namingResolver,
+ final SqlOperations sqlOperations) {
+ this.driverClass = driverClass;
+ this.namingResolver = namingResolver;
+ this.sqlOperations = sqlOperations;
+ }
+
+ @Override
+ public AirbyteConnectionStatus check(final JsonNode config) {
+ final DataSource dataSource = getDataSource(config);
+
+ try {
+ final JdbcDatabase database = getDatabase(dataSource);
+ final String outputSchema = namingResolver.getIdentifier(config.get(JdbcUtils.SCHEMA_KEY).asText());
+ attemptSQLCreateAndDropTableOperations(outputSchema, database, namingResolver, sqlOperations);
+ return new AirbyteConnectionStatus().withStatus(Status.SUCCEEDED);
+ } catch (final ConnectionErrorException ex) {
+ final String message = getErrorMessage(ex.getStateCode(), ex.getErrorCode(), ex.getExceptionMessage(), ex);
+ AirbyteTraceMessageUtility.emitConfigErrorTrace(ex, message);
+ return new AirbyteConnectionStatus()
+ .withStatus(Status.FAILED)
+ .withMessage(message);
+ } catch (final Exception e) {
+ LOGGER.error("Exception while checking connection: ", e);
+ return new AirbyteConnectionStatus()
+ .withStatus(Status.FAILED)
+ .withMessage("Could not connect with provided configuration. \n" + e.getMessage());
+ } finally {
+ try {
+ DataSourceFactory.close(dataSource);
+ } catch (final Exception e) {
+ LOGGER.warn("Unable to close data source.", e);
+ }
+ }
+ }
+
+ /**
+ * This method is deprecated. It verifies table creation, but not insert right to a newly created
+ * table. Use attemptTableOperations with the attemptInsert argument instead.
+ */
+ @Deprecated
+ public static void attemptSQLCreateAndDropTableOperations(final String outputSchema,
+ final JdbcDatabase database,
+ final NamingConventionTransformer namingResolver,
+ final SqlOperations sqlOps)
+ throws Exception {
+ attemptTableOperations(outputSchema, database, namingResolver, sqlOps, false);
+ }
+
+ /**
+ * Verifies if provided creds has enough permissions. Steps are: 1. Create schema if not exists. 2.
+ * Create test table. 3. Insert dummy record to newly created table if "attemptInsert" set to true.
+ * 4. Delete table created on step 2.
+ *
+ * @param outputSchema - schema to tests against.
+ * @param database - database to tests against.
+ * @param namingResolver - naming resolver.
+ * @param sqlOps - SqlOperations object
+ * @param attemptInsert - set true if need to make attempt to insert dummy records to newly created
+ * table. Set false to skip insert step.
+ * @throws Exception
+ */
+ public static void attemptTableOperations(final String outputSchema,
+ final JdbcDatabase database,
+ final NamingConventionTransformer namingResolver,
+ final SqlOperations sqlOps,
+ final boolean attemptInsert)
+ throws Exception {
+ // verify we have write permissions on the target schema by creating a table with a random name,
+ // then dropping that table
+ try {
+ // Get metadata from the database to see whether connection is possible
+ database.bufferedResultSetQuery(conn -> conn.getMetaData().getCatalogs(), JdbcUtils.getDefaultSourceOperations()::rowToJson);
+
+ // verify we have write permissions on the target schema by creating a table with a random name,
+ // then dropping that table
+ final String outputTableName = namingResolver.getIdentifier("_airbyte_connection_test_" + UUID.randomUUID().toString().replaceAll("-", ""));
+ sqlOps.createSchemaIfNotExists(database, outputSchema);
+ sqlOps.createTableIfNotExists(database, outputSchema, outputTableName);
+ // verify if user has permission to make SQL INSERT queries
+ try {
+ if (attemptInsert) {
+ sqlOps.insertRecords(database, List.of(getDummyRecord()), outputSchema, outputTableName);
+ }
+ } finally {
+ sqlOps.dropTableIfExists(database, outputSchema, outputTableName);
+ }
+ } catch (final SQLException e) {
+ if (Objects.isNull(e.getCause()) || !(e.getCause() instanceof SQLException)) {
+ throw new ConnectionErrorException(e.getSQLState(), e.getErrorCode(), e.getMessage(), e);
+ } else {
+ final SQLException cause = (SQLException) e.getCause();
+ throw new ConnectionErrorException(e.getSQLState(), cause.getErrorCode(), cause.getMessage(), e);
+ }
+ } catch (final Exception e) {
+ throw new Exception(e);
+ }
+ }
+
+ /**
+ * Generates a dummy AirbyteRecordMessage with random values.
+ *
+ * @return AirbyteRecordMessage object with dummy values that may be used to test insert permission.
+ */
+ private static AirbyteRecordMessage getDummyRecord() {
+ final JsonNode dummyDataToInsert = Jsons.deserialize("{ \"field1\": true }");
+ return new AirbyteRecordMessage()
+ .withStream("stream1")
+ .withData(dummyDataToInsert)
+ .withEmittedAt(1602637589000L);
+ }
+
+ protected DataSource getDataSource(final JsonNode config) {
+ final JsonNode jdbcConfig = toJdbcConfig(config);
+ return DataSourceFactory.create(
+ jdbcConfig.get(JdbcUtils.USERNAME_KEY).asText(),
+ jdbcConfig.has(JdbcUtils.PASSWORD_KEY) ? jdbcConfig.get(JdbcUtils.PASSWORD_KEY).asText() : null,
+ driverClass,
+ jdbcConfig.get(JdbcUtils.JDBC_URL_KEY).asText(),
+ getConnectionProperties(config));
+ }
+
+ protected JdbcDatabase getDatabase(final DataSource dataSource) {
+ return new DefaultJdbcDatabase(dataSource);
+ }
+
+ protected Map getConnectionProperties(final JsonNode config) {
+ final Map customProperties = JdbcUtils.parseJdbcParameters(config, JdbcUtils.JDBC_URL_PARAMS_KEY);
+ final Map defaultProperties = getDefaultConnectionProperties(config);
+ assertCustomParametersDontOverwriteDefaultParameters(customProperties, defaultProperties);
+ return MoreMaps.merge(customProperties, defaultProperties);
+ }
+
+ private void assertCustomParametersDontOverwriteDefaultParameters(final Map customParameters,
+ final Map defaultParameters) {
+ for (final String key : defaultParameters.keySet()) {
+ if (customParameters.containsKey(key) && !Objects.equals(customParameters.get(key), defaultParameters.get(key))) {
+ throw new IllegalArgumentException("Cannot overwrite default JDBC parameter " + key);
+ }
+ }
+ }
+
+ protected abstract Map getDefaultConnectionProperties(final JsonNode config);
+
+ public abstract JsonNode toJdbcConfig(JsonNode config);
+
+ @Override
+ public AirbyteMessageConsumer getConsumer(final JsonNode config,
+ final ConfiguredAirbyteCatalog catalog,
+ final Consumer outputRecordCollector) {
+ return JdbcBufferedConsumerFactory.create(outputRecordCollector, getDatabase(getDataSource(config)), sqlOperations, namingResolver, config,
+ catalog);
+ }
+
+}
diff --git a/airbyte-integrations/bases/bases-destination-jdbc-async/src/main/java/io/airbyte/integrations/destination/jdbc/DataAdapter.java b/airbyte-integrations/bases/bases-destination-jdbc-async/src/main/java/io/airbyte/integrations/destination/jdbc/DataAdapter.java
new file mode 100644
index 0000000000000..c445eeddacdd4
--- /dev/null
+++ b/airbyte-integrations/bases/bases-destination-jdbc-async/src/main/java/io/airbyte/integrations/destination/jdbc/DataAdapter.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.jdbc;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+public class DataAdapter {
+
+ private final Predicate filterValueNode;
+ private final Function valueNodeAdapter;
+
+ /**
+ * Data adapter allows applying destination data rules. For example, Postgres destination can't
+ * process text value with \u0000 unicode. You can describe filter condition for a value node and
+ * function which adapts filtered value nodes.
+ *
+ * @param filterValueNode - filter condition which decide which value node should be adapted
+ * @param valueNodeAdapter - transformation function which returns adapted value node
+ */
+ public DataAdapter(
+ final Predicate filterValueNode,
+ final Function valueNodeAdapter) {
+ this.filterValueNode = filterValueNode;
+ this.valueNodeAdapter = valueNodeAdapter;
+ }
+
+ public void adapt(final JsonNode messageData) {
+ if (messageData != null) {
+ adaptAllValueNodes(messageData);
+ }
+ }
+
+ private void adaptAllValueNodes(final JsonNode rootNode) {
+ adaptValueNodes(null, rootNode, null);
+ }
+
+ /**
+ * The method inspects json node. In case, it's a value node we check the node by CheckFunction and
+ * apply ValueNodeAdapter. Filtered nodes will be updated by adapted version. If element is an array
+ * or an object, this we run the method recursively for them.
+ *
+ * @param fieldName Name of a json node
+ * @param node Json node
+ * @param parentNode Parent json node
+ */
+ private void adaptValueNodes(final String fieldName, final JsonNode node, final JsonNode parentNode) {
+ if (node.isValueNode() && filterValueNode.test(node)) {
+ if (fieldName != null) {
+ final var adaptedNode = valueNodeAdapter.apply(node);
+ ((ObjectNode) parentNode).set(fieldName, adaptedNode);
+ } else
+ throw new RuntimeException("Unexpected value node without fieldName. Node: " + node);
+ } else if (node.isArray()) {
+ node.elements().forEachRemaining(arrayNode -> adaptValueNodes(null, arrayNode, node));
+ } else {
+ node.fields().forEachRemaining(stringJsonNodeEntry -> adaptValueNodes(stringJsonNodeEntry.getKey(), stringJsonNodeEntry.getValue(), node));
+ }
+ }
+
+}
diff --git a/airbyte-integrations/bases/bases-destination-jdbc-async/src/main/java/io/airbyte/integrations/destination/jdbc/JdbcBufferedConsumerFactory.java b/airbyte-integrations/bases/bases-destination-jdbc-async/src/main/java/io/airbyte/integrations/destination/jdbc/JdbcBufferedConsumerFactory.java
new file mode 100644
index 0000000000000..70d53fd806a47
--- /dev/null
+++ b/airbyte-integrations/bases/bases-destination-jdbc-async/src/main/java/io/airbyte/integrations/destination/jdbc/JdbcBufferedConsumerFactory.java
@@ -0,0 +1,213 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.jdbc;
+
+import static io.airbyte.integrations.destination.jdbc.constants.GlobalDataSizeConstants.DEFAULT_MAX_BATCH_SIZE_BYTES;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.google.common.base.Preconditions;
+import io.airbyte.commons.json.Jsons;
+import io.airbyte.db.jdbc.JdbcDatabase;
+import io.airbyte.db.jdbc.JdbcUtils;
+import io.airbyte.integrations.base.AirbyteMessageConsumer;
+import io.airbyte.integrations.destination.NamingConventionTransformer;
+import io.airbyte.integrations.destination.buffered_stream_consumer.BufferedStreamConsumer;
+import io.airbyte.integrations.destination.buffered_stream_consumer.OnCloseFunction;
+import io.airbyte.integrations.destination.buffered_stream_consumer.OnStartFunction;
+import io.airbyte.integrations.destination.buffered_stream_consumer.RecordWriter;
+import io.airbyte.integrations.destination.record_buffer.InMemoryRecordBufferingStrategy;
+import io.airbyte.protocol.models.v0.AirbyteMessage;
+import io.airbyte.protocol.models.v0.AirbyteRecordMessage;
+import io.airbyte.protocol.models.v0.AirbyteStream;
+import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair;
+import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog;
+import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream;
+import io.airbyte.protocol.models.v0.DestinationSyncMode;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Strategy:
+ *
+ * 1. Create a final table for each stream
+ *
+ * 2. Accumulate records in a buffer. One buffer per stream
+ *
+ * 3. As records accumulate write them in batch to the database. We set a minimum numbers of records
+ * before writing to avoid wasteful record-wise writes. In the case with slow syncs this will be
+ * superseded with a periodic record flush from {@link BufferedStreamConsumer#periodicBufferFlush()}
+ *
+ * 4. Once all records have been written to buffer, flush the buffer and write any remaining records
+ * to the database (regardless of how few are left)
+ */
+public class JdbcBufferedConsumerFactory {
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(JdbcBufferedConsumerFactory.class);
+
+ public static AirbyteMessageConsumer create(final Consumer outputRecordCollector,
+ final JdbcDatabase database,
+ final SqlOperations sqlOperations,
+ final NamingConventionTransformer namingResolver,
+ final JsonNode config,
+ final ConfiguredAirbyteCatalog catalog) {
+ final List writeConfigs = createWriteConfigs(namingResolver, config, catalog, sqlOperations.isSchemaRequired());
+
+ return new BufferedStreamConsumer(
+ outputRecordCollector,
+ onStartFunction(database, sqlOperations, writeConfigs),
+ new InMemoryRecordBufferingStrategy(recordWriterFunction(database, sqlOperations, writeConfigs, catalog), DEFAULT_MAX_BATCH_SIZE_BYTES),
+ onCloseFunction(database, sqlOperations, writeConfigs),
+ catalog,
+ sqlOperations::isValidData);
+ }
+
+ private static List createWriteConfigs(final NamingConventionTransformer namingResolver,
+ final JsonNode config,
+ final ConfiguredAirbyteCatalog catalog,
+ final boolean schemaRequired) {
+ if (schemaRequired) {
+ Preconditions.checkState(config.has("schema"), "jdbc destinations must specify a schema.");
+ }
+ final Instant now = Instant.now();
+ return catalog.getStreams().stream().map(toWriteConfig(namingResolver, config, now, schemaRequired)).collect(Collectors.toList());
+ }
+
+ private static Function toWriteConfig(
+ final NamingConventionTransformer namingResolver,
+ final JsonNode config,
+ final Instant now,
+ final boolean schemaRequired) {
+ return stream -> {
+ Preconditions.checkNotNull(stream.getDestinationSyncMode(), "Undefined destination sync mode");
+ final AirbyteStream abStream = stream.getStream();
+
+ final String defaultSchemaName = schemaRequired ? namingResolver.getIdentifier(config.get("schema").asText())
+ : namingResolver.getIdentifier(config.get(JdbcUtils.DATABASE_KEY).asText());
+ final String outputSchema = getOutputSchema(abStream, defaultSchemaName, namingResolver);
+
+ final String streamName = abStream.getName();
+ final String tableName = namingResolver.getRawTableName(streamName);
+ final String tmpTableName = namingResolver.getTmpTableName(streamName);
+ final DestinationSyncMode syncMode = stream.getDestinationSyncMode();
+
+ final WriteConfig writeConfig = new WriteConfig(streamName, abStream.getNamespace(), outputSchema, tmpTableName, tableName, syncMode);
+ LOGGER.info("Write config: {}", writeConfig);
+
+ return writeConfig;
+ };
+ }
+
+ /**
+ * Defer to the {@link AirbyteStream}'s namespace. If this is not set, use the destination's default
+ * schema. This namespace is source-provided, and can be potentially empty.
+ *
+ * The logic here matches the logic in the catalog_process.py for Normalization. Any modifications
+ * need to be reflected there and vice versa.
+ */
+ private static String getOutputSchema(final AirbyteStream stream,
+ final String defaultDestSchema,
+ final NamingConventionTransformer namingResolver) {
+ return stream.getNamespace() != null
+ ? namingResolver.getNamespace(stream.getNamespace())
+ : namingResolver.getNamespace(defaultDestSchema);
+ }
+
+ /**
+ * Sets up destination storage through:
+ *
+ * 1. Creates Schema (if not exists)
+ *
+ * 2. Creates airybte_raw table (if not exists)
+ *
+ * 3. Truncates table if sync mode is in OVERWRITE
+ *
+ * @param database JDBC database to connect to
+ * @param sqlOperations interface for execution SQL queries
+ * @param writeConfigs settings for each stream
+ * @return
+ */
+ private static OnStartFunction onStartFunction(final JdbcDatabase database,
+ final SqlOperations sqlOperations,
+ final List writeConfigs) {
+ return () -> {
+ LOGGER.info("Preparing raw tables in destination started for {} streams", writeConfigs.size());
+ final List queryList = new ArrayList<>();
+ for (final WriteConfig writeConfig : writeConfigs) {
+ final String schemaName = writeConfig.getOutputSchemaName();
+ final String dstTableName = writeConfig.getOutputTableName();
+ LOGGER.info("Preparing raw table in destination started for stream {}. schema: {}, table name: {}",
+ writeConfig.getStreamName(),
+ schemaName,
+ dstTableName);
+ sqlOperations.createSchemaIfNotExists(database, schemaName);
+ sqlOperations.createTableIfNotExists(database, schemaName, dstTableName);
+ switch (writeConfig.getSyncMode()) {
+ case OVERWRITE -> queryList.add(sqlOperations.truncateTableQuery(database, schemaName, dstTableName));
+ case APPEND, APPEND_DEDUP -> {}
+ default -> throw new IllegalStateException("Unrecognized sync mode: " + writeConfig.getSyncMode());
+ }
+ }
+ sqlOperations.executeTransaction(database, queryList);
+ LOGGER.info("Preparing raw tables in destination completed.");
+ };
+ }
+
+ /**
+ * Writes {@link AirbyteRecordMessage} to JDBC database's airbyte_raw table
+ *
+ * @param database JDBC database to connect to
+ * @param sqlOperations interface of SQL queries to execute
+ * @param writeConfigs settings for each stream
+ * @param catalog catalog of all streams to sync
+ * @return
+ */
+ private static RecordWriter recordWriterFunction(final JdbcDatabase database,
+ final SqlOperations sqlOperations,
+ final List writeConfigs,
+ final ConfiguredAirbyteCatalog catalog) {
+ final Map pairToWriteConfig = writeConfigs.stream()
+ .collect(Collectors.toUnmodifiableMap(JdbcBufferedConsumerFactory::toNameNamespacePair, Function.identity()));
+
+ return (pair, records) -> {
+ if (!pairToWriteConfig.containsKey(pair)) {
+ throw new IllegalArgumentException(
+ String.format("Message contained record from a stream that was not in the catalog. \ncatalog: %s", Jsons.serialize(catalog)));
+ }
+
+ final WriteConfig writeConfig = pairToWriteConfig.get(pair);
+ sqlOperations.insertRecords(database, records, writeConfig.getOutputSchemaName(), writeConfig.getOutputTableName());
+ };
+ }
+
+ /**
+ * Closes connection to JDBC database and other tear down functionality
+ *
+ * @param database JDBC database to connect to
+ * @param sqlOperations interface used to execute SQL queries
+ * @param writeConfigs settings for each stream
+ * @return
+ */
+ private static OnCloseFunction onCloseFunction(final JdbcDatabase database,
+ final SqlOperations sqlOperations,
+ final List writeConfigs) {
+ return (hasFailed) -> {
+ if (!hasFailed) {
+ sqlOperations.onDestinationCloseOperations(database, writeConfigs);
+ }
+ };
+ }
+
+ private static AirbyteStreamNameNamespacePair toNameNamespacePair(final WriteConfig config) {
+ return new AirbyteStreamNameNamespacePair(config.getStreamName(), config.getNamespace());
+ }
+
+}
diff --git a/airbyte-integrations/bases/bases-destination-jdbc-async/src/main/java/io/airbyte/integrations/destination/jdbc/JdbcSqlOperations.java b/airbyte-integrations/bases/bases-destination-jdbc-async/src/main/java/io/airbyte/integrations/destination/jdbc/JdbcSqlOperations.java
new file mode 100644
index 0000000000000..522c5fda8c47b
--- /dev/null
+++ b/airbyte-integrations/bases/bases-destination-jdbc-async/src/main/java/io/airbyte/integrations/destination/jdbc/JdbcSqlOperations.java
@@ -0,0 +1,165 @@
+/*
+ * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.integrations.destination.jdbc;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import io.airbyte.commons.exceptions.ConfigErrorException;
+import io.airbyte.commons.json.Jsons;
+import io.airbyte.db.jdbc.JdbcDatabase;
+import io.airbyte.integrations.base.JavaBaseConstants;
+import io.airbyte.protocol.models.v0.AirbyteRecordMessage;
+import java.io.File;
+import java.io.PrintWriter;
+import java.nio.charset.StandardCharsets;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.commons.csv.CSVFormat;
+import org.apache.commons.csv.CSVPrinter;
+
+@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+public abstract class JdbcSqlOperations implements SqlOperations {
+
+ protected static final String SHOW_SCHEMAS = "show schemas;";
+ protected static final String NAME = "name";
+
+ // this adapter modifies record message before inserting them to the destination
+ protected final Optional dataAdapter;
+ private final Set schemaSet = new HashSet<>();
+
+ protected JdbcSqlOperations() {
+ this.dataAdapter = Optional.empty();
+ }
+
+ protected JdbcSqlOperations(final DataAdapter dataAdapter) {
+ this.dataAdapter = Optional.of(dataAdapter);
+ }
+
+ @Override
+ public void createSchemaIfNotExists(final JdbcDatabase database, final String schemaName) throws Exception {
+ try {
+ if (!schemaSet.contains(schemaName) && !isSchemaExists(database, schemaName)) {
+ database.execute(String.format("CREATE SCHEMA IF NOT EXISTS %s;", schemaName));
+ schemaSet.add(schemaName);
+ }
+ } catch (Exception e) {
+ throw checkForKnownConfigExceptions(e).orElseThrow(() -> e);
+ }
+ }
+
+ /**
+ * When an exception occurs, we may recognize it as an issue with the users permissions or other
+ * configuration options. In these cases, we can wrap the exception in a
+ * {@link ConfigErrorException} which will exclude the error from our on-call paging/reporting
+ *
+ * @param e the exception to check.
+ * @return A ConfigErrorException with a message with actionable feedback to the user.
+ */
+ protected Optional checkForKnownConfigExceptions(Exception e) {
+ return Optional.empty();
+ }
+
+ @Override
+ public void createTableIfNotExists(final JdbcDatabase database, final String schemaName, final String tableName) throws SQLException {
+ try {
+ database.execute(createTableQuery(database, schemaName, tableName));
+ } catch (SQLException e) {
+ throw checkForKnownConfigExceptions(e).orElseThrow(() -> e);
+ }
+ }
+
+ @Override
+ public String createTableQuery(final JdbcDatabase database, final String schemaName, final String tableName) {
+ return String.format(
+ "CREATE TABLE IF NOT EXISTS %s.%s ( \n"
+ + "%s VARCHAR PRIMARY KEY,\n"
+ + "%s JSONB,\n"
+ + "%s TIMESTAMP WITH TIME ZONE DEFAULT CURRENT_TIMESTAMP\n"
+ + ");\n",
+ schemaName, tableName, JavaBaseConstants.COLUMN_NAME_AB_ID, JavaBaseConstants.COLUMN_NAME_DATA, JavaBaseConstants.COLUMN_NAME_EMITTED_AT);
+ }
+
+ protected void writeBatchToFile(final File tmpFile, final List records) throws Exception {
+ try (final PrintWriter writer = new PrintWriter(tmpFile, StandardCharsets.UTF_8);
+ final CSVPrinter csvPrinter = new CSVPrinter(writer, CSVFormat.DEFAULT)) {
+ for (final AirbyteRecordMessage record : records) {
+ final var uuid = UUID.randomUUID().toString();
+ final var jsonData = Jsons.serialize(formatData(record.getData()));
+ final var emittedAt = Timestamp.from(Instant.ofEpochMilli(record.getEmittedAt()));
+ csvPrinter.printRecord(uuid, jsonData, emittedAt);
+ }
+ }
+ }
+
+ protected JsonNode formatData(final JsonNode data) {
+ return data;
+ }
+
+ @Override
+ public String truncateTableQuery(final JdbcDatabase database, final String schemaName, final String tableName) {
+ return String.format("TRUNCATE TABLE %s.%s;\n", schemaName, tableName);
+ }
+
+ @Override
+ public String insertTableQuery(final JdbcDatabase database, final String schemaName, final String srcTableName, final String dstTableName) {
+ return String.format("INSERT INTO %s.%s SELECT * FROM %s.%s;\n", schemaName, dstTableName, schemaName, srcTableName);
+ }
+
+ @Override
+ public void executeTransaction(final JdbcDatabase database, final List queries) throws Exception {
+ final StringBuilder appendedQueries = new StringBuilder();
+ appendedQueries.append("BEGIN;\n");
+ for (final String query : queries) {
+ appendedQueries.append(query);
+ }
+ appendedQueries.append("COMMIT;");
+ database.execute(appendedQueries.toString());
+ }
+
+ @Override
+ public void dropTableIfExists(final JdbcDatabase database, final String schemaName, final String tableName) throws SQLException {
+ try {
+ database.execute(dropTableIfExistsQuery(schemaName, tableName));
+ } catch (SQLException e) {
+ throw checkForKnownConfigExceptions(e).orElseThrow(() -> e);
+ }
+ }
+
+ private String dropTableIfExistsQuery(final String schemaName, final String tableName) {
+ return String.format("DROP TABLE IF EXISTS %s.%s;\n", schemaName, tableName);
+ }
+
+ @Override
+ public boolean isSchemaRequired() {
+ return true;
+ }
+
+ @Override
+ public boolean isValidData(final JsonNode data) {
+ return true;
+ }
+
+ @Override
+ public final void insertRecords(final JdbcDatabase database,
+ final List records,
+ final String schemaName,
+ final String tableName)
+ throws Exception {
+ dataAdapter.ifPresent(adapter -> records.forEach(airbyteRecordMessage -> adapter.adapt(airbyteRecordMessage.getData())));
+ insertRecordsInternal(database, records, schemaName, tableName);
+ }
+
+ protected abstract void insertRecordsInternal(JdbcDatabase database,
+ List