metadataInjector) {
- super();
- this.metadataInjector = metadataInjector;
- }
-
- @Override
- public AirbyteRecordData convertDatabaseRowToAirbyteRecordData(final ResultSet queryContext) throws SQLException {
- final AirbyteRecordData recordData = super.convertDatabaseRowToAirbyteRecordData(queryContext);
- final ObjectNode jsonNode = (ObjectNode) recordData.rawRowData();
- if (!metadataInjector.isPresent()) {
- return recordData;
- }
- metadataInjector.get().inject(jsonNode);
- return new AirbyteRecordData(jsonNode, recordData.meta());
- }
-
- /**
- * @param colIndex 1-based column index.
- */
- @Override
- public void copyToJsonField(final ResultSet resultSet, final int colIndex, final ObjectNode json) throws SQLException {
- final ResultSetMetaData metaData = (ResultSetMetaData) resultSet.getMetaData();
- final Field field = metaData.getFields()[colIndex - 1];
- final String columnName = field.getName();
- final MysqlType columnType = field.getMysqlType();
-
- // Attempt to access the column. this allows us to know if it is null before we do
- // type-specific parsing. If the column is null, we will populate the null value and skip attempting
- // to
- // parse the column value.
- resultSet.getObject(colIndex);
- if (resultSet.wasNull()) {
- json.putNull(columnName);
- } else {
- // https://dev.mysql.com/doc/connector-j/8.0/en/connector-j-reference-type-conversions.html
- switch (columnType) {
- case BIT -> {
- if (field.getLength() == 1L) {
- // BIT(1) is boolean
- putBoolean(json, columnName, resultSet, colIndex);
- } else {
- putBinary(json, columnName, resultSet, colIndex);
- }
- }
- case BOOLEAN -> putBoolean(json, columnName, resultSet, colIndex);
- case TINYINT -> {
- if (field.getLength() == 1L) {
- // TINYINT(1) is boolean
- putBoolean(json, columnName, resultSet, colIndex);
- } else {
- putShortInt(json, columnName, resultSet, colIndex);
- }
- }
- case TINYINT_UNSIGNED, YEAR -> putShortInt(json, columnName, resultSet, colIndex);
- case SMALLINT, SMALLINT_UNSIGNED, MEDIUMINT, MEDIUMINT_UNSIGNED -> putInteger(json, columnName, resultSet, colIndex);
- case INT, INT_UNSIGNED -> {
- if (field.isUnsigned()) {
- putBigInt(json, columnName, resultSet, colIndex);
- } else {
- putInteger(json, columnName, resultSet, colIndex);
- }
- }
- case BIGINT, BIGINT_UNSIGNED -> putBigInt(json, columnName, resultSet, colIndex);
- case FLOAT, FLOAT_UNSIGNED -> putFloat(json, columnName, resultSet, colIndex);
- case DOUBLE, DOUBLE_UNSIGNED -> putDouble(json, columnName, resultSet, colIndex);
- case DECIMAL, DECIMAL_UNSIGNED -> {
- if (field.getDecimals() == 0) {
- putBigInt(json, columnName, resultSet, colIndex);
- } else {
- putBigDecimal(json, columnName, resultSet, colIndex);
- }
- }
- case DATE -> putDate(json, columnName, resultSet, colIndex);
- case DATETIME -> putTimestamp(json, columnName, resultSet, colIndex);
- case TIMESTAMP -> putTimestampWithTimezone(json, columnName, resultSet, colIndex);
- case TIME -> putTime(json, columnName, resultSet, colIndex);
- case CHAR, VARCHAR -> putString(json, columnName, resultSet, colIndex);
- case TINYBLOB, BLOB, MEDIUMBLOB, LONGBLOB, BINARY, VARBINARY, GEOMETRY -> putBinary(json, columnName, resultSet, colIndex);
- case TINYTEXT, TEXT, MEDIUMTEXT, LONGTEXT, JSON, ENUM, SET -> putString(json, columnName, resultSet, colIndex);
- case NULL -> json.set(columnName, NullNode.instance);
- default -> putDefault(json, columnName, resultSet, colIndex);
- }
- }
- }
-
- /**
- * MySQL boolean is equivalent to tinyint(1).
- */
- @Override
- protected void putBoolean(final ObjectNode node, final String columnName, final ResultSet resultSet, final int index) throws SQLException {
- node.put(columnName, resultSet.getInt(index) > 0);
- }
-
- @Override
- public void setCursorField(final PreparedStatement preparedStatement,
- final int parameterIndex,
- final MysqlType cursorFieldType,
- final String value)
- throws SQLException {
- switch (cursorFieldType) {
- case BIT -> setBit(preparedStatement, parameterIndex, value);
- case BOOLEAN -> setBoolean(preparedStatement, parameterIndex, value);
- case YEAR, TINYINT, TINYINT_UNSIGNED, SMALLINT, SMALLINT_UNSIGNED, MEDIUMINT, MEDIUMINT_UNSIGNED -> setInteger(preparedStatement,
- parameterIndex,
- value);
- case INT, INT_UNSIGNED, BIGINT, BIGINT_UNSIGNED -> setBigInteger(preparedStatement, parameterIndex, value);
- case FLOAT, FLOAT_UNSIGNED, DOUBLE, DOUBLE_UNSIGNED -> setDouble(preparedStatement, parameterIndex, value);
- case DECIMAL, DECIMAL_UNSIGNED -> setDecimal(preparedStatement, parameterIndex, value);
- case DATE -> setDate(preparedStatement, parameterIndex, value);
- case DATETIME -> setTimestamp(preparedStatement, parameterIndex, value);
- case TIMESTAMP -> setTimestampWithTimezone(preparedStatement, parameterIndex, value);
- case TIME -> setTime(preparedStatement, parameterIndex, value);
- case CHAR, VARCHAR, TINYTEXT, TEXT, MEDIUMTEXT, LONGTEXT, ENUM, SET -> setString(preparedStatement, parameterIndex, value);
- case TINYBLOB, BLOB, MEDIUMBLOB, LONGBLOB, BINARY, VARBINARY -> setBinary(preparedStatement, parameterIndex, value);
- // since cursor are expected to be comparable, handle cursor typing strictly and error on
- // unrecognized types
- default -> throw new IllegalArgumentException(String.format("%s cannot be used as a cursor.", cursorFieldType));
- }
- }
-
- @Override
- public MysqlType getDatabaseFieldType(final JsonNode field) {
- try {
- // MysqlType#getByName can handle the full MySQL type name
- // e.g. MEDIUMINT UNSIGNED
- final MysqlType literalType = MysqlType.getByName(field.get(INTERNAL_COLUMN_TYPE_NAME).asText());
- final int columnSize = field.get(INTERNAL_COLUMN_SIZE).asInt();
- switch (literalType) {
- // BIT(1) and TINYINT(1) are interpreted as boolean
- case BIT, TINYINT -> {
- if (columnSize == 1) {
- return MysqlType.BOOLEAN;
- }
- }
- case YEAR -> {
- return SMALLINT;
- }
- // When CHAR[N] and VARCHAR[N] columns have binary character set, the returned
- // types are BINARY[N] and VARBINARY[N], respectively. So we don't need to
- // convert them here. This is verified in MySqlSourceDatatypeTest.
- case DECIMAL -> {
- if (field.get(INTERNAL_DECIMAL_DIGITS) != null && field.get(INTERNAL_DECIMAL_DIGITS).asInt() == 0) {
- return BIGINT;
- }
- }
- case DECIMAL_UNSIGNED -> {
- if (field.get(INTERNAL_DECIMAL_DIGITS) != null && field.get(INTERNAL_DECIMAL_DIGITS).asInt() == 0) {
- return BIGINT_UNSIGNED;
- }
- }
- }
- return literalType;
- } catch (final IllegalArgumentException ex) {
- LOGGER.warn(String.format("Could not convert column: %s from table: %s.%s with type: %s (type name: %s). Casting to VARCHAR.",
- field.get(INTERNAL_COLUMN_NAME),
- field.get(INTERNAL_SCHEMA_NAME),
- field.get(INTERNAL_TABLE_NAME),
- field.get(INTERNAL_COLUMN_TYPE),
- field.get(INTERNAL_COLUMN_TYPE_NAME)));
- return MysqlType.VARCHAR;
- }
- }
-
- @Override
- public boolean isCursorType(final MysqlType type) {
- return ALLOWED_CURSOR_TYPES.contains(type);
- }
-
- @Override
- public JsonSchemaType getAirbyteType(final MysqlType mysqlType) {
- return switch (mysqlType) {
- case
- // TINYINT(1) is boolean, but it should have been converted to MysqlType.BOOLEAN in {@link
- // getFieldType}
- TINYINT, TINYINT_UNSIGNED, SMALLINT, SMALLINT_UNSIGNED, INT, MEDIUMINT, MEDIUMINT_UNSIGNED, INT_UNSIGNED, BIGINT, BIGINT_UNSIGNED -> JsonSchemaType.INTEGER;
- case FLOAT, FLOAT_UNSIGNED, DOUBLE, DOUBLE_UNSIGNED, DECIMAL, DECIMAL_UNSIGNED -> JsonSchemaType.NUMBER;
- case BOOLEAN -> JsonSchemaType.BOOLEAN;
- case NULL -> JsonSchemaType.NULL;
- // BIT(1) is boolean, but it should have been converted to MysqlType.BOOLEAN in {@link getFieldType}
- case BIT, TINYBLOB, BLOB, MEDIUMBLOB, LONGBLOB, BINARY, VARBINARY, GEOMETRY -> JsonSchemaType.STRING_BASE_64;
- case TIME -> JsonSchemaType.STRING_TIME_WITHOUT_TIMEZONE;
- case DATETIME -> JsonSchemaType.STRING_TIMESTAMP_WITHOUT_TIMEZONE;
- case TIMESTAMP -> JsonSchemaType.STRING_TIMESTAMP_WITH_TIMEZONE;
- case DATE -> JsonSchemaType.STRING_DATE;
- default -> JsonSchemaType.STRING;
- };
- }
-
- @Override
- protected void setDate(final PreparedStatement preparedStatement, final int parameterIndex, final String value) throws SQLException {
- try {
- preparedStatement.setObject(parameterIndex, LocalDate.parse(value));
- } catch (final DateTimeParseException e) {
- // This is just for backward compatibility for connectors created on versions before PR
- // https://github.com/airbytehq/airbyte/pull/15504
- LOGGER.warn("Exception occurred while trying to parse value for date column the new way, trying the old way", e);
- super.setDate(preparedStatement, parameterIndex, value);
- }
- }
-
- @Override
- protected void setTimestamp(final PreparedStatement preparedStatement, final int parameterIndex, final String value) throws SQLException {
- try {
- preparedStatement.setObject(parameterIndex, LocalDateTime.parse(value));
- } catch (final DateTimeParseException e) {
- // This is just for backward compatibility for connectors created on versions before PR
- // https://github.com/airbytehq/airbyte/pull/15504
- LOGGER.warn("Exception occurred while trying to parse value for datetime column the new way, trying the old way", e);
- preparedStatement.setObject(parameterIndex, OffsetDateTime.parse(value));
- }
- }
-
- private void setTimestampWithTimezone(final PreparedStatement preparedStatement, final int parameterIndex, final String value) throws SQLException {
- try {
- preparedStatement.setObject(parameterIndex, OffsetDateTime.parse(value));
- } catch (final DateTimeParseException e) {
- // This is just for backward compatibility for connectors created on versions before PR
- // https://github.com/airbytehq/airbyte/pull/15504
- LOGGER.warn("Exception occurred while trying to parse value for timestamp column the new way, trying the old way", e);
- preparedStatement.setObject(parameterIndex, LocalDateTime.parse(value));
- }
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/MySqlSpecConstants.java b/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/MySqlSpecConstants.java
deleted file mode 100644
index 7735470482da..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/MySqlSpecConstants.java
+++ /dev/null
@@ -1,15 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.source.mysql;
-
-// Constants defined in
-// airbyte-integrations/connectors/source-postgres/src/main/resources/spec.json.
-public class MySqlSpecConstants {
-
- public static final String INVALID_CDC_CURSOR_POSITION_PROPERTY = "invalid_cdc_cursor_position_behavior";
- public static final String FAIL_SYNC_OPTION = "Fail sync";
- public static final String RESYNC_DATA_OPTION = "Re-sync data";
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/MySqlStreamingQueryConfig.java b/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/MySqlStreamingQueryConfig.java
deleted file mode 100644
index 029cfc2c3d66..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/MySqlStreamingQueryConfig.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.source.mysql;
-
-import io.airbyte.cdk.db.jdbc.streaming.AdaptiveStreamingQueryConfig;
-import java.sql.Connection;
-import java.sql.SQLException;
-import java.sql.Statement;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class MySqlStreamingQueryConfig extends AdaptiveStreamingQueryConfig {
-
- private static final Logger LOGGER = LoggerFactory.getLogger(MySqlStreamingQueryConfig.class);
-
- public MySqlStreamingQueryConfig() {
- super();
- }
-
- @Override
- public void initialize(final Connection connection, final Statement preparedStatement) throws SQLException {
- preparedStatement.setFetchSize(Integer.MIN_VALUE);
- LOGGER.info("Set initial fetch size: {} rows", preparedStatement.getFetchSize());
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/CdcConfigurationHelper.java b/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/CdcConfigurationHelper.java
deleted file mode 100644
index 2f5d02caf72f..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/CdcConfigurationHelper.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.source.mysql.cdc;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import io.airbyte.cdk.db.jdbc.JdbcDatabase;
-import io.airbyte.commons.exceptions.ConfigErrorException;
-import io.airbyte.commons.functional.CheckedConsumer;
-import java.sql.SQLException;
-import java.time.ZoneId;
-import java.util.List;
-import java.util.Optional;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Helper class for MySqlSource used to check cdc configuration in case of:
- *
- * 1. adding new source and checking operations #getCheckOperations method.
- *
- *
- * 2. checking whether binlog required from saved cdc offset is available on mysql server
- * #checkBinlog method
- *
- * 3. configuring initial CDC wait time. TODO : There is a lot of shared logic for this
- * functionality between MySQL and Postgres. Refactor it to reduce code de-duplication.
- */
-public class CdcConfigurationHelper {
-
- private static final Logger LOGGER = LoggerFactory.getLogger(CdcConfigurationHelper.class);
- private static final String LOG_BIN = "log_bin";
- private static final String BINLOG_FORMAT = "binlog_format";
- private static final String BINLOG_ROW_IMAGE = "binlog_row_image";
-
- /**
- * Method will get required configurations for cdc sync
- *
- * @return list of List>
- */
- public static List> getCheckOperations() {
- return List.of(getMasterStatusOperation(),
- getCheckOperation(LOG_BIN, "ON"),
- getCheckOperation(BINLOG_FORMAT, "ROW"),
- getCheckOperation(BINLOG_ROW_IMAGE, "FULL"));
-
- }
-
- // Checks whether the user has REPLICATION CLIENT privilege needed to query status information about
- // the binary log files, which are needed for CDC.
- private static CheckedConsumer getMasterStatusOperation() {
- return database -> {
- try {
- database.unsafeResultSetQuery(
- connection -> connection.createStatement().executeQuery("SHOW MASTER STATUS"),
- resultSet -> resultSet);
- } catch (final SQLException e) {
- throw new ConfigErrorException("Please grant REPLICATION CLIENT privilege, so that binary log files are available"
- + " for CDC mode.");
- }
- };
- }
-
- private static CheckedConsumer getCheckOperation(final String name, final String value) {
- return database -> {
- final List result = database.queryStrings(
- connection -> connection.createStatement().executeQuery(String.format("show variables where Variable_name = '%s'", name)),
- resultSet -> resultSet.getString("Value"));
-
- if (result.size() != 1) {
- throw new RuntimeException("Could not query the variable " + name);
- }
-
- final String resultValue = result.get(0);
- if (!resultValue.equalsIgnoreCase(value)) {
- throw new RuntimeException(String.format("The variable \"%s\" should be set to \"%s\", but it is \"%s\"", name, value, resultValue));
- }
- };
- }
-
- private static Optional getCdcServerTimezone(final JsonNode config) {
- final JsonNode replicationMethod = config.get("replication_method");
- if (replicationMethod != null && replicationMethod.has("server_time_zone")) {
- final String serverTimeZone = config.get("replication_method").get("server_time_zone").asText();
- return Optional.of(serverTimeZone);
- }
- return Optional.empty();
- }
-
- public static void checkServerTimeZoneConfig(final JsonNode config) {
- final Optional serverTimeZone = getCdcServerTimezone(config);
- if (serverTimeZone.isPresent()) {
- final String timeZone = serverTimeZone.get();
- if (!timeZone.isEmpty() && !ZoneId.getAvailableZoneIds().contains((timeZone))) {
- throw new IllegalArgumentException(String.format("Given timezone %s is not valid. The given timezone must conform to the IANNA standard. "
- + "See https://www.iana.org/time-zones for more details", serverTimeZone.get()));
- }
- }
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/CustomMySQLTinyIntOneToBooleanConverter.java b/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/CustomMySQLTinyIntOneToBooleanConverter.java
deleted file mode 100644
index 38a4162287b7..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/CustomMySQLTinyIntOneToBooleanConverter.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.source.mysql.cdc;
-
-import io.debezium.connector.binlog.converters.TinyIntOneToBooleanConverter;
-import io.debezium.spi.converter.RelationalColumn;
-import org.apache.kafka.connect.data.SchemaBuilder;
-
-public class CustomMySQLTinyIntOneToBooleanConverter extends TinyIntOneToBooleanConverter {
-
- @Override
- public void converterFor(final RelationalColumn field, final ConverterRegistration registration) {
- if (!"TINYINT".equalsIgnoreCase(field.typeName())) {
- return;
- }
- super.converterFor(field, registration);
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySQLDateTimeConverter.java b/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySQLDateTimeConverter.java
deleted file mode 100644
index 233948c5b31b..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySQLDateTimeConverter.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.source.mysql.cdc;
-
-import io.airbyte.cdk.db.jdbc.DateTimeConverter;
-import io.airbyte.cdk.db.jdbc.JdbcDatabase;
-import io.airbyte.cdk.integrations.debezium.internals.DebeziumConverterUtils;
-import io.debezium.spi.converter.CustomConverter;
-import io.debezium.spi.converter.RelationalColumn;
-import io.debezium.time.Conversions;
-import java.time.LocalDate;
-import java.time.LocalTime;
-import java.util.Arrays;
-import java.util.Locale;
-import java.util.Properties;
-import java.util.concurrent.TimeUnit;
-import org.apache.kafka.connect.data.SchemaBuilder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This is a custom debezium converter used in MySQL to handle the DATETIME data type. We need a
- * custom converter cause by default debezium returns the DATETIME values as numbers. We need to
- * convert it to proper format. Ref :
- * https://debezium.io/documentation/reference/2.1/development/converters.html This is built from
- * reference with {@link io.debezium.connector.mysql.converters.TinyIntOneToBooleanConverter} If you
- * rename this class then remember to rename the datetime.type property value in
- * {@link MySqlCdcProperties#commonProperties(JdbcDatabase)} (If you don't rename, a test would
- * still fail but it might be tricky to figure out where to change the property name)
- */
-public class MySQLDateTimeConverter implements CustomConverter {
-
- private static final Logger LOGGER = LoggerFactory.getLogger(MySQLDateTimeConverter.class);
-
- private final String[] DATE_TYPES = {"DATE", "DATETIME", "TIME", "TIMESTAMP"};
-
- @Override
- public void configure(final Properties props) {}
-
- @Override
- public void converterFor(final RelationalColumn field, final ConverterRegistration registration) {
- if (Arrays.stream(DATE_TYPES).anyMatch(s -> s.equalsIgnoreCase(field.typeName()))) {
- registerDate(field, registration);
- }
- }
-
- private int getTimePrecision(final RelationalColumn field) {
- return field.length().orElse(-1);
- }
-
- // Ref :
- // https://debezium.io/documentation/reference/2.1/connectors/mysql.html#mysql-temporal-types
- private void registerDate(final RelationalColumn field, final ConverterRegistration registration) {
- final var fieldType = field.typeName();
-
- registration.register(SchemaBuilder.string().optional(), x -> {
- if (x == null) {
- return DebeziumConverterUtils.convertDefaultValue(field);
- }
-
- switch (fieldType.toUpperCase(Locale.ROOT)) {
- case "DATETIME":
- if (x instanceof final Long l) {
- if (getTimePrecision(field) <= 3) {
- return DateTimeConverter.convertToTimestamp(Conversions.toInstantFromMillis(l));
- }
- if (getTimePrecision(field) <= 6) {
- return DateTimeConverter.convertToTimestamp(Conversions.toInstantFromMicros(l));
- }
- }
- return DateTimeConverter.convertToTimestamp(x);
- case "DATE":
- if (x instanceof final Integer i) {
- return DateTimeConverter.convertToDate(LocalDate.ofEpochDay(i));
- }
- return DateTimeConverter.convertToDate(x);
- case "TIME":
- if (x instanceof Long) {
- long l = Math.multiplyExact((Long) x, TimeUnit.MICROSECONDS.toNanos(1));
- return DateTimeConverter.convertToTime(LocalTime.ofNanoOfDay(l));
- }
- return DateTimeConverter.convertToTime(x);
- case "TIMESTAMP":
- return DateTimeConverter.convertToTimestampWithTimezone(x);
- default:
- throw new IllegalArgumentException("Unknown field type " + fieldType.toUpperCase(Locale.ROOT));
- }
- });
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySqlCdcConnectorMetadataInjector.java b/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySqlCdcConnectorMetadataInjector.java
deleted file mode 100644
index d43f83e725c9..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySqlCdcConnectorMetadataInjector.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.source.mysql.cdc;
-
-import static io.airbyte.cdk.integrations.debezium.internals.DebeziumEventConverter.CDC_DELETED_AT;
-import static io.airbyte.cdk.integrations.debezium.internals.DebeziumEventConverter.CDC_UPDATED_AT;
-import static io.airbyte.integrations.source.mysql.MySqlSource.CDC_DEFAULT_CURSOR;
-import static io.airbyte.integrations.source.mysql.MySqlSource.CDC_LOG_FILE;
-import static io.airbyte.integrations.source.mysql.MySqlSource.CDC_LOG_POS;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-import io.airbyte.cdk.integrations.debezium.CdcMetadataInjector;
-import io.airbyte.integrations.source.mysql.cdc.MySqlDebeziumStateUtil.MysqlDebeziumStateAttributes;
-import java.time.Instant;
-import java.util.concurrent.atomic.AtomicLong;
-
-public class MySqlCdcConnectorMetadataInjector implements CdcMetadataInjector {
-
- private final long emittedAtConverted;
-
- // This now makes this class stateful. Please make sure to use the same instance within a sync
- private final AtomicLong recordCounter = new AtomicLong(1);
- private static final long ONE_HUNDRED_MILLION = 100_000_000;
- private static MySqlCdcConnectorMetadataInjector mySqlCdcConnectorMetadataInjector;
-
- private MySqlCdcConnectorMetadataInjector(final Instant emittedAt) {
- this.emittedAtConverted = emittedAt.getEpochSecond() * ONE_HUNDRED_MILLION;
- }
-
- public static MySqlCdcConnectorMetadataInjector getInstance(final Instant emittedAt) {
- if (mySqlCdcConnectorMetadataInjector == null) {
- mySqlCdcConnectorMetadataInjector = new MySqlCdcConnectorMetadataInjector(emittedAt);
- }
-
- return mySqlCdcConnectorMetadataInjector;
- }
-
- @Override
- public void addMetaData(final ObjectNode event, final JsonNode source) {
- event.put(CDC_LOG_FILE, source.get("file").asText());
- event.put(CDC_LOG_POS, source.get("pos").asLong());
- event.put(CDC_DEFAULT_CURSOR, getCdcDefaultCursor());
- }
-
- @Override
- public void addMetaDataToRowsFetchedOutsideDebezium(final ObjectNode record,
- final String transactionTimestamp,
- final MysqlDebeziumStateAttributes debeziumStateAttributes) {
- record.put(CDC_UPDATED_AT, transactionTimestamp);
- record.put(CDC_LOG_FILE, debeziumStateAttributes.binlogFilename());
- record.put(CDC_LOG_POS, debeziumStateAttributes.binlogPosition());
- record.put(CDC_DELETED_AT, (String) null);
- record.put(CDC_DEFAULT_CURSOR, getCdcDefaultCursor());
- }
-
- @Override
- public String namespace(final JsonNode source) {
- return source.get("db").asText();
- }
-
- @Override
- public String name(JsonNode source) {
- return source.get("table").asText();
- }
-
- private Long getCdcDefaultCursor() {
- return this.emittedAtConverted + this.recordCounter.getAndIncrement();
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySqlCdcPosition.java b/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySqlCdcPosition.java
deleted file mode 100644
index 04cc8430142f..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySqlCdcPosition.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.source.mysql.cdc;
-
-import java.util.Objects;
-
-public class MySqlCdcPosition {
-
- public final String fileName;
- public final Long position;
-
- public MySqlCdcPosition(final String fileName, final Long position) {
- this.fileName = fileName;
- this.position = position;
- }
-
- @Override
- public boolean equals(final Object obj) {
- if (obj instanceof final MySqlCdcPosition mySqlCdcPosition) {
- return fileName.equals(mySqlCdcPosition.fileName) && mySqlCdcPosition.position.equals(position);
- }
- return false;
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(fileName, position);
- }
-
- @Override
- public String toString() {
- return "FileName: " + fileName + ", Position : " + position;
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySqlCdcProperties.java b/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySqlCdcProperties.java
deleted file mode 100644
index 9b54d0588f67..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySqlCdcProperties.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.source.mysql.cdc;
-
-import static io.airbyte.cdk.integrations.source.jdbc.JdbcSSLConnectionUtils.CLIENT_KEY_STORE_PASS;
-import static io.airbyte.cdk.integrations.source.jdbc.JdbcSSLConnectionUtils.CLIENT_KEY_STORE_URL;
-import static io.airbyte.cdk.integrations.source.jdbc.JdbcSSLConnectionUtils.SSL_MODE;
-import static io.airbyte.cdk.integrations.source.jdbc.JdbcSSLConnectionUtils.TRUST_KEY_STORE_PASS;
-import static io.airbyte.cdk.integrations.source.jdbc.JdbcSSLConnectionUtils.TRUST_KEY_STORE_URL;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import io.airbyte.cdk.db.jdbc.JdbcDatabase;
-import io.airbyte.cdk.db.jdbc.JdbcUtils;
-import io.airbyte.cdk.integrations.source.jdbc.JdbcSSLConnectionUtils.SslMode;
-import io.airbyte.integrations.source.mysql.MySqlSource;
-import java.net.URI;
-import java.nio.file.Path;
-import java.time.Duration;
-import java.util.Properties;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class MySqlCdcProperties {
-
- private static final Logger LOGGER = LoggerFactory.getLogger(MySqlCdcProperties.class);
- private static final Duration HEARTBEAT_INTERVAL = Duration.ofSeconds(10L);
-
- // Test execution latency is lower when heartbeats are more frequent.
- private static final Duration HEARTBEAT_INTERVAL_IN_TESTS = Duration.ofSeconds(1L);
-
- public static Properties getDebeziumProperties(final JdbcDatabase database) {
- final JsonNode sourceConfig = database.getSourceConfig();
- final Properties props = commonProperties(database);
- // snapshot config
- if (sourceConfig.has("snapshot_mode")) {
- // The parameter `snapshot_mode` is passed in test to simulate reading the binlog directly and skip
- // initial snapshot
- props.setProperty("snapshot.mode", sourceConfig.get("snapshot_mode").asText());
- } else {
- // https://debezium.io/documentation/reference/2.2/connectors/mysql.html#mysql-property-snapshot-mode
- props.setProperty("snapshot.mode", "when_needed");
- }
-
- return props;
- }
-
- private static Properties commonProperties(final JdbcDatabase database) {
- final Properties props = new Properties();
- final JsonNode sourceConfig = database.getSourceConfig();
- final JsonNode dbConfig = database.getDatabaseConfig();
- // debezium engine configuration
- props.setProperty("connector.class", "io.debezium.connector.mysql.MySqlConnector");
-
- props.setProperty("database.server.id", String.valueOf(generateServerID()));
- // https://debezium.io/documentation/reference/2.2/connectors/mysql.html#mysql-boolean-values
- // https://debezium.io/documentation/reference/2.2/development/converters.html
- /**
- * {@link io.debezium.connector.mysql.converters.TinyIntOneToBooleanConverter}
- * {@link MySQLConverter}
- */
- props.setProperty("converters", "boolean, datetime");
- props.setProperty("boolean.type", CustomMySQLTinyIntOneToBooleanConverter.class.getName());
- props.setProperty("datetime.type", MySQLDateTimeConverter.class.getName());
-
- final Duration heartbeatInterval =
- (database.getSourceConfig().has("is_test") && database.getSourceConfig().get("is_test").asBoolean())
- ? HEARTBEAT_INTERVAL_IN_TESTS
- : HEARTBEAT_INTERVAL;
- props.setProperty("heartbeat.interval.ms", Long.toString(heartbeatInterval.toMillis()));
-
- // For CDC mode, the user cannot provide timezone arguments as JDBC parameters - they are
- // specifically defined in the replication_method
- // config.
- if (sourceConfig.get("replication_method").has("server_time_zone")) {
- final String serverTimeZone = sourceConfig.get("replication_method").get("server_time_zone").asText();
- if (!serverTimeZone.isEmpty()) {
- /**
- * Per Debezium docs,
- * https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-temporal-types
- * this property is now connectionTimeZone {@link com.mysql.cj.conf.PropertyKey#connectionTimeZone}
- **/
- props.setProperty("database.connectionTimeZone", serverTimeZone);
- }
- }
-
- // Check params for SSL connection in config and add properties for CDC SSL connection
- // https://debezium.io/documentation/reference/2.2/connectors/mysql.html#mysql-property-database-ssl-mode
- if (!sourceConfig.has(JdbcUtils.SSL_KEY) || sourceConfig.get(JdbcUtils.SSL_KEY).asBoolean()) {
- if (dbConfig.has(SSL_MODE) && !dbConfig.get(SSL_MODE).asText().isEmpty()) {
- props.setProperty("database.ssl.mode", MySqlSource.toSslJdbcParam(SslMode.valueOf(dbConfig.get(SSL_MODE).asText())));
-
- if (dbConfig.has(TRUST_KEY_STORE_URL) && !dbConfig.get(TRUST_KEY_STORE_URL).asText().isEmpty()) {
- props.setProperty("database.ssl.truststore", Path.of(URI.create(dbConfig.get(TRUST_KEY_STORE_URL).asText())).toString());
- }
-
- if (dbConfig.has(TRUST_KEY_STORE_PASS) && !dbConfig.get(TRUST_KEY_STORE_PASS).asText().isEmpty()) {
- props.setProperty("database.ssl.truststore.password", dbConfig.get(TRUST_KEY_STORE_PASS).asText());
- }
-
- if (dbConfig.has(CLIENT_KEY_STORE_URL) && !dbConfig.get(CLIENT_KEY_STORE_URL).asText().isEmpty()) {
- props.setProperty("database.ssl.keystore", Path.of(URI.create(dbConfig.get(CLIENT_KEY_STORE_URL).asText())).toString());
- }
-
- if (dbConfig.has(CLIENT_KEY_STORE_PASS) && !dbConfig.get(CLIENT_KEY_STORE_PASS).asText().isEmpty()) {
- props.setProperty("database.ssl.keystore.password", dbConfig.get(CLIENT_KEY_STORE_PASS).asText());
- }
-
- } else {
- props.setProperty("database.ssl.mode", "required");
- }
- }
-
- // https://debezium.io/documentation/reference/2.2/connectors/mysql.html#mysql-property-snapshot-locking-mode
- // This is to make sure other database clients are allowed to write to a table while Airbyte is
- // taking a snapshot. There is a risk involved that
- // if any database client makes a schema change then the sync might break
- props.setProperty("snapshot.locking.mode", "none");
- // https://debezium.io/documentation/reference/2.2/connectors/mysql.html#mysql-property-include-schema-changes
- props.setProperty("include.schema.changes", "false");
- // This to make sure that binary data represented as a base64-encoded String.
- // https://debezium.io/documentation/reference/2.2/connectors/mysql.html#mysql-property-binary-handling-mode
- props.setProperty("binary.handling.mode", "base64");
- props.setProperty("database.include.list", sourceConfig.get("database").asText());
-
- return props;
- }
-
- private static int generateServerID() {
- final int min = 5400;
- final int max = 6400;
-
- final int serverId = (int) Math.floor(Math.random() * (max - min + 1) + min);
- LOGGER.info("Randomly generated Server ID : " + serverId);
- return serverId;
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySqlCdcSavedInfoFetcher.java b/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySqlCdcSavedInfoFetcher.java
deleted file mode 100644
index 7d23671afacb..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySqlCdcSavedInfoFetcher.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.source.mysql.cdc;
-
-import static io.airbyte.integrations.source.mysql.cdc.MysqlCdcStateConstants.IS_COMPRESSED;
-import static io.airbyte.integrations.source.mysql.cdc.MysqlCdcStateConstants.MYSQL_CDC_OFFSET;
-import static io.airbyte.integrations.source.mysql.cdc.MysqlCdcStateConstants.MYSQL_DB_HISTORY;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import io.airbyte.cdk.integrations.debezium.CdcSavedInfoFetcher;
-import io.airbyte.cdk.integrations.debezium.internals.AirbyteSchemaHistoryStorage.SchemaHistory;
-import io.airbyte.cdk.integrations.source.relationaldb.models.CdcState;
-import java.util.Optional;
-
-public class MySqlCdcSavedInfoFetcher implements CdcSavedInfoFetcher {
-
- private final JsonNode savedOffset;
- private final JsonNode savedSchemaHistory;
- private final boolean isSavedSchemaHistoryCompressed;
-
- public MySqlCdcSavedInfoFetcher(final CdcState savedState) {
- final boolean savedStatePresent = savedState != null && savedState.getState() != null;
- this.savedOffset = savedStatePresent ? savedState.getState().get(MYSQL_CDC_OFFSET) : null;
- this.savedSchemaHistory = savedStatePresent ? savedState.getState().get(MYSQL_DB_HISTORY) : null;
- this.isSavedSchemaHistoryCompressed =
- savedStatePresent && savedState.getState().has(IS_COMPRESSED) && savedState.getState().get(IS_COMPRESSED).asBoolean();
- }
-
- @Override
- public JsonNode getSavedOffset() {
- return savedOffset;
- }
-
- @Override
- public SchemaHistory> getSavedSchemaHistory() {
- return new SchemaHistory<>(Optional.ofNullable(savedSchemaHistory), isSavedSchemaHistoryCompressed);
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySqlCdcStateHandler.java b/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySqlCdcStateHandler.java
deleted file mode 100644
index a6f672c7ab40..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySqlCdcStateHandler.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.source.mysql.cdc;
-
-import static io.airbyte.integrations.source.mysql.cdc.MySqlDebeziumStateUtil.serialize;
-import static io.airbyte.integrations.source.mysql.cdc.MysqlCdcStateConstants.COMPRESSION_ENABLED;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import io.airbyte.cdk.integrations.debezium.CdcStateHandler;
-import io.airbyte.cdk.integrations.debezium.internals.AirbyteSchemaHistoryStorage.SchemaHistory;
-import io.airbyte.cdk.integrations.source.relationaldb.models.CdcState;
-import io.airbyte.cdk.integrations.source.relationaldb.state.StateManager;
-import io.airbyte.protocol.models.v0.AirbyteMessage;
-import io.airbyte.protocol.models.v0.AirbyteMessage.Type;
-import io.airbyte.protocol.models.v0.AirbyteStateMessage;
-import java.util.Map;
-import java.util.Optional;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class MySqlCdcStateHandler implements CdcStateHandler {
-
- private static final Logger LOGGER = LoggerFactory.getLogger(MySqlCdcStateHandler.class);
-
- private final StateManager stateManager;
-
- public MySqlCdcStateHandler(final StateManager stateManager) {
- this.stateManager = stateManager;
- }
-
- @Override
- public boolean isCdcCheckpointEnabled() {
- return true;
- }
-
- @Override
- public AirbyteMessage saveState(final Map offset, final SchemaHistory dbHistory) {
- final JsonNode asJson = serialize(offset, dbHistory);
-
- LOGGER.info("debezium state: {}", asJson);
-
- final CdcState cdcState = new CdcState().withState(asJson);
- stateManager.getCdcStateManager().setCdcState(cdcState);
- /*
- * Namespace pair is ignored by global state manager, but is needed for satisfy the API contract.
- * Therefore, provide an empty optional.
- */
- final AirbyteStateMessage stateMessage = stateManager.emit(Optional.empty());
- return new AirbyteMessage().withType(Type.STATE).withState(stateMessage);
- }
-
- @Override
- public AirbyteMessage saveStateAfterCompletionOfSnapshotOfNewStreams() {
- LOGGER.info("Snapshot of new tables is complete, saving state");
- /*
- * Namespace pair is ignored by global state manager, but is needed for satisfy the API contract.
- * Therefore, provide an empty optional.
- */
- final AirbyteStateMessage stateMessage = stateManager.emit(Optional.empty());
- return new AirbyteMessage().withType(Type.STATE).withState(stateMessage);
- }
-
- @Override
- public boolean compressSchemaHistoryForState() {
- return COMPRESSION_ENABLED;
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySqlCdcTargetPosition.java b/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySqlCdcTargetPosition.java
deleted file mode 100644
index 201a2417b3ef..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySqlCdcTargetPosition.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.source.mysql.cdc;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import io.airbyte.cdk.db.jdbc.JdbcDatabase;
-import io.airbyte.cdk.integrations.debezium.CdcTargetPosition;
-import io.airbyte.cdk.integrations.debezium.internals.ChangeEventWithMetadata;
-import io.airbyte.cdk.integrations.debezium.internals.SnapshotMetadata;
-import io.airbyte.commons.json.Jsons;
-import java.sql.SQLException;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Stream;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class MySqlCdcTargetPosition implements CdcTargetPosition {
-
- private static final Logger LOGGER = LoggerFactory.getLogger(MySqlCdcTargetPosition.class);
- private final MySqlCdcPosition targetPosition;
-
- public MySqlCdcTargetPosition(final String fileName, final Long position) {
- this(new MySqlCdcPosition(fileName, position));
- }
-
- @Override
- public boolean equals(final Object obj) {
- if (obj instanceof final MySqlCdcTargetPosition cdcTargetPosition) {
- return targetPosition.equals(cdcTargetPosition.targetPosition);
- }
- return false;
- }
-
- @Override
- public int hashCode() {
- return targetPosition.hashCode();
- }
-
- @Override
- public String toString() {
- return targetPosition.toString();
- }
-
- public MySqlCdcTargetPosition(final MySqlCdcPosition targetPosition) {
- this.targetPosition = targetPosition;
- }
-
- public static MySqlCdcTargetPosition targetPosition(final JdbcDatabase database) {
- try (final Stream stream = database.unsafeResultSetQuery(
- connection -> connection.createStatement().executeQuery("SHOW MASTER STATUS"),
- resultSet -> {
- final String file = resultSet.getString("File");
- final long position = resultSet.getLong("Position");
- if (file == null || position == 0) {
- return new MySqlCdcTargetPosition(null, null);
- }
- return new MySqlCdcTargetPosition(file, position);
- })) {
- final List masterStatus = stream.toList();
- final MySqlCdcTargetPosition targetPosition = masterStatus.get(0);
- LOGGER.info("Target File position : " + targetPosition);
- return targetPosition;
- } catch (final SQLException e) {
- throw new RuntimeException(e);
- }
-
- }
-
- @Override
- public boolean reachedTargetPosition(final ChangeEventWithMetadata changeEventWithMetadata) {
- if (changeEventWithMetadata.isSnapshotEvent()) {
- return false;
- } else if (SnapshotMetadata.LAST == changeEventWithMetadata.getSnapshotMetadata()) {
- LOGGER.info("Signalling close because Snapshot is complete");
- return true;
- } else {
- final String eventFileName = changeEventWithMetadata.getEventValueAsJson().get("source").get("file").asText();
- final long eventPosition = changeEventWithMetadata.getEventValueAsJson().get("source").get("pos").asLong();
- final boolean isEventPositionAfter =
- eventFileName.compareTo(targetPosition.fileName) > 0 || (eventFileName.compareTo(
- targetPosition.fileName) == 0 && eventPosition >= targetPosition.position);
- if (isEventPositionAfter) {
- LOGGER.info("Signalling close because record's binlog file : " + eventFileName + " , position : " + eventPosition
- + " is after target file : "
- + targetPosition.fileName + " , target position : " + targetPosition.position);
- }
- return isEventPositionAfter;
- }
-
- }
-
- @Override
- public boolean reachedTargetPosition(final MySqlCdcPosition positionFromHeartbeat) {
- return positionFromHeartbeat.fileName.compareTo(targetPosition.fileName) > 0 ||
- (positionFromHeartbeat.fileName.compareTo(targetPosition.fileName) == 0
- && positionFromHeartbeat.position >= targetPosition.position);
- }
-
- @Override
- public boolean isHeartbeatSupported() {
- return true;
- }
-
- @Override
- public boolean isEventAheadOffset(final Map offset, final ChangeEventWithMetadata event) {
- if (offset.size() != 1) {
- return false;
- }
-
- final String eventFileName = event.getEventValueAsJson().get("source").get("file").asText();
- final long eventPosition = event.getEventValueAsJson().get("source").get("pos").asLong();
-
- final JsonNode offsetJson = Jsons.deserialize((String) offset.values().toArray()[0]);
-
- final String offsetFileName = offsetJson.get("file").asText();
- final long offsetPosition = offsetJson.get("pos").asLong();
- if (eventFileName.compareTo(offsetFileName) != 0) {
- return eventFileName.compareTo(offsetFileName) > 0;
- }
-
- return eventPosition > offsetPosition;
- }
-
- @Override
- public boolean isSameOffset(final Map offsetA, final Map offsetB) {
- if ((offsetA == null || offsetA.size() != 1) || (offsetB == null || offsetB.size() != 1)) {
- return false;
- }
-
- final JsonNode offsetJsonA = Jsons.deserialize((String) offsetA.values().toArray()[0]);
- final String offsetAFileName = offsetJsonA.get("file").asText();
- final long offsetAPosition = offsetJsonA.get("pos").asLong();
-
- final JsonNode offsetJsonB = Jsons.deserialize((String) offsetB.values().toArray()[0]);
- final String offsetBFileName = offsetJsonB.get("file").asText();
- final long offsetBPosition = offsetJsonB.get("pos").asLong();
-
- return offsetAFileName.equals(offsetBFileName) && offsetAPosition == offsetBPosition;
- }
-
- @Override
- public MySqlCdcPosition extractPositionFromHeartbeatOffset(final Map sourceOffset) {
- return new MySqlCdcPosition(sourceOffset.get("file").toString(), (Long) sourceOffset.get("pos"));
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySqlDebeziumStateUtil.java b/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySqlDebeziumStateUtil.java
deleted file mode 100644
index 36d6115a3cec..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MySqlDebeziumStateUtil.java
+++ /dev/null
@@ -1,371 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.source.mysql.cdc;
-
-import static io.airbyte.integrations.source.mysql.cdc.MysqlCdcStateConstants.COMPRESSION_ENABLED;
-import static io.debezium.relational.RelationalDatabaseConnectorConfig.DATABASE_NAME;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.google.common.annotations.VisibleForTesting;
-import io.airbyte.cdk.db.jdbc.JdbcDatabase;
-import io.airbyte.cdk.db.jdbc.JdbcUtils;
-import io.airbyte.cdk.integrations.debezium.internals.AirbyteFileOffsetBackingStore;
-import io.airbyte.cdk.integrations.debezium.internals.AirbyteSchemaHistoryStorage;
-import io.airbyte.cdk.integrations.debezium.internals.AirbyteSchemaHistoryStorage.SchemaHistory;
-import io.airbyte.cdk.integrations.debezium.internals.DebeziumPropertiesManager;
-import io.airbyte.cdk.integrations.debezium.internals.DebeziumRecordPublisher;
-import io.airbyte.cdk.integrations.debezium.internals.DebeziumStateUtil;
-import io.airbyte.cdk.integrations.debezium.internals.RecordWaitTimeUtil;
-import io.airbyte.cdk.integrations.debezium.internals.RelationalDbDebeziumPropertiesManager;
-import io.airbyte.commons.json.Jsons;
-import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog;
-import io.debezium.config.Configuration;
-import io.debezium.connector.common.OffsetReader;
-import io.debezium.connector.mysql.MySqlConnectorConfig;
-import io.debezium.connector.mysql.MySqlOffsetContext;
-import io.debezium.connector.mysql.MySqlOffsetContext.Loader;
-import io.debezium.connector.mysql.MySqlPartition;
-import io.debezium.connector.mysql.gtid.MySqlGtidSet;
-import io.debezium.engine.ChangeEvent;
-import io.debezium.pipeline.spi.Offsets;
-import io.debezium.pipeline.spi.Partition;
-import java.sql.PreparedStatement;
-import java.sql.SQLException;
-import java.time.Duration;
-import java.time.Instant;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Stream;
-import org.apache.kafka.connect.storage.FileOffsetBackingStore;
-import org.apache.kafka.connect.storage.OffsetStorageReaderImpl;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class MySqlDebeziumStateUtil implements DebeziumStateUtil {
-
- private static final Logger LOGGER = LoggerFactory.getLogger(MySqlDebeziumStateUtil.class);
-
- public boolean savedOffsetStillPresentOnServer(final JdbcDatabase database, final MysqlDebeziumStateAttributes savedState) {
- if (savedState.gtidSet().isPresent()) {
- final Optional availableGtidStr = getStateAttributesFromDB(database).gtidSet();
- if (availableGtidStr.isEmpty()) {
- // Last offsets had GTIDs but the server does not use them
- LOGGER.info("Connector used GTIDs previously, but MySQL server does not know of any GTIDs or they are not enabled");
- return false;
- }
- final MySqlGtidSet gtidSetFromSavedState = new MySqlGtidSet(savedState.gtidSet().get());
- // Get the GTID set that is available in the server
- final MySqlGtidSet availableGtidSet = new MySqlGtidSet(availableGtidStr.get());
- if (gtidSetFromSavedState.isContainedWithin(availableGtidSet)) {
- LOGGER.info("MySQL server current GTID set {} does contain the GTID set required by the connector {}", availableGtidSet,
- gtidSetFromSavedState);
- final Optional gtidSetToReplicate = subtractGtidSet(availableGtidSet, gtidSetFromSavedState, database);
- if (gtidSetToReplicate.isPresent()) {
- final Optional purgedGtidSet = purgedGtidSet(database);
- if (purgedGtidSet.isPresent()) {
- LOGGER.info("MySQL server has already purged {} GTIDs", purgedGtidSet.get());
- final Optional nonPurgedGtidSetToReplicate = subtractGtidSet(gtidSetToReplicate.get(), purgedGtidSet.get(), database);
- if (nonPurgedGtidSetToReplicate.isPresent()) {
- LOGGER.info("GTIDs known by the MySQL server but not processed yet {}, for replication are available only {}", gtidSetToReplicate,
- nonPurgedGtidSetToReplicate);
- if (!gtidSetToReplicate.equals(nonPurgedGtidSetToReplicate)) {
- LOGGER.info("Some of the GTIDs needed to replicate have been already purged by MySQL server");
- return false;
- }
- }
- }
- }
- return true;
- }
- LOGGER.info("Connector last known GTIDs are {}, but MySQL server only has {}", gtidSetFromSavedState, availableGtidSet);
- return false;
- }
-
- final List existingLogFiles = getExistingLogFiles(database);
- final boolean found = existingLogFiles.stream().anyMatch(savedState.binlogFilename()::equals);
- if (!found) {
- LOGGER.info("Connector requires binlog file '{}', but MySQL server only has {}", savedState.binlogFilename(),
- String.join(", ", existingLogFiles));
- } else {
- LOGGER.info("MySQL server has the binlog file '{}' required by the connector", savedState.binlogFilename());
- }
-
- return found;
-
- }
-
- private List getExistingLogFiles(final JdbcDatabase database) {
- try (final Stream stream = database.unsafeResultSetQuery(
- connection -> connection.createStatement().executeQuery("SHOW BINARY LOGS"),
- resultSet -> resultSet.getString(1))) {
- return stream.toList();
- } catch (final SQLException e) {
- throw new RuntimeException(e);
- }
- }
-
- private Optional subtractGtidSet(final MySqlGtidSet set1, final MySqlGtidSet set2, final JdbcDatabase database) {
- try (final Stream stream = database.unsafeResultSetQuery(
- connection -> {
- final PreparedStatement ps = connection.prepareStatement("SELECT GTID_SUBTRACT(?, ?)");
- ps.setString(1, set1.toString());
- ps.setString(2, set2.toString());
- return ps.executeQuery();
- },
- resultSet -> new MySqlGtidSet(resultSet.getString(1)))) {
- final List gtidSets = stream.toList();
- if (gtidSets.isEmpty()) {
- return Optional.empty();
- } else if (gtidSets.size() == 1) {
- return Optional.of(gtidSets.get(0));
- } else {
- throw new RuntimeException("Not expecting gtid set size to be greater than 1");
- }
- } catch (final SQLException e) {
- throw new RuntimeException(e);
- }
- }
-
- private Optional purgedGtidSet(final JdbcDatabase database) {
- try (final Stream> stream = database.unsafeResultSetQuery(
- connection -> connection.createStatement().executeQuery("SELECT @@global.gtid_purged"),
- resultSet -> {
- if (resultSet.getMetaData().getColumnCount() > 0) {
- String string = resultSet.getString(1);
- if (string != null && !string.isEmpty()) {
- return Optional.of(new MySqlGtidSet(string));
- }
- }
- return Optional.empty();
- })) {
- final List> gtidSet = stream.toList();
- if (gtidSet.isEmpty()) {
- return Optional.empty();
- } else if (gtidSet.size() == 1) {
- return gtidSet.get(0);
- } else {
- throw new RuntimeException("Not expecting the size to be greater than 1");
- }
- } catch (final SQLException e) {
- throw new RuntimeException(e);
- }
- }
-
- public Optional savedOffset(final Properties baseProperties,
- final ConfiguredAirbyteCatalog catalog,
- final JsonNode cdcOffset,
- final JsonNode config) {
- if (Objects.isNull(cdcOffset)) {
- return Optional.empty();
- }
-
- final var offsetManager = AirbyteFileOffsetBackingStore.initializeState(cdcOffset, Optional.empty());
- final DebeziumPropertiesManager debeziumPropertiesManager = new RelationalDbDebeziumPropertiesManager(baseProperties, config, catalog,
- new ArrayList());
- final Properties debeziumProperties = debeziumPropertiesManager.getDebeziumProperties(offsetManager);
- return parseSavedOffset(debeziumProperties);
- }
-
- private Optional parseSavedOffset(final Properties properties) {
- FileOffsetBackingStore fileOffsetBackingStore = null;
- OffsetStorageReaderImpl offsetStorageReader = null;
-
- try {
- fileOffsetBackingStore = getFileOffsetBackingStore(properties);
- offsetStorageReader = getOffsetStorageReader(fileOffsetBackingStore, properties);
-
- final MySqlConnectorConfig connectorConfig = new MySqlConnectorConfig(Configuration.from(properties));
- final MySqlOffsetContext.Loader loader = new MySqlOffsetContext.Loader(connectorConfig);
- final Set partitions =
- Collections.singleton(new MySqlPartition(connectorConfig.getLogicalName(), properties.getProperty(DATABASE_NAME.name())));
-
- final OffsetReader offsetReader = new OffsetReader<>(offsetStorageReader,
- loader);
- final Map offsets = offsetReader.offsets(partitions);
-
- return extractStateAttributes(partitions, offsets);
- } finally {
- LOGGER.info("Closing offsetStorageReader and fileOffsetBackingStore");
- if (offsetStorageReader != null) {
- offsetStorageReader.close();
- }
-
- if (fileOffsetBackingStore != null) {
- fileOffsetBackingStore.stop();
- }
- }
- }
-
- private Optional extractStateAttributes(final Set partitions,
- final Map offsets) {
- boolean found = false;
- for (final Partition partition : partitions) {
- final MySqlOffsetContext mySqlOffsetContext = offsets.get(partition);
-
- if (mySqlOffsetContext != null) {
- found = true;
- LOGGER.info("Found previous partition offset {}: {}", partition, mySqlOffsetContext.getOffset());
- }
- }
-
- if (!found) {
- LOGGER.info("No previous offsets found");
- return Optional.empty();
- }
-
- final Offsets of = Offsets.of(offsets);
- final MySqlOffsetContext previousOffset = of.getTheOnlyOffset();
-
- return Optional.of(new MysqlDebeziumStateAttributes(previousOffset.getSource().binlogFilename(), previousOffset.getSource().binlogPosition(),
- Optional.ofNullable(previousOffset.gtidSet())));
-
- }
-
- public JsonNode constructInitialDebeziumState(final Properties properties,
- final ConfiguredAirbyteCatalog catalog,
- final JdbcDatabase database) {
- // https://debezium.io/documentation/reference/2.2/connectors/mysql.html#mysql-property-snapshot-mode
- // We use the recovery property cause using this mode will instruct Debezium to
- // construct the db schema history.
- // Note that we used to use schema_only_recovery mode, but this mode has been deprecated.
- properties.setProperty("snapshot.mode", "recovery");
- final String dbName = database.getSourceConfig().get(JdbcUtils.DATABASE_KEY).asText();
- // Topic.prefix is sanitized version of database name. At this stage properties does not have this
- // value - it's set in RelationalDbDebeziumPropertiesManager.
- final AirbyteFileOffsetBackingStore offsetManager = AirbyteFileOffsetBackingStore.initializeState(
- constructBinlogOffset(database, dbName, DebeziumPropertiesManager.sanitizeTopicPrefix(dbName)),
- Optional.empty());
- final AirbyteSchemaHistoryStorage schemaHistoryStorage =
- AirbyteSchemaHistoryStorage.initializeDBHistory(new SchemaHistory<>(Optional.empty(), false), COMPRESSION_ENABLED);
- final LinkedBlockingQueue> queue = new LinkedBlockingQueue<>();
- final var debeziumPropertiesManager =
- new RelationalDbDebeziumPropertiesManager(properties, database.getSourceConfig(), catalog, new ArrayList());
-
- try (final DebeziumRecordPublisher publisher = new DebeziumRecordPublisher(debeziumPropertiesManager)) {
- publisher.start(queue, offsetManager, Optional.of(schemaHistoryStorage));
- final Instant engineStartTime = Instant.now();
- while (!publisher.hasClosed()) {
- final ChangeEvent event = queue.poll(10, TimeUnit.SECONDS);
- if (event == null) {
- Duration initialWaitingDuration = Duration.ofMinutes(5L);
- // If initial waiting seconds is configured and it's greater than 5 minutes, use that value instead
- // of the default value
- final Duration configuredDuration = RecordWaitTimeUtil.getFirstRecordWaitTime(database.getSourceConfig());
- if (configuredDuration.compareTo(initialWaitingDuration) > 0) {
- initialWaitingDuration = configuredDuration;
- }
- if (Duration.between(engineStartTime, Instant.now()).compareTo(initialWaitingDuration) > 0) {
- LOGGER.error("No record is returned even after {} seconds of waiting, closing the engine", initialWaitingDuration.getSeconds());
- publisher.close();
- throw new RuntimeException(
- "Building schema history has timed out. Please consider increasing the debezium wait time in advanced options.");
- }
- continue;
- }
- LOGGER.info("A record is returned, closing the engine since the state is constructed");
- publisher.close();
- break;
- }
- } catch (final Exception e) {
- throw new RuntimeException(e);
- }
-
- final Map offset = offsetManager.read();
- final SchemaHistory schemaHistory = schemaHistoryStorage.read();
-
- assert !offset.isEmpty();
- assert Objects.nonNull(schemaHistory);
- assert Objects.nonNull(schemaHistory.getSchema());
-
- final JsonNode asJson = serialize(offset, schemaHistory);
- LOGGER.info("Initial Debezium state constructed: {}", asJson);
-
- if (asJson.get(MysqlCdcStateConstants.MYSQL_DB_HISTORY).asText().isBlank()) {
- throw new RuntimeException("Schema history snapshot returned empty history.");
- }
- return asJson;
- }
-
- public static JsonNode serialize(final Map offset, final SchemaHistory dbHistory) {
- final Map state = new HashMap<>();
- state.put(MysqlCdcStateConstants.MYSQL_CDC_OFFSET, offset);
- state.put(MysqlCdcStateConstants.MYSQL_DB_HISTORY, dbHistory.getSchema());
- state.put(MysqlCdcStateConstants.IS_COMPRESSED, dbHistory.isCompressed());
-
- return Jsons.jsonNode(state);
- }
-
- /**
- * Method to construct initial Debezium state which can be passed onto Debezium engine to make it
- * process binlogs from a specific file and position and skip snapshot phase
- */
- private JsonNode constructBinlogOffset(final JdbcDatabase database, final String debeziumName, final String topicPrefixName) {
- return format(getStateAttributesFromDB(database), debeziumName, topicPrefixName, Instant.now());
- }
-
- @VisibleForTesting
- public JsonNode format(final MysqlDebeziumStateAttributes attributes, final String debeziumName, final String topicPrefixName, final Instant time) {
- final String key = "[\"" + debeziumName + "\",{\"server\":\"" + topicPrefixName + "\"}]";
- final String gtidSet = attributes.gtidSet().isPresent() ? ",\"gtids\":\"" + attributes.gtidSet().get() + "\"" : "";
- final String value =
- "{\"transaction_id\":null,\"ts_sec\":" + time.getEpochSecond() + ",\"file\":\"" + attributes.binlogFilename() + "\",\"pos\":"
- + attributes.binlogPosition()
- + gtidSet + "}";
-
- final Map result = new HashMap<>();
- result.put(key, value);
-
- final JsonNode jsonNode = Jsons.jsonNode(result);
- LOGGER.info("Initial Debezium state offset constructed: {}", jsonNode);
-
- return jsonNode;
- }
-
- public static MysqlDebeziumStateAttributes getStateAttributesFromDB(final JdbcDatabase database) {
- try (final Stream stream = database.unsafeResultSetQuery(
- connection -> connection.createStatement().executeQuery("SHOW MASTER STATUS"),
- resultSet -> {
- final String file = resultSet.getString("File");
- final long position = resultSet.getLong("Position");
- assert file != null;
- assert position >= 0;
- if (resultSet.getMetaData().getColumnCount() > 4) {
- // This column exists only in MySQL 5.6.5 or later ...
- final String gtidSet = resultSet.getString(5); // GTID set, may be null, blank, or contain a GTID set
- return new MysqlDebeziumStateAttributes(file, position, removeNewLineChars(gtidSet));
- }
- return new MysqlDebeziumStateAttributes(file, position, Optional.empty());
- })) {
- final List stateAttributes = stream.toList();
- assert stateAttributes.size() == 1;
- return stateAttributes.get(0);
- } catch (final SQLException e) {
- throw new RuntimeException(e);
- }
- }
-
- private static Optional removeNewLineChars(final String gtidSet) {
- if (gtidSet != null && !gtidSet.trim().isEmpty()) {
- // Remove all the newline chars that exist in the GTID set string ...
- return Optional.of(gtidSet.replace("\n", "").replace("\r", ""));
- }
-
- return Optional.empty();
- }
-
- public record MysqlDebeziumStateAttributes(String binlogFilename, long binlogPosition, Optional gtidSet) {
-
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MysqlCdcStateConstants.java b/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MysqlCdcStateConstants.java
deleted file mode 100644
index cac1bfd997d5..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cdc/MysqlCdcStateConstants.java
+++ /dev/null
@@ -1,14 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.source.mysql.cdc;
-
-public class MysqlCdcStateConstants {
-
- public static final String MYSQL_CDC_OFFSET = "mysql_cdc_offset";
- public static final String MYSQL_DB_HISTORY = "mysql_db_history";
- public static final String IS_COMPRESSED = "is_compressed";
- public static final boolean COMPRESSION_ENABLED = true;
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cursor_based/MySqlCursorBasedStateManager.java b/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cursor_based/MySqlCursorBasedStateManager.java
deleted file mode 100644
index 9741c9b2b7ca..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/cursor_based/MySqlCursorBasedStateManager.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.source.mysql.cursor_based;
-
-import com.google.common.collect.Lists;
-import io.airbyte.cdk.integrations.source.relationaldb.CursorInfo;
-import io.airbyte.cdk.integrations.source.relationaldb.state.StreamStateManager;
-import io.airbyte.commons.json.Jsons;
-import io.airbyte.integrations.source.mysql.internal.models.CursorBasedStatus;
-import io.airbyte.integrations.source.mysql.internal.models.InternalModels.StateType;
-import io.airbyte.protocol.models.v0.AirbyteStateMessage;
-import io.airbyte.protocol.models.v0.AirbyteStateMessage.AirbyteStateType;
-import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair;
-import io.airbyte.protocol.models.v0.AirbyteStreamState;
-import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog;
-import io.airbyte.protocol.models.v0.StreamDescriptor;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class MySqlCursorBasedStateManager extends StreamStateManager {
-
- private static final Logger LOGGER = LoggerFactory.getLogger(MySqlCursorBasedStateManager.class);
-
- public MySqlCursorBasedStateManager(final List airbyteStateMessages, final ConfiguredAirbyteCatalog catalog) {
- super(airbyteStateMessages, catalog);
- }
-
- @Override
- public AirbyteStateMessage toState(final Optional pair) {
- if (pair.isPresent()) {
- final Map pairToCursorInfoMap = getPairToCursorInfoMap();
- final Optional cursorInfo = Optional.ofNullable(pairToCursorInfoMap.get(pair.get()));
-
- if (cursorInfo.isPresent()) {
- LOGGER.debug("Generating state message for {}...", pair);
- return new AirbyteStateMessage()
- .withType(AirbyteStateType.STREAM)
- // Temporarily include legacy state for backwards compatibility with the platform
- .withStream(generateStreamState(pair.get(), cursorInfo.get()));
- } else {
- LOGGER.warn("Cursor information could not be located in state for stream {}. Returning a new, empty state message...", pair);
- return new AirbyteStateMessage().withType(AirbyteStateType.STREAM).withStream(new AirbyteStreamState());
- }
- } else {
- LOGGER.warn("Stream not provided. Returning a new, empty state message...");
- return new AirbyteStateMessage().withType(AirbyteStateType.STREAM).withStream(new AirbyteStreamState());
- }
- }
-
- /**
- * Generates the stream state for the given stream and cursor information.
- *
- * @param airbyteStreamNameNamespacePair The stream.
- * @param cursorInfo The current cursor.
- * @return The {@link AirbyteStreamState} representing the current state of the stream.
- */
- private AirbyteStreamState generateStreamState(final AirbyteStreamNameNamespacePair airbyteStreamNameNamespacePair,
- final CursorInfo cursorInfo) {
- return new AirbyteStreamState()
- .withStreamDescriptor(
- new StreamDescriptor().withName(airbyteStreamNameNamespacePair.getName()).withNamespace(airbyteStreamNameNamespacePair.getNamespace()))
- .withStreamState(Jsons.jsonNode(generateDbStreamState(airbyteStreamNameNamespacePair, cursorInfo)));
- }
-
- private CursorBasedStatus generateDbStreamState(final AirbyteStreamNameNamespacePair airbyteStreamNameNamespacePair,
- final CursorInfo cursorInfo) {
- final CursorBasedStatus state = new CursorBasedStatus();
- state.setStateType(StateType.CURSOR_BASED);
- state.setVersion(2L);
- state.setStreamName(airbyteStreamNameNamespacePair.getName());
- state.setStreamNamespace(airbyteStreamNameNamespacePair.getNamespace());
- state.setCursorField(cursorInfo.getCursorField() == null ? Collections.emptyList() : Lists.newArrayList(cursorInfo.getCursorField()));
- state.setCursor(cursorInfo.getCursor());
- if (cursorInfo.getCursorRecordCount() > 0L) {
- state.setCursorRecordCount(cursorInfo.getCursorRecordCount());
- }
- return state;
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/initialsync/CdcMetadataInjector.java b/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/initialsync/CdcMetadataInjector.java
deleted file mode 100644
index cde1f645a60f..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/initialsync/CdcMetadataInjector.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.source.mysql.initialsync;
-
-import com.fasterxml.jackson.databind.node.ObjectNode;
-import io.airbyte.integrations.source.mysql.cdc.MySqlCdcConnectorMetadataInjector;
-import io.airbyte.integrations.source.mysql.cdc.MySqlDebeziumStateUtil.MysqlDebeziumStateAttributes;
-
-public class CdcMetadataInjector {
-
- private final String transactionTimestamp;
- private final MysqlDebeziumStateAttributes stateAttributes;
- private final MySqlCdcConnectorMetadataInjector metadataInjector;
-
- public CdcMetadataInjector(final String transactionTimestamp,
- final MysqlDebeziumStateAttributes stateAttributes,
- final MySqlCdcConnectorMetadataInjector metadataInjector) {
- this.transactionTimestamp = transactionTimestamp;
- this.stateAttributes = stateAttributes;
- this.metadataInjector = metadataInjector;
- }
-
- public void inject(final ObjectNode record) {
- metadataInjector.addMetaDataToRowsFetchedOutsideDebezium(record, transactionTimestamp, stateAttributes);
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/initialsync/MySqlInitialLoadGlobalStateManager.java b/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/initialsync/MySqlInitialLoadGlobalStateManager.java
deleted file mode 100644
index 9b3de8d4047e..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/initialsync/MySqlInitialLoadGlobalStateManager.java
+++ /dev/null
@@ -1,203 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.source.mysql.initialsync;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import io.airbyte.cdk.integrations.source.relationaldb.models.CdcState;
-import io.airbyte.cdk.integrations.source.relationaldb.models.DbStreamState;
-import io.airbyte.cdk.integrations.source.relationaldb.state.StateManager;
-import io.airbyte.commons.json.Jsons;
-import io.airbyte.integrations.source.mysql.initialsync.MySqlInitialReadUtil.InitialLoadStreams;
-import io.airbyte.integrations.source.mysql.initialsync.MySqlInitialReadUtil.PrimaryKeyInfo;
-import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair;
-import io.airbyte.protocol.models.v0.AirbyteGlobalState;
-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.ConfiguredAirbyteCatalog;
-import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream;
-import io.airbyte.protocol.models.v0.StreamDescriptor;
-import io.airbyte.protocol.models.v0.SyncMode;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class MySqlInitialLoadGlobalStateManager extends MySqlInitialLoadStateManager {
-
- private static final Logger LOGGER = LoggerFactory.getLogger(MySqlInitialLoadGlobalStateManager.class);
- protected StateManager stateManager;
-
- // Only one global state is emitted, which is fanned out into many entries in the DB by platform. As
- // a result, we need to keep track of streams that
- // have completed the snapshot.
- private Set streamsThatHaveCompletedSnapshot;
-
- // No special handling for resumable full refresh streams. We will report the cursor as it is.
- private Set resumableFullRefreshStreams;
-
- // non ResumableFullRefreshStreams do not have any state. We only report count for them.
- private Set nonResumableFullRefreshStreams;
- private Set completedNonResumableFullRefreshStreams;
-
- private final boolean savedOffsetStillPresentOnServer;
- private final ConfiguredAirbyteCatalog catalog;
- private final CdcState defaultCdcState;
-
- public MySqlInitialLoadGlobalStateManager(final InitialLoadStreams initialLoadStreams,
- final Map pairToPrimaryKeyInfo,
- final StateManager stateManager,
- final ConfiguredAirbyteCatalog catalog,
-
- final boolean savedOffsetStillPresentOnServer,
- final CdcState defaultCdcState) {
- this.stateManager = stateManager;
- this.pairToPrimaryKeyLoadStatus = MySqlInitialLoadStateManager.initPairToPrimaryKeyLoadStatusMap(initialLoadStreams.pairToInitialLoadStatus());
- this.pairToPrimaryKeyInfo = pairToPrimaryKeyInfo;
- this.catalog = catalog;
- this.savedOffsetStillPresentOnServer = savedOffsetStillPresentOnServer;
- this.defaultCdcState = defaultCdcState;
- this.streamStateForIncrementalRunSupplier = pair -> Jsons.emptyObject();
- initStreams(initialLoadStreams, catalog);
- }
-
- private void initStreams(final InitialLoadStreams initialLoadStreams,
- final ConfiguredAirbyteCatalog catalog) {
- this.streamsThatHaveCompletedSnapshot = new HashSet<>();
- this.resumableFullRefreshStreams = new HashSet<>();
- this.nonResumableFullRefreshStreams = new HashSet<>();
- this.completedNonResumableFullRefreshStreams = new HashSet<>();
-
- catalog.getStreams().forEach(configuredAirbyteStream -> {
- var pairInStream =
- new AirbyteStreamNameNamespacePair(configuredAirbyteStream.getStream().getName(), configuredAirbyteStream.getStream().getNamespace());
- if (!initialLoadStreams.streamsForInitialLoad().contains(configuredAirbyteStream)
- && configuredAirbyteStream.getSyncMode() == SyncMode.INCREMENTAL) {
- this.streamsThatHaveCompletedSnapshot.add(pairInStream);
- }
- if (configuredAirbyteStream.getSyncMode() == SyncMode.FULL_REFRESH) {
- if (configuredAirbyteStream.getStream().getSourceDefinedPrimaryKey() != null
- && !configuredAirbyteStream.getStream().getSourceDefinedPrimaryKey().isEmpty()) {
- this.resumableFullRefreshStreams.add(pairInStream);
- } else {
- this.nonResumableFullRefreshStreams.add(pairInStream);
- }
- }
- });
- }
-
- private AirbyteGlobalState generateGlobalState(final List streamStates) {
- CdcState cdcState = stateManager.getCdcStateManager().getCdcState();
-
- if (!savedOffsetStillPresentOnServer || cdcState == null
- || cdcState.getState() == null) {
- cdcState = defaultCdcState;
- }
-
- final AirbyteGlobalState globalState = new AirbyteGlobalState();
- globalState.setSharedState(Jsons.jsonNode(cdcState));
- globalState.setStreamStates(streamStates);
- return globalState;
- }
-
- @Override
- public AirbyteStateMessage generateStateMessageAtCheckpoint(final ConfiguredAirbyteStream airbyteStream) {
- final List streamStates = new ArrayList<>();
- streamsThatHaveCompletedSnapshot.forEach(stream -> {
- final DbStreamState state = getFinalState(stream);
- streamStates.add(getAirbyteStreamState(stream, Jsons.jsonNode(state)));
- });
-
- resumableFullRefreshStreams.forEach(stream -> {
- var pkStatus = getPrimaryKeyLoadStatus(stream);
- if (pkStatus != null) {
- streamStates.add(getAirbyteStreamState(stream, (Jsons.jsonNode(pkStatus))));
- }
- });
-
- completedNonResumableFullRefreshStreams.forEach(stream -> {
- streamStates.add(new AirbyteStreamState()
- .withStreamDescriptor(
- new StreamDescriptor().withName(stream.getName()).withNamespace(stream.getNamespace())));
- });
-
- if (airbyteStream.getSyncMode() == SyncMode.INCREMENTAL) {
- AirbyteStreamNameNamespacePair pair =
- new AirbyteStreamNameNamespacePair(airbyteStream.getStream().getName(), airbyteStream.getStream().getNamespace());
- var pkStatus = getPrimaryKeyLoadStatus(pair);
- streamStates.add(getAirbyteStreamState(pair, (Jsons.jsonNode(pkStatus))));
- }
-
- return new AirbyteStateMessage()
- .withType(AirbyteStateType.GLOBAL)
- .withGlobal(generateGlobalState(streamStates));
- }
-
- @Override
- public AirbyteStateMessage createFinalStateMessage(final ConfiguredAirbyteStream airbyteStream) {
- final AirbyteStreamNameNamespacePair pair =
- new AirbyteStreamNameNamespacePair(airbyteStream.getStream().getName(), airbyteStream.getStream().getNamespace());
- if (airbyteStream.getSyncMode() == SyncMode.INCREMENTAL) {
- streamsThatHaveCompletedSnapshot.add(pair);
- } else if (nonResumableFullRefreshStreams.contains(pair)) {
- completedNonResumableFullRefreshStreams.add(pair);
- }
- final List streamStates = new ArrayList<>();
-
- streamsThatHaveCompletedSnapshot.forEach(stream -> {
- final DbStreamState state = getFinalState(stream);
- streamStates.add(getAirbyteStreamState(stream, Jsons.jsonNode(state)));
- });
-
- resumableFullRefreshStreams.forEach(stream -> {
- var pkStatus = getPrimaryKeyLoadStatus(stream);
- streamStates.add(getAirbyteStreamState(stream, (Jsons.jsonNode(pkStatus))));
- });
-
- completedNonResumableFullRefreshStreams.forEach(stream -> {
- streamStates.add(new AirbyteStreamState()
- .withStreamDescriptor(
- new StreamDescriptor().withName(stream.getName()).withNamespace(stream.getNamespace())));
- });
-
- return new AirbyteStateMessage()
- .withType(AirbyteStateType.GLOBAL)
- .withGlobal(generateGlobalState(streamStates));
- }
-
- @Override
- public PrimaryKeyInfo getPrimaryKeyInfo(final AirbyteStreamNameNamespacePair pair) {
- return pairToPrimaryKeyInfo.get(pair);
- }
-
- private AirbyteStreamState getAirbyteStreamState(final io.airbyte.protocol.models.AirbyteStreamNameNamespacePair pair, final JsonNode stateData) {
- assert Objects.nonNull(pair);
- assert Objects.nonNull(pair.getName());
- assert Objects.nonNull(pair.getNamespace());
-
- return new AirbyteStreamState()
- .withStreamDescriptor(
- new StreamDescriptor().withName(pair.getName()).withNamespace(pair.getNamespace()))
- .withStreamState(stateData);
- }
-
- private DbStreamState getFinalState(final io.airbyte.protocol.models.AirbyteStreamNameNamespacePair pair) {
- assert Objects.nonNull(pair);
- assert Objects.nonNull(pair.getName());
- assert Objects.nonNull(pair.getNamespace());
-
- return new DbStreamState()
- .withStreamName(pair.getName())
- .withStreamNamespace(pair.getNamespace())
- .withCursorField(Collections.emptyList())
- .withCursor(null);
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/initialsync/MySqlInitialLoadHandler.java b/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/initialsync/MySqlInitialLoadHandler.java
deleted file mode 100644
index f9684c33ea1f..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/initialsync/MySqlInitialLoadHandler.java
+++ /dev/null
@@ -1,223 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.source.mysql.initialsync;
-
-import static io.airbyte.cdk.integrations.debezium.DebeziumIteratorConstants.SYNC_CHECKPOINT_DURATION_PROPERTY;
-import static io.airbyte.cdk.integrations.debezium.DebeziumIteratorConstants.SYNC_CHECKPOINT_RECORDS_PROPERTY;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.google.common.annotations.VisibleForTesting;
-import com.mysql.cj.MysqlType;
-import io.airbyte.cdk.db.jdbc.AirbyteRecordData;
-import io.airbyte.cdk.db.jdbc.JdbcDatabase;
-import io.airbyte.cdk.db.jdbc.JdbcUtils;
-import io.airbyte.cdk.integrations.debezium.DebeziumIteratorConstants;
-import io.airbyte.cdk.integrations.source.relationaldb.DbSourceDiscoverUtil;
-import io.airbyte.cdk.integrations.source.relationaldb.InitialLoadHandler;
-import io.airbyte.cdk.integrations.source.relationaldb.TableInfo;
-import io.airbyte.cdk.integrations.source.relationaldb.state.SourceStateIterator;
-import io.airbyte.cdk.integrations.source.relationaldb.state.StateEmitFrequency;
-import io.airbyte.cdk.integrations.source.relationaldb.streamstatus.StreamStatusTraceEmitterIterator;
-import io.airbyte.commons.stream.AirbyteStreamStatusHolder;
-import io.airbyte.commons.stream.AirbyteStreamUtils;
-import io.airbyte.commons.util.AutoCloseableIterator;
-import io.airbyte.commons.util.AutoCloseableIterators;
-import io.airbyte.integrations.source.mysql.MySqlQueryUtils.TableSizeInfo;
-import io.airbyte.integrations.source.mysql.MySqlSourceOperations;
-import io.airbyte.integrations.source.mysql.internal.models.PrimaryKeyLoadStatus;
-import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair;
-import io.airbyte.protocol.models.CommonField;
-import io.airbyte.protocol.models.v0.*;
-import io.airbyte.protocol.models.v0.AirbyteMessage.Type;
-import java.time.Duration;
-import java.time.Instant;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-import org.jetbrains.annotations.NotNull;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class MySqlInitialLoadHandler implements InitialLoadHandler {
-
- private static final Logger LOGGER = LoggerFactory.getLogger(MySqlInitialLoadHandler.class);
-
- private static final long RECORD_LOGGING_SAMPLE_RATE = 1_000_000;
- private final JsonNode config;
- private final JdbcDatabase database;
- private final MySqlSourceOperations sourceOperations;
- private final String quoteString;
- private final MySqlInitialLoadStateManager initialLoadStateManager;
- private final Optional> streamStateForIncrementalRunSupplier;
-
- private static final long QUERY_TARGET_SIZE_GB = 1_073_741_824;
- private static final long DEFAULT_CHUNK_SIZE = 1_000_000;
- private long MAX_CHUNK_SIZE = Long.MAX_VALUE;
- final Map tableSizeInfoMap;
-
- public MySqlInitialLoadHandler(final JsonNode config,
- final JdbcDatabase database,
- final MySqlSourceOperations sourceOperations,
- final String quoteString,
- final MySqlInitialLoadStateManager initialLoadStateManager,
- final Optional> streamStateForIncrementalRunSupplier,
- final Map tableSizeInfoMap) {
- this.config = config;
- this.database = database;
- this.sourceOperations = sourceOperations;
- this.quoteString = quoteString;
- this.initialLoadStateManager = initialLoadStateManager;
- this.streamStateForIncrementalRunSupplier = streamStateForIncrementalRunSupplier;
- this.tableSizeInfoMap = tableSizeInfoMap;
- adjustChunkSizeLimitForMySQLVariants();
- }
-
- private void adjustChunkSizeLimitForMySQLVariants() {
- // For PSDB, we need to limit the chunk size to 100k rows to avoid the query being killed by the
- // server.
- // Reference:
- // https://planetscale.com/docs/reference/planetscale-system-limits
- if (config.get(JdbcUtils.HOST_KEY).asText().toLowerCase().contains("psdb.cloud"))
- MAX_CHUNK_SIZE = 100_000;
- }
-
- public List> getIncrementalIterators(
- final ConfiguredAirbyteCatalog catalog,
- final Map>> tableNameToTable,
- final Instant emittedAt,
- final boolean decorateWithStartedStatus,
- final boolean decorateWithCompletedStatus,
- final Optional cdcInitialLoadTimeout) {
- final List> iteratorList = new ArrayList<>();
- for (final ConfiguredAirbyteStream airbyteStream : catalog.getStreams()) {
- final AirbyteStream stream = airbyteStream.getStream();
- final String streamName = stream.getName();
- final String namespace = stream.getNamespace();
- final AirbyteStreamNameNamespacePair pair = new AirbyteStreamNameNamespacePair(streamName, namespace);
- if (airbyteStream.getSyncMode().equals(SyncMode.INCREMENTAL)) {
- final String fullyQualifiedTableName = DbSourceDiscoverUtil.getFullyQualifiedTableName(namespace, streamName);
- final TableInfo> table = tableNameToTable.get(fullyQualifiedTableName);
- if (decorateWithStartedStatus) {
- iteratorList.add(
- new StreamStatusTraceEmitterIterator(new AirbyteStreamStatusHolder(pair, AirbyteStreamStatusTraceMessage.AirbyteStreamStatus.STARTED)));
- }
-
- iteratorList.add(getIteratorForStream(airbyteStream, table, emittedAt, cdcInitialLoadTimeout));
- if (decorateWithCompletedStatus) {
- iteratorList.add(new StreamStatusTraceEmitterIterator(
- new AirbyteStreamStatusHolder(pair, AirbyteStreamStatusTraceMessage.AirbyteStreamStatus.COMPLETE)));
- }
- }
- }
- return iteratorList;
- }
-
- @Override
- public AutoCloseableIterator getIteratorForStream(
- @NotNull ConfiguredAirbyteStream airbyteStream,
- @NotNull TableInfo> table,
- @NotNull Instant emittedAt,
- @NotNull final Optional cdcInitialLoadTimeout) {
-
- final AirbyteStream stream = airbyteStream.getStream();
- final String streamName = stream.getName();
- final String namespace = stream.getNamespace();
- final AirbyteStreamNameNamespacePair pair = new AirbyteStreamNameNamespacePair(streamName, namespace);
- final List selectedDatabaseFields = table.getFields()
- .stream()
- .map(CommonField::getName)
- .filter(CatalogHelpers.getTopLevelFieldNames(airbyteStream)::contains)
- .collect(Collectors.toList());
- final AutoCloseableIterator queryStream =
- new MySqlInitialLoadRecordIterator(database, sourceOperations, quoteString, initialLoadStateManager, selectedDatabaseFields, pair,
- Long.min(calculateChunkSize(tableSizeInfoMap.get(pair), pair), MAX_CHUNK_SIZE), isCompositePrimaryKey(airbyteStream), emittedAt,
- cdcInitialLoadTimeout);
- final AutoCloseableIterator recordIterator =
- getRecordIterator(queryStream, streamName, namespace, emittedAt.toEpochMilli());
- final AutoCloseableIterator recordAndMessageIterator = augmentWithState(recordIterator, airbyteStream, pair);
-
- return augmentWithLogs(recordAndMessageIterator, pair, streamName);
- }
-
- private static boolean isCompositePrimaryKey(final ConfiguredAirbyteStream stream) {
- return stream.getStream().getSourceDefinedPrimaryKey().size() > 1;
- }
-
- // Calculates the number of rows to fetch per query.
- @VisibleForTesting
- public static long calculateChunkSize(final TableSizeInfo tableSizeInfo, final AirbyteStreamNameNamespacePair pair) {
- // If table size info could not be calculated, a default chunk size will be provided.
- if (tableSizeInfo == null || tableSizeInfo.tableSize() == 0 || tableSizeInfo.avgRowLength() == 0) {
- LOGGER.info("Chunk size could not be determined for pair: {}, defaulting to {} rows", pair, DEFAULT_CHUNK_SIZE);
- return DEFAULT_CHUNK_SIZE;
- }
- final long avgRowLength = tableSizeInfo.avgRowLength();
- final long chunkSize = QUERY_TARGET_SIZE_GB / avgRowLength;
- LOGGER.info("Chunk size determined for pair: {}, is {}", pair, chunkSize);
- return chunkSize;
- }
-
- // Transforms the given iterator to create an {@link AirbyteRecordMessage}
- private AutoCloseableIterator getRecordIterator(
- final AutoCloseableIterator recordIterator,
- final String streamName,
- final String namespace,
- final long emittedAt) {
- return AutoCloseableIterators.transform(recordIterator, r -> new AirbyteMessage()
- .withType(Type.RECORD)
- .withRecord(new AirbyteRecordMessage()
- .withStream(streamName)
- .withNamespace(namespace)
- .withEmittedAt(emittedAt)
- .withData(r.rawRowData())
- .withMeta(isMetaChangesEmptyOrNull(r.meta()) ? null : r.meta())));
- }
-
- private boolean isMetaChangesEmptyOrNull(AirbyteRecordMessageMeta meta) {
- return meta == null || meta.getChanges() == null || meta.getChanges().isEmpty();
- }
-
- // Augments the given iterator with record count logs.
- private AutoCloseableIterator augmentWithLogs(final AutoCloseableIterator iterator,
- final io.airbyte.protocol.models.AirbyteStreamNameNamespacePair pair,
- final String streamName) {
- final AtomicLong recordCount = new AtomicLong();
- return AutoCloseableIterators.transform(iterator,
- AirbyteStreamUtils.convertFromNameAndNamespace(pair.getName(), pair.getNamespace()),
- r -> {
- final long count = recordCount.incrementAndGet();
- if (count % RECORD_LOGGING_SAMPLE_RATE == 0) {
- LOGGER.info("Reading stream {}. Records read: {}", streamName, count);
- }
- return r;
- });
- }
-
- private AutoCloseableIterator augmentWithState(final AutoCloseableIterator recordIterator,
- final ConfiguredAirbyteStream airbyteStream,
- final AirbyteStreamNameNamespacePair pair) {
-
- final PrimaryKeyLoadStatus currentPkLoadStatus = initialLoadStateManager.getPrimaryKeyLoadStatus(pair);
-
- final Duration syncCheckpointDuration =
- config.get(SYNC_CHECKPOINT_DURATION_PROPERTY) != null ? Duration.ofSeconds(config.get(SYNC_CHECKPOINT_DURATION_PROPERTY).asLong())
- : DebeziumIteratorConstants.SYNC_CHECKPOINT_DURATION;
- final Long syncCheckpointRecords = config.get(SYNC_CHECKPOINT_RECORDS_PROPERTY) != null ? config.get(SYNC_CHECKPOINT_RECORDS_PROPERTY).asLong()
- : DebeziumIteratorConstants.SYNC_CHECKPOINT_RECORDS;
-
- if (streamStateForIncrementalRunSupplier.isPresent()) {
- initialLoadStateManager.setStreamStateForIncrementalRunSupplier(streamStateForIncrementalRunSupplier.get());
- }
- return AutoCloseableIterators.transformIterator(
- r -> new SourceStateIterator<>(r, airbyteStream, initialLoadStateManager,
- new StateEmitFrequency(syncCheckpointRecords, syncCheckpointDuration)),
- recordIterator, pair);
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/initialsync/MySqlInitialLoadRecordIterator.java b/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/initialsync/MySqlInitialLoadRecordIterator.java
deleted file mode 100644
index 1093560ad43d..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/initialsync/MySqlInitialLoadRecordIterator.java
+++ /dev/null
@@ -1,231 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.source.mysql.initialsync;
-
-import static io.airbyte.cdk.db.DbAnalyticsUtils.cdcSnapshotForceShutdownMessage;
-
-import com.google.common.collect.AbstractIterator;
-import com.mysql.cj.MysqlType;
-import io.airbyte.cdk.db.JdbcCompatibleSourceOperations;
-import io.airbyte.cdk.db.jdbc.AirbyteRecordData;
-import io.airbyte.cdk.db.jdbc.JdbcDatabase;
-import io.airbyte.cdk.integrations.base.AirbyteTraceMessageUtility;
-import io.airbyte.cdk.integrations.source.relationaldb.RelationalDbQueryUtils;
-import io.airbyte.commons.exceptions.TransientErrorException;
-import io.airbyte.commons.util.AutoCloseableIterator;
-import io.airbyte.commons.util.AutoCloseableIterators;
-import io.airbyte.integrations.source.mysql.initialsync.MySqlInitialReadUtil.PrimaryKeyInfo;
-import io.airbyte.integrations.source.mysql.internal.models.PrimaryKeyLoadStatus;
-import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair;
-import java.sql.Connection;
-import java.sql.PreparedStatement;
-import java.sql.SQLException;
-import java.time.Duration;
-import java.time.Instant;
-import java.util.List;
-import java.util.Optional;
-import java.util.stream.Stream;
-import javax.annotation.CheckForNull;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This record iterator operates over a single stream. It continuously reads data from a table via
- * multiple queries with the configured chunk size until the entire table is processed. The next
- * query uses the highest watermark of the primary key seen in the previous subquery. Consider a
- * table with chunk size = 1,000,000, and 3,500,000 records. The series of queries executed are :
- * Query 1 : select * from table order by pk limit 1,800,000, pk_max = pk_max_1 Query 2 : select *
- * from table where pk > pk_max_1 order by pk limit 1,800,000, pk_max = pk_max_2 Query 3 : select *
- * from table where pk > pk_max_2 order by pk limit 1,800,000, pk_max = pk_max_3 Query 4 : select *
- * from table where pk > pk_max_3 order by pk limit 1,800,000, pk_max = pk_max_4 Query 5 : select *
- * from table where pk > pk_max_4 order by pk limit 1,800,000. Final query, since there are zero
- * records processed here.
- */
-@SuppressWarnings("try")
-public class MySqlInitialLoadRecordIterator extends AbstractIterator
- implements AutoCloseableIterator {
-
- private static final Logger LOGGER = LoggerFactory.getLogger(MySqlInitialLoadRecordIterator.class);
-
- private final JdbcCompatibleSourceOperations sourceOperations;
-
- private final String quoteString;
- private final MySqlInitialLoadStateManager initialLoadStateManager;
- private final List columnNames;
- private final AirbyteStreamNameNamespacePair pair;
- private final JdbcDatabase database;
- // Represents the number of rows to get with each query.
- private final long chunkSize;
- private final PrimaryKeyInfo pkInfo;
- private final boolean isCompositeKeyLoad;
-
- private final Instant startInstant;
- private int numSubqueries = 0;
- private AutoCloseableIterator currentIterator;
-
- private Optional cdcInitialLoadTimeout;
- private boolean isCdcSync;
-
- MySqlInitialLoadRecordIterator(
- final JdbcDatabase database,
- final JdbcCompatibleSourceOperations sourceOperations,
- final String quoteString,
- final MySqlInitialLoadStateManager initialLoadStateManager,
- final List columnNames,
- final AirbyteStreamNameNamespacePair pair,
- final long chunkSize,
- final boolean isCompositeKeyLoad,
- final Instant startInstant,
- final Optional cdcInitialLoadTimeout) {
- this.database = database;
- this.sourceOperations = sourceOperations;
- this.quoteString = quoteString;
- this.initialLoadStateManager = initialLoadStateManager;
- this.columnNames = columnNames;
- this.pair = pair;
- this.chunkSize = chunkSize;
- this.pkInfo = initialLoadStateManager.getPrimaryKeyInfo(pair);
- this.isCompositeKeyLoad = isCompositeKeyLoad;
- this.startInstant = startInstant;
- this.cdcInitialLoadTimeout = cdcInitialLoadTimeout;
- this.isCdcSync = isCdcSync(initialLoadStateManager);
- }
-
- @CheckForNull
- @Override
- protected AirbyteRecordData computeNext() {
- if (isCdcSync && cdcInitialLoadTimeout.isPresent()
- && Duration.between(startInstant, Instant.now()).compareTo(cdcInitialLoadTimeout.get()) > 0) {
- final String cdcInitialLoadTimeoutMessage = String.format(
- "Initial load for table %s has taken longer than %s hours, Canceling sync so that CDC replication can catch-up on subsequent attempt, and then initial snapshotting will resume",
- getAirbyteStream().get(), cdcInitialLoadTimeout.get().toHours());
- LOGGER.info(cdcInitialLoadTimeoutMessage);
- AirbyteTraceMessageUtility.emitAnalyticsTrace(cdcSnapshotForceShutdownMessage());
- throw new TransientErrorException(cdcInitialLoadTimeoutMessage);
- }
- if (shouldBuildNextSubquery()) {
- try {
- // We will only issue one query for a composite key load. If we have already processed all the data
- // associated with this
- // query, we should indicate that we are done processing for the given stream.
- if (isCompositeKeyLoad && numSubqueries >= 1) {
- return endOfData();
- }
- // Previous stream (and connection) must be manually closed in this iterator.
- if (currentIterator != null) {
- currentIterator.close();
- }
-
- LOGGER.info("Subquery number : {}", numSubqueries);
- final Stream stream = database.unsafeQuery(
- this::getPkPreparedStatement, sourceOperations::convertDatabaseRowToAirbyteRecordData);
-
- currentIterator = AutoCloseableIterators.fromStream(stream, pair);
- numSubqueries++;
- // If the current subquery has no records associated with it, the entire stream has been read.
- if (!currentIterator.hasNext()) {
- return endOfData();
- }
- } catch (final Exception e) {
- throw new RuntimeException(e);
- }
- }
- return currentIterator.next();
- }
-
- private boolean shouldBuildNextSubquery() {
- // The next sub-query should be built if (i) it is the first subquery in the sequence. (ii) the
- // previous subquery has finished.
- return (currentIterator == null || !currentIterator.hasNext());
- }
-
- private PreparedStatement getPkPreparedStatement(final Connection connection) {
- try {
- final String tableName = pair.getName();
- final String schemaName = pair.getNamespace();
- LOGGER.info("Preparing query for table: {}", tableName);
- final String fullTableName = RelationalDbQueryUtils.getFullyQualifiedTableNameWithQuoting(schemaName, tableName,
- quoteString);
-
- final String wrappedColumnNames = RelationalDbQueryUtils.enquoteIdentifierList(columnNames, quoteString);
-
- final PrimaryKeyLoadStatus pkLoadStatus = initialLoadStateManager.getPrimaryKeyLoadStatus(pair);
-
- if (pkLoadStatus == null) {
- LOGGER.info("pkLoadStatus is null");
- final String quotedCursorField = RelationalDbQueryUtils.enquoteIdentifier(pkInfo.pkFieldName(), quoteString);
- final String sql;
- // We cannot load in chunks for a composite key load, since each field might not have distinct
- // values.
- if (isCompositeKeyLoad) {
- sql = String.format("SELECT %s FROM %s ORDER BY %s", wrappedColumnNames, fullTableName,
- quotedCursorField);
- } else {
- sql = String.format("SELECT %s FROM %s ORDER BY %s LIMIT %s", wrappedColumnNames, fullTableName,
- quotedCursorField, chunkSize);
- }
- final PreparedStatement preparedStatement = connection.prepareStatement(sql);
- LOGGER.info("Executing query for table {}: {}", tableName, preparedStatement);
- return preparedStatement;
- } else {
- LOGGER.info("pkLoadStatus value is : {}", pkLoadStatus.getPkVal());
- final String quotedCursorField = RelationalDbQueryUtils.enquoteIdentifier(pkLoadStatus.getPkName(), quoteString);
- final String sql;
- // We cannot load in chunks for a composite key load, since each field might not have distinct
- // values. Furthermore, we have to issue a >=
- // query since we may not have processed all of the data associated with the last saved primary key
- // value.
- if (isCompositeKeyLoad) {
- sql = String.format("SELECT %s FROM %s WHERE %s >= ? ORDER BY %s", wrappedColumnNames, fullTableName,
- quotedCursorField, quotedCursorField);
- } else {
- // The pk max value could be null - this can happen in the case of empty tables. In this case, we
- // can just issue a query
- // without any chunking.
- if (pkInfo.pkMaxValue() != null) {
- sql = String.format("SELECT %s FROM %s WHERE %s > ? AND %s <= ? ORDER BY %s LIMIT %s", wrappedColumnNames, fullTableName,
- quotedCursorField, quotedCursorField, quotedCursorField, chunkSize);
- } else {
- sql = String.format("SELECT %s FROM %s WHERE %s > ? ORDER BY %s", wrappedColumnNames, fullTableName,
- quotedCursorField, quotedCursorField);
- }
- }
- final PreparedStatement preparedStatement = connection.prepareStatement(sql);
- final MysqlType cursorFieldType = pkInfo.fieldType();
- sourceOperations.setCursorField(preparedStatement, 1, cursorFieldType, pkLoadStatus.getPkVal());
- if (!isCompositeKeyLoad && pkInfo.pkMaxValue() != null) {
- sourceOperations.setCursorField(preparedStatement, 2, cursorFieldType, pkInfo.pkMaxValue());
- }
- LOGGER.info("Executing query for table {}: {}", tableName, preparedStatement);
- return preparedStatement;
- }
- } catch (final SQLException e) {
- throw new RuntimeException(e);
- }
- }
-
- @Override
- public void close() throws Exception {
- if (currentIterator != null) {
- currentIterator.close();
- }
- }
-
- @Override
- public Optional getAirbyteStream() {
- return Optional.of(pair);
- }
-
- private boolean isCdcSync(MySqlInitialLoadStateManager initialLoadStateManager) {
- if (initialLoadStateManager instanceof MySqlInitialLoadGlobalStateManager) {
- LOGGER.info("Running a cdc sync");
- return true;
- } else {
- LOGGER.info("Not running a cdc sync");
- return false;
- }
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/initialsync/MySqlInitialLoadStateManager.java b/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/initialsync/MySqlInitialLoadStateManager.java
deleted file mode 100644
index 6109191bcedf..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/initialsync/MySqlInitialLoadStateManager.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.source.mysql.initialsync;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import io.airbyte.cdk.integrations.source.relationaldb.state.SourceStateMessageProducer;
-import io.airbyte.integrations.source.mysql.initialsync.MySqlInitialReadUtil.PrimaryKeyInfo;
-import io.airbyte.integrations.source.mysql.internal.models.InternalModels.StateType;
-import io.airbyte.integrations.source.mysql.internal.models.PrimaryKeyLoadStatus;
-import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair;
-import io.airbyte.protocol.models.v0.AirbyteMessage;
-import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Objects;
-import java.util.function.Function;
-
-public abstract class MySqlInitialLoadStateManager implements SourceStateMessageProducer {
-
- public static final long MYSQL_STATUS_VERSION = 2;
- public static final String STATE_TYPE_KEY = "state_type";
- public static final String PRIMARY_KEY_STATE_TYPE = "primary_key";
-
- protected Function streamStateForIncrementalRunSupplier;
-
- protected Map pairToPrimaryKeyLoadStatus;
-
- // Map of pair to the primary key info (field name & data type) associated with it.
- protected Map pairToPrimaryKeyInfo;
-
- void setStreamStateForIncrementalRunSupplier(final Function streamStateForIncrementalRunSupplier) {
- this.streamStateForIncrementalRunSupplier = streamStateForIncrementalRunSupplier;
- }
-
- // Updates the {@link PrimaryKeyLoadStatus} for the state associated with the given pair
- public void updatePrimaryKeyLoadState(final AirbyteStreamNameNamespacePair pair, final PrimaryKeyLoadStatus pkLoadStatus) {
- pairToPrimaryKeyLoadStatus.put(pair, pkLoadStatus);
- }
-
- // Returns the previous state emitted, represented as a {@link PrimaryKeyLoadStatus} associated with
- // the stream.
- public PrimaryKeyLoadStatus getPrimaryKeyLoadStatus(final AirbyteStreamNameNamespacePair pair) {
- return pairToPrimaryKeyLoadStatus.get(pair);
- }
-
- // Returns the current {@PrimaryKeyInfo}, associated with the stream. This includes the data type &
- // the column name associated with the stream.
- public abstract PrimaryKeyInfo getPrimaryKeyInfo(final AirbyteStreamNameNamespacePair pair);
-
- protected JsonNode getIncrementalState(final AirbyteStreamNameNamespacePair pair) {
- final PrimaryKeyLoadStatus currentPkLoadStatus = getPrimaryKeyLoadStatus(pair);
- return (currentPkLoadStatus == null || currentPkLoadStatus.getIncrementalState() == null) ? streamStateForIncrementalRunSupplier.apply(pair)
- : currentPkLoadStatus.getIncrementalState();
- }
-
- @Override
- public AirbyteMessage processRecordMessage(final ConfiguredAirbyteStream stream, final AirbyteMessage message) {
- if (Objects.nonNull(message)) {
- final AirbyteStreamNameNamespacePair pair = new AirbyteStreamNameNamespacePair(stream.getStream().getName(), stream.getStream().getNamespace());
- final String pkFieldName = this.getPrimaryKeyInfo(pair).pkFieldName();
- final String lastPk = message.getRecord().getData().get(pkFieldName).asText();
- final PrimaryKeyLoadStatus pkStatus = new PrimaryKeyLoadStatus()
- .withVersion(MYSQL_STATUS_VERSION)
- .withStateType(StateType.PRIMARY_KEY)
- .withPkName(pkFieldName)
- .withPkVal(lastPk)
- .withIncrementalState(getIncrementalState(pair));
- this.updatePrimaryKeyLoadState(pair, pkStatus);
- }
- return message;
- }
-
- @Override
- public boolean shouldEmitStateMessage(final ConfiguredAirbyteStream stream) {
- return true;
- }
-
- public static Map initPairToPrimaryKeyLoadStatusMap(
- final Map pairToPkStatus) {
- final Map map = new HashMap<>();
- pairToPkStatus.forEach((pair, pkStatus) -> {
- final AirbyteStreamNameNamespacePair updatedPair = new AirbyteStreamNameNamespacePair(pair.getName(), pair.getNamespace());
- map.put(updatedPair, pkStatus);
- });
- return map;
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/initialsync/MySqlInitialLoadStreamStateManager.java b/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/initialsync/MySqlInitialLoadStreamStateManager.java
deleted file mode 100644
index 3cf91c569226..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/initialsync/MySqlInitialLoadStreamStateManager.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.source.mysql.initialsync;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import io.airbyte.commons.json.Jsons;
-import io.airbyte.integrations.source.mysql.initialsync.MySqlInitialReadUtil.InitialLoadStreams;
-import io.airbyte.integrations.source.mysql.initialsync.MySqlInitialReadUtil.PrimaryKeyInfo;
-import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair;
-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.ConfiguredAirbyteCatalog;
-import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream;
-import io.airbyte.protocol.models.v0.StreamDescriptor;
-import java.util.Map;
-import java.util.Objects;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This state manager extends the StreamStateManager to enable writing the state_type and version
- * keys to the stream state when they're going through the iterator Once we have verified that
- * expanding StreamStateManager itself to include this functionality, this class will be removed
- */
-public class MySqlInitialLoadStreamStateManager extends MySqlInitialLoadStateManager {
-
- private static final Logger LOGGER = LoggerFactory.getLogger(MySqlInitialLoadStreamStateManager.class);
-
- public MySqlInitialLoadStreamStateManager(final ConfiguredAirbyteCatalog catalog,
- final InitialLoadStreams initialLoadStreams,
- final Map pairToPrimaryKeyInfo) {
- this.pairToPrimaryKeyInfo = pairToPrimaryKeyInfo;
- this.pairToPrimaryKeyLoadStatus = MySqlInitialLoadStateManager.initPairToPrimaryKeyLoadStatusMap(initialLoadStreams.pairToInitialLoadStatus());
- this.streamStateForIncrementalRunSupplier = pair -> Jsons.emptyObject();
- }
-
- @Override
- public AirbyteStateMessage createFinalStateMessage(final ConfiguredAirbyteStream stream) {
- AirbyteStreamNameNamespacePair pair = new AirbyteStreamNameNamespacePair(stream.getStream().getName(), stream.getStream().getNamespace());
- final JsonNode incrementalState = getIncrementalState(pair);
- if (incrementalState == null || incrementalState.isEmpty()) {
- // resumeable full refresh
- return generateStateMessageAtCheckpoint(stream);
- }
-
- return new AirbyteStateMessage()
- .withType(AirbyteStateType.STREAM)
- .withStream(getAirbyteStreamState(pair, incrementalState));
- }
-
- @Override
- public PrimaryKeyInfo getPrimaryKeyInfo(final io.airbyte.protocol.models.AirbyteStreamNameNamespacePair pair) {
- return pairToPrimaryKeyInfo.get(pair);
- }
-
- @Override
- public AirbyteStateMessage generateStateMessageAtCheckpoint(final ConfiguredAirbyteStream stream) {
- AirbyteStreamNameNamespacePair pair = new AirbyteStreamNameNamespacePair(stream.getStream().getName(), stream.getStream().getNamespace());
- var pkStatus = getPrimaryKeyLoadStatus(pair);
- return new AirbyteStateMessage()
- .withType(AirbyteStateType.STREAM)
- .withStream(getAirbyteStreamState(pair, Jsons.jsonNode(pkStatus)));
- }
-
- protected AirbyteStreamState getAirbyteStreamState(final io.airbyte.protocol.models.AirbyteStreamNameNamespacePair pair, final JsonNode stateData) {
- LOGGER.info("STATE DATA FOR {}: {}", pair.getNamespace().concat("_").concat(pair.getName()), stateData);
- assert Objects.nonNull(pair.getName());
- assert Objects.nonNull(pair.getNamespace());
-
- return new AirbyteStreamState()
- .withStreamDescriptor(
- new StreamDescriptor().withName(pair.getName()).withNamespace(pair.getNamespace()))
- .withStreamState(stateData);
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/initialsync/MySqlInitialReadUtil.java b/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/initialsync/MySqlInitialReadUtil.java
deleted file mode 100644
index 3a7d99c2805f..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/java/io/airbyte/integrations/source/mysql/initialsync/MySqlInitialReadUtil.java
+++ /dev/null
@@ -1,582 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.source.mysql.initialsync;
-
-import static io.airbyte.cdk.db.DbAnalyticsUtils.cdcCursorInvalidMessage;
-import static io.airbyte.cdk.db.DbAnalyticsUtils.cdcResyncMessage;
-import static io.airbyte.cdk.db.DbAnalyticsUtils.wassOccurrenceMessage;
-import static io.airbyte.integrations.source.mysql.MySqlQueryUtils.getTableSizeInfoForStreams;
-import static io.airbyte.integrations.source.mysql.MySqlSpecConstants.FAIL_SYNC_OPTION;
-import static io.airbyte.integrations.source.mysql.MySqlSpecConstants.INVALID_CDC_CURSOR_POSITION_PROPERTY;
-import static io.airbyte.integrations.source.mysql.MySqlSpecConstants.RESYNC_DATA_OPTION;
-import static io.airbyte.integrations.source.mysql.cdc.MysqlCdcStateConstants.MYSQL_CDC_OFFSET;
-import static io.airbyte.integrations.source.mysql.initialsync.MySqlInitialLoadGlobalStateManager.STATE_TYPE_KEY;
-import static io.airbyte.integrations.source.mysql.initialsync.MySqlInitialLoadStateManager.PRIMARY_KEY_STATE_TYPE;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.google.common.collect.Sets;
-import com.mysql.cj.MysqlType;
-import io.airbyte.cdk.db.jdbc.JdbcDatabase;
-import io.airbyte.cdk.integrations.base.AirbyteTraceMessageUtility;
-import io.airbyte.cdk.integrations.debezium.AirbyteDebeziumHandler;
-import io.airbyte.cdk.integrations.debezium.internals.DebeziumEventConverter;
-import io.airbyte.cdk.integrations.debezium.internals.RecordWaitTimeUtil;
-import io.airbyte.cdk.integrations.debezium.internals.RelationalDbDebeziumEventConverter;
-import io.airbyte.cdk.integrations.debezium.internals.RelationalDbDebeziumPropertiesManager;
-import io.airbyte.cdk.integrations.source.relationaldb.CdcStateManager;
-import io.airbyte.cdk.integrations.source.relationaldb.DbSourceDiscoverUtil;
-import io.airbyte.cdk.integrations.source.relationaldb.InitialLoadTimeoutUtil;
-import io.airbyte.cdk.integrations.source.relationaldb.TableInfo;
-import io.airbyte.cdk.integrations.source.relationaldb.models.CdcState;
-import io.airbyte.cdk.integrations.source.relationaldb.state.StateManager;
-import io.airbyte.cdk.integrations.source.relationaldb.streamstatus.StreamStatusTraceEmitterIterator;
-import io.airbyte.commons.exceptions.ConfigErrorException;
-import io.airbyte.commons.json.Jsons;
-import io.airbyte.commons.stream.AirbyteStreamStatusHolder;
-import io.airbyte.commons.util.AutoCloseableIterator;
-import io.airbyte.commons.util.AutoCloseableIterators;
-import io.airbyte.integrations.source.mysql.MySqlQueryUtils;
-import io.airbyte.integrations.source.mysql.MySqlSourceOperations;
-import io.airbyte.integrations.source.mysql.cdc.MySqlCdcConnectorMetadataInjector;
-import io.airbyte.integrations.source.mysql.cdc.MySqlCdcPosition;
-import io.airbyte.integrations.source.mysql.cdc.MySqlCdcProperties;
-import io.airbyte.integrations.source.mysql.cdc.MySqlCdcSavedInfoFetcher;
-import io.airbyte.integrations.source.mysql.cdc.MySqlCdcStateHandler;
-import io.airbyte.integrations.source.mysql.cdc.MySqlCdcTargetPosition;
-import io.airbyte.integrations.source.mysql.cdc.MySqlDebeziumStateUtil;
-import io.airbyte.integrations.source.mysql.cdc.MySqlDebeziumStateUtil.MysqlDebeziumStateAttributes;
-import io.airbyte.integrations.source.mysql.internal.models.CursorBasedStatus;
-import io.airbyte.integrations.source.mysql.internal.models.PrimaryKeyLoadStatus;
-import io.airbyte.protocol.models.CommonField;
-import io.airbyte.protocol.models.v0.*;
-import java.time.Duration;
-import java.time.Instant;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class MySqlInitialReadUtil {
-
- private static final Logger LOGGER = LoggerFactory.getLogger(MySqlInitialReadUtil.class);
-
- public static Optional getMySqlFullRefreshInitialLoadHandler(final JdbcDatabase database,
- final ConfiguredAirbyteCatalog catalog,
- final MySqlInitialLoadGlobalStateManager initialLoadStateManager,
- final StateManager stateManager,
- final ConfiguredAirbyteStream fullRefreshStream,
- final Instant emittedAt,
- final String quoteString,
- final boolean savedOffsetStillPresentOnServer) {
- final InitialLoadStreams initialLoadStreams =
- cdcStreamsForInitialPrimaryKeyLoad(stateManager.getCdcStateManager(), catalog, savedOffsetStillPresentOnServer);
-
- // State manager will need to know all streams in order to produce a state message
- // But for initial load handler we only want to produce iterator on the single full refresh stream.
- if (!initialLoadStreams.streamsForInitialLoad().isEmpty()) {
-
- // Filter on initialLoadStream
- final var pair = new AirbyteStreamNameNamespacePair(fullRefreshStream.getStream().getName(), fullRefreshStream.getStream().getNamespace());
- final var pkStatus = initialLoadStreams.pairToInitialLoadStatus.get(pair);
- final Map fullRefreshPkStatus;
- if (pkStatus == null) {
- fullRefreshPkStatus = Map.of();
- } else {
- fullRefreshPkStatus = Map.of(pair, pkStatus);
- }
-
- var fullRefreshStreamInitialLoad = new InitialLoadStreams(List.of(fullRefreshStream),
- fullRefreshPkStatus);
- return Optional
- .of(getMySqlInitialLoadHandler(database, emittedAt, quoteString, fullRefreshStreamInitialLoad, initialLoadStateManager, Optional.empty()));
- }
- return Optional.empty();
- }
-
- private static MySqlInitialLoadHandler getMySqlInitialLoadHandler(
- final JdbcDatabase database,
- final Instant emittedAt,
- final String quoteString,
- final InitialLoadStreams initialLoadStreams,
- final MySqlInitialLoadStateManager initialLoadStateManager,
- final Optional cdcMetadataInjector) {
- final JsonNode sourceConfig = database.getSourceConfig();
-
- final MySqlSourceOperations sourceOperations =
- new MySqlSourceOperations(cdcMetadataInjector);
- return new MySqlInitialLoadHandler(sourceConfig, database,
- sourceOperations,
- quoteString,
- initialLoadStateManager,
- Optional.empty(),
- getTableSizeInfoForStreams(database, initialLoadStreams.streamsForInitialLoad(), quoteString));
- }
-
- private static CdcState getDefaultCdcState(final JdbcDatabase database,
- final ConfiguredAirbyteCatalog catalog) {
-
- // Construct the initial state for MySQL. If there is already existing state, we use that instead
- // since that is associated with the debezium
- // state associated with the initial sync.
- final MySqlDebeziumStateUtil mySqlDebeziumStateUtil = new MySqlDebeziumStateUtil();
- final JsonNode initialDebeziumState = mySqlDebeziumStateUtil.constructInitialDebeziumState(
- MySqlCdcProperties.getDebeziumProperties(database), catalog, database);
- return new CdcState().withState(initialDebeziumState);
- }
-
- public static boolean isSavedOffsetStillPresentOnServer(final JdbcDatabase database,
- final ConfiguredAirbyteCatalog catalog,
- final StateManager stateManager) {
- final MySqlDebeziumStateUtil mySqlDebeziumStateUtil = new MySqlDebeziumStateUtil();
- final JsonNode sourceConfig = database.getSourceConfig();
- final JsonNode initialDebeziumState = mySqlDebeziumStateUtil.constructInitialDebeziumState(
- MySqlCdcProperties.getDebeziumProperties(database), catalog, database);
-
- final JsonNode state =
- (stateManager.getCdcStateManager().getCdcState() == null || stateManager.getCdcStateManager().getCdcState().getState() == null)
- ? initialDebeziumState
- : Jsons.clone(stateManager.getCdcStateManager().getCdcState().getState());
-
- final Optional savedOffset = mySqlDebeziumStateUtil.savedOffset(
- MySqlCdcProperties.getDebeziumProperties(database), catalog, state.get(MYSQL_CDC_OFFSET), sourceConfig);
-
- final boolean savedOffsetStillPresentOnServer =
- savedOffset.isPresent() && mySqlDebeziumStateUtil.savedOffsetStillPresentOnServer(database, savedOffset.get());
- if (!savedOffsetStillPresentOnServer) {
- AirbyteTraceMessageUtility.emitAnalyticsTrace(cdcCursorInvalidMessage());
- if (!sourceConfig.get("replication_method").has(INVALID_CDC_CURSOR_POSITION_PROPERTY) || sourceConfig.get("replication_method").get(
- INVALID_CDC_CURSOR_POSITION_PROPERTY).asText().equals(FAIL_SYNC_OPTION)) {
- throw new ConfigErrorException(
- "Saved offset no longer present on the server. Please reset the connection, and then increase binlog retention and/or increase sync frequency. See https://docs.airbyte.com/integrations/sources/mysql/mysql-troubleshooting#under-cdc-incremental-mode-there-are-still-full-refresh-syncs for more details.");
- } else if (sourceConfig.get("replication_method").get(INVALID_CDC_CURSOR_POSITION_PROPERTY).asText().equals(RESYNC_DATA_OPTION)) {
- AirbyteTraceMessageUtility.emitAnalyticsTrace(cdcResyncMessage());
- LOGGER.warn("Saved offset no longer present on the server, Airbyte is going to trigger a sync from scratch");
- }
- }
- return savedOffsetStillPresentOnServer;
- }
-
- public static MySqlInitialLoadGlobalStateManager getMySqlInitialLoadGlobalStateManager(final JdbcDatabase database,
- final ConfiguredAirbyteCatalog catalog,
- final StateManager stateManager,
- final Map>> tableNameToTable,
- final String quoteString,
- final boolean savedOffsetStillPresentOnServer) {
- final InitialLoadStreams initialLoadStreams =
- cdcStreamsForInitialPrimaryKeyLoad(stateManager.getCdcStateManager(), catalog, savedOffsetStillPresentOnServer);
-
- return new MySqlInitialLoadGlobalStateManager(initialLoadStreams,
- initPairToPrimaryKeyInfoMap(database, catalog, tableNameToTable, quoteString),
- stateManager, catalog, savedOffsetStillPresentOnServer, getDefaultCdcState(database, catalog));
- }
-
- /*
- * Returns the read iterators associated with : 1. Initial cdc read snapshot via primary key
- * queries. 2. Incremental cdc reads via debezium.
- *
- * The initial load iterators need to always be run before the incremental cdc iterators. This is to
- * prevent advancing the binlog offset in the state before all streams have snapshotted. Otherwise,
- * there could be data loss.
- */
- public static List> getCdcReadIterators(final JdbcDatabase database,
- final ConfiguredAirbyteCatalog catalog,
- final Map>> tableNameToTable,
- final StateManager stateManager,
- final MySqlInitialLoadGlobalStateManager initialLoadGlobalStateManager,
- final Instant emittedAt,
- final String quoteString,
- final boolean savedOffsetStillPresentOnServer) {
- final JsonNode sourceConfig = database.getSourceConfig();
- final Duration firstRecordWaitTime = RecordWaitTimeUtil.getFirstRecordWaitTime(sourceConfig);
- LOGGER.info("First record waiting time: {} seconds", firstRecordWaitTime.getSeconds());
- final Duration initialLoadTimeout = InitialLoadTimeoutUtil.getInitialLoadTimeout(sourceConfig);
- // Determine the streams that need to be loaded via primary key sync.
- final List> initialLoadIterator = new ArrayList<>();
- final InitialLoadStreams initialLoadStreams =
- cdcStreamsForInitialPrimaryKeyLoad(stateManager.getCdcStateManager(), catalog, savedOffsetStillPresentOnServer);
-
- final MySqlCdcConnectorMetadataInjector metadataInjector = MySqlCdcConnectorMetadataInjector.getInstance(emittedAt);
- final CdcState stateToBeUsed;
- final CdcState cdcState = stateManager.getCdcStateManager().getCdcState();
- if (!savedOffsetStillPresentOnServer || cdcState == null
- || cdcState.getState() == null) {
- stateToBeUsed = getDefaultCdcState(database, catalog);
- } else {
- stateToBeUsed = cdcState;
- }
-
- // Debezium is started for streams that have been started - that is they have been partially or
- // fully completed.
- final var startedCdcStreamList = catalog.getStreams().stream()
- .filter(stream -> stream.getSyncMode() == SyncMode.INCREMENTAL)
- .filter(stream -> isStreamPartiallyOrFullyCompleted(stream, initialLoadStreams))
- .map(stream -> stream.getStream().getNamespace() + "." + stream.getStream().getName()).toList();
-
- final var allCdcStreamList = catalog.getStreams().stream()
- .filter(stream -> stream.getSyncMode() == SyncMode.INCREMENTAL)
- .map(stream -> stream.getStream().getNamespace() + "." + stream.getStream().getName()).toList();
-
- // If there are streams to sync via primary key load, build the relevant iterators.
- if (!initialLoadStreams.streamsForInitialLoad().isEmpty()) {
-
- final MysqlDebeziumStateAttributes stateAttributes = MySqlDebeziumStateUtil.getStateAttributesFromDB(database);
-
- final MySqlInitialLoadHandler initialLoadHandler =
- getMySqlInitialLoadHandler(database, emittedAt, quoteString, initialLoadStreams, initialLoadGlobalStateManager,
- Optional.of(new CdcMetadataInjector(emittedAt.toString(), stateAttributes, metadataInjector)));
-
- // Start and complete stream status messages are emitted while constructing the full set of initial
- // load and incremental debezium iterators.
- initialLoadIterator.addAll(initialLoadHandler.getIncrementalIterators(
- new ConfiguredAirbyteCatalog().withStreams(initialLoadStreams.streamsForInitialLoad()),
- tableNameToTable,
- emittedAt, false, false, Optional.of(initialLoadTimeout)));
- }
-
- // CDC stream status messages should be emitted for streams.
- final List> cdcStreamsStartStatusEmitters = catalog.getStreams().stream()
- .filter(stream -> stream.getSyncMode() == SyncMode.INCREMENTAL)
- .map(stream -> (AutoCloseableIterator) new StreamStatusTraceEmitterIterator(
- new AirbyteStreamStatusHolder(
- new io.airbyte.protocol.models.AirbyteStreamNameNamespacePair(stream.getStream().getName(), stream.getStream().getNamespace()),
- AirbyteStreamStatusTraceMessage.AirbyteStreamStatus.STARTED)))
- .toList();
-
- final List> cdcStreamsEndStatusEmitters = catalog.getStreams().stream()
- .filter(stream -> stream.getSyncMode() == SyncMode.INCREMENTAL)
- .map(stream -> (AutoCloseableIterator) new StreamStatusTraceEmitterIterator(
- new AirbyteStreamStatusHolder(
- new io.airbyte.protocol.models.AirbyteStreamNameNamespacePair(stream.getStream().getName(), stream.getStream().getNamespace()),
- AirbyteStreamStatusTraceMessage.AirbyteStreamStatus.COMPLETE)))
- .toList();
-
- // Build the incremental CDC iterators.
- final AirbyteDebeziumHandler handler = new AirbyteDebeziumHandler(
- sourceConfig,
- MySqlCdcTargetPosition.targetPosition(database),
- true,
- firstRecordWaitTime,
- AirbyteDebeziumHandler.QUEUE_CAPACITY,
- false);
- final var eventConverter = new RelationalDbDebeziumEventConverter(metadataInjector, emittedAt);
-
- if (startedCdcStreamList.isEmpty()) {
- LOGGER.info("First sync - no cdc streams have been completed or started");
- /*
- * This is the first run case - no initial loads have been started. In this case, we want to run the
- * iterators in the following order: 1. Run the initial load iterators. This step will timeout and
- * throw a transient error if run for too long (> 8hrs by default). 2. Run the debezium iterators
- * with ALL of the incremental streams configured. This is because if step 1 completes, the initial
- * load can be considered finished.
- */
- final var propertiesManager = new RelationalDbDebeziumPropertiesManager(
- MySqlCdcProperties.getDebeziumProperties(database), sourceConfig, catalog, allCdcStreamList);
- final Supplier> incrementalIteratorsSupplier = getCdcIncrementalIteratorsSupplier(handler,
- propertiesManager, eventConverter, stateToBeUsed, stateManager);
- return Collections.singletonList(
- AutoCloseableIterators.concatWithEagerClose(
- Stream
- .of(
- cdcStreamsStartStatusEmitters,
- initialLoadIterator,
- Collections.singletonList(AutoCloseableIterators.lazyIterator(incrementalIteratorsSupplier, null)),
- cdcStreamsEndStatusEmitters)
- .flatMap(Collection::stream)
- .collect(Collectors.toList()),
- AirbyteTraceMessageUtility::emitStreamStatusTrace));
- } else if (initialLoadIterator.isEmpty()) {
- LOGGER.info("Initial load has finished completely - only reading the binlog");
- /*
- * In this case, the initial load has completed and only debezium should be run. The iterators
- * should be run in the following order: 1. Run the debezium iterators with ALL of the incremental
- * streams configured.
- */
- final var propertiesManager = new RelationalDbDebeziumPropertiesManager(
- MySqlCdcProperties.getDebeziumProperties(database), sourceConfig, catalog, allCdcStreamList);
- final Supplier> incrementalIteratorSupplier = getCdcIncrementalIteratorsSupplier(handler,
- propertiesManager, eventConverter, stateToBeUsed, stateManager);
- return Collections.singletonList(
- AutoCloseableIterators.concatWithEagerClose(
- Stream
- .of(
- cdcStreamsStartStatusEmitters,
- Collections.singletonList(AutoCloseableIterators.lazyIterator(incrementalIteratorSupplier, null)),
- cdcStreamsEndStatusEmitters)
- .flatMap(Collection::stream)
- .collect(Collectors.toList()),
- AirbyteTraceMessageUtility::emitStreamStatusTrace));
- } else {
- LOGGER.info("Initial load is in progress - reading binlog first and then resuming with initial load.");
- /*
- * In this case, the initial load has partially completed (WASS case). The iterators should be run
- * in the following order: 1. Run the debezium iterators with only the incremental streams which
- * have been fully or partially completed configured. 2. Resume initial load for partially completed
- * and not started streams. This step will timeout and throw a transient error if run for too long
- * (> 8hrs by default).
- */
- AirbyteTraceMessageUtility.emitAnalyticsTrace(wassOccurrenceMessage());
- final var propertiesManager = new RelationalDbDebeziumPropertiesManager(
- MySqlCdcProperties.getDebeziumProperties(database), sourceConfig, catalog, startedCdcStreamList);
- final Supplier> incrementalIteratorSupplier = getCdcIncrementalIteratorsSupplier(handler,
- propertiesManager, eventConverter, stateToBeUsed, stateManager);
- return Collections.singletonList(
- AutoCloseableIterators.concatWithEagerClose(
- Stream
- .of(
- cdcStreamsStartStatusEmitters,
- Collections.singletonList(AutoCloseableIterators.lazyIterator(incrementalIteratorSupplier, null)),
- initialLoadIterator,
- cdcStreamsEndStatusEmitters)
- .flatMap(Collection::stream)
- .collect(Collectors.toList()),
- AirbyteTraceMessageUtility::emitStreamStatusTrace));
- }
- }
-
- @SuppressWarnings("unchecked")
- private static Supplier> getCdcIncrementalIteratorsSupplier(AirbyteDebeziumHandler handler,
- RelationalDbDebeziumPropertiesManager propertiesManager,
- DebeziumEventConverter eventConverter,
- CdcState stateToBeUsed,
- StateManager stateManager) {
- return () -> handler.getIncrementalIterators(
- propertiesManager, eventConverter, new MySqlCdcSavedInfoFetcher(stateToBeUsed), new MySqlCdcStateHandler(stateManager));
- }
-
- /**
- * CDC specific: Determines the streams to sync for initial primary key load. These include streams
- * that are (i) currently in primary key load (ii) newly added incremental streams.
- */
- public static InitialLoadStreams cdcStreamsForInitialPrimaryKeyLoad(final CdcStateManager stateManager,
- final ConfiguredAirbyteCatalog fullCatalog,
- final boolean savedOffsetStillPresentOnServer) {
-
- if (!savedOffsetStillPresentOnServer) {
- // Add a filter here to identify resumable full refresh streams.
- return new InitialLoadStreams(
- fullCatalog.getStreams()
- .stream()
- .collect(Collectors.toList()),
- new HashMap<>());
- }
-
- final AirbyteStateMessage airbyteStateMessage = stateManager.getRawStateMessage();
- final Set streamsStillinPkSync = new HashSet<>();
-
- // Build a map of stream <-> initial load status for streams that currently have an initial primary
- // key load in progress.
- final Map pairToInitialLoadStatus = new HashMap<>();
- if (airbyteStateMessage != null && airbyteStateMessage.getGlobal() != null && airbyteStateMessage.getGlobal().getStreamStates() != null) {
- airbyteStateMessage.getGlobal().getStreamStates().forEach(stateMessage -> {
- final JsonNode streamState = stateMessage.getStreamState();
- final StreamDescriptor streamDescriptor = stateMessage.getStreamDescriptor();
- if (streamState == null || streamDescriptor == null) {
- return;
- }
-
- if (streamState.has(STATE_TYPE_KEY)) {
- if (streamState.get(STATE_TYPE_KEY).asText().equalsIgnoreCase(PRIMARY_KEY_STATE_TYPE)) {
- final PrimaryKeyLoadStatus primaryKeyLoadStatus = Jsons.object(streamState, PrimaryKeyLoadStatus.class);
- final AirbyteStreamNameNamespacePair pair = new AirbyteStreamNameNamespacePair(streamDescriptor.getName(),
- streamDescriptor.getNamespace());
- pairToInitialLoadStatus.put(pair, primaryKeyLoadStatus);
- streamsStillinPkSync.add(pair);
- }
- }
- });
- }
-
- final List streamsForPkSync = new ArrayList<>();
- fullCatalog.getStreams().stream()
- .filter(stream -> streamsStillinPkSync.contains(AirbyteStreamNameNamespacePair.fromAirbyteStream(stream.getStream())))
- .map(Jsons::clone)
- .forEach(streamsForPkSync::add);
- final List newlyAddedStreams =
- identifyStreamsToSnapshot(fullCatalog, stateManager.getInitialStreamsSynced());
- streamsForPkSync.addAll(newlyAddedStreams);
-
- return new InitialLoadStreams(streamsForPkSync, pairToInitialLoadStatus);
- }
-
- /**
- * Determines the streams to sync for initial primary key load. These include streams that are (i)
- * currently in primary key load (ii) newly added incremental streams.
- */
- public static InitialLoadStreams streamsForInitialPrimaryKeyLoad(final StateManager stateManager,
- final ConfiguredAirbyteCatalog fullCatalog) {
-
- final List rawStateMessages = stateManager.getRawStateMessages();
- final Set streamsStillInPkSync = new HashSet<>();
- final Set alreadySeenStreamPairs = new HashSet<>();
-
- // Build a map of stream <-> initial load status for streams that currently have an initial primary
- // key load in progress.
- final Map pairToInitialLoadStatus = new HashMap<>();
-
- if (rawStateMessages != null) {
- rawStateMessages.forEach(stateMessage -> {
- final AirbyteStreamState stream = stateMessage.getStream();
- final JsonNode streamState = stream.getStreamState();
- final StreamDescriptor streamDescriptor = stateMessage.getStream().getStreamDescriptor();
- if (streamState == null || streamDescriptor == null) {
- return;
- }
-
- final AirbyteStreamNameNamespacePair pair = new AirbyteStreamNameNamespacePair(streamDescriptor.getName(),
- streamDescriptor.getNamespace());
-
- // Build a map of stream <-> initial load status for streams that currently have an initial primary
- // key load in progress.
-
- if (streamState.has(STATE_TYPE_KEY)) {
- if (streamState.get(STATE_TYPE_KEY).asText().equalsIgnoreCase(PRIMARY_KEY_STATE_TYPE)) {
- final PrimaryKeyLoadStatus primaryKeyLoadStatus = Jsons.object(streamState, PrimaryKeyLoadStatus.class);
- pairToInitialLoadStatus.put(pair, primaryKeyLoadStatus);
- streamsStillInPkSync.add(pair);
- }
- alreadySeenStreamPairs.add(new AirbyteStreamNameNamespacePair(streamDescriptor.getName(), streamDescriptor.getNamespace()));
- }
- });
- }
- final List streamsForPkSync = new ArrayList<>();
- fullCatalog.getStreams().stream()
- .filter(stream -> streamsStillInPkSync.contains(AirbyteStreamNameNamespacePair.fromAirbyteStream(stream.getStream())))
- .map(Jsons::clone)
- .forEach(streamsForPkSync::add);
-
- final List newlyAddedStreams = identifyStreamsToSnapshot(fullCatalog,
- Collections.unmodifiableSet(alreadySeenStreamPairs));
- streamsForPkSync.addAll(newlyAddedStreams);
- return new InitialLoadStreams(streamsForPkSync.stream().filter(MySqlInitialReadUtil::streamHasPrimaryKey).collect(Collectors.toList()),
- pairToInitialLoadStatus);
- }
-
- private static boolean streamHasPrimaryKey(final ConfiguredAirbyteStream stream) {
- return stream.getStream().getSourceDefinedPrimaryKey().size() > 0;
- }
-
- public static InitialLoadStreams filterStreamInIncrementalMode(final InitialLoadStreams stream) {
- return new InitialLoadStreams(
- stream.streamsForInitialLoad.stream().filter(airbyteStream -> airbyteStream.getSyncMode() == SyncMode.INCREMENTAL)
- .collect(Collectors.toList()),
- stream.pairToInitialLoadStatus);
- }
-
- public static List identifyStreamsToSnapshot(final ConfiguredAirbyteCatalog catalog,
- final Set alreadySyncedStreams) {
- final Set allStreams = AirbyteStreamNameNamespacePair.fromConfiguredCatalog(catalog);
- final Set newlyAddedStreams = new HashSet<>(Sets.difference(allStreams, alreadySyncedStreams));
- // Add a filter here to exclude non resumable full refresh streams.
- return catalog.getStreams().stream()
- .filter(stream -> newlyAddedStreams.contains(AirbyteStreamNameNamespacePair.fromAirbyteStream(stream.getStream())))
- .map(Jsons::clone)
- .collect(Collectors.toList());
- }
-
- public static List identifyStreamsForCursorBased(final ConfiguredAirbyteCatalog catalog,
- final List streamsForInitialLoad) {
-
- final Set initialLoadStreamsNamespacePairs =
- streamsForInitialLoad.stream().map(stream -> AirbyteStreamNameNamespacePair.fromAirbyteStream(stream.getStream()))
- .collect(
- Collectors.toSet());
- return catalog.getStreams().stream()
- .filter(stream -> !initialLoadStreamsNamespacePairs.contains(AirbyteStreamNameNamespacePair.fromAirbyteStream(stream.getStream())))
- .map(Jsons::clone)
- .collect(Collectors.toList());
- }
-
- // Build a map of stream <-> primary key info (primary key field name + datatype) for all streams
- // currently undergoing initial primary key syncs.
- public static Map initPairToPrimaryKeyInfoMap(
- final JdbcDatabase database,
- final ConfiguredAirbyteCatalog catalog,
- final Map>> tableNameToTable,
- final String quoteString) {
- final Map pairToPkInfoMap = new HashMap<>();
- // For every stream that was in primary initial key sync, we want to maintain information about the
- // current primary key info associated with the
- // stream
- catalog.getStreams().forEach(stream -> {
- final io.airbyte.protocol.models.AirbyteStreamNameNamespacePair pair =
- new io.airbyte.protocol.models.AirbyteStreamNameNamespacePair(stream.getStream().getName(), stream.getStream().getNamespace());
- final Optional pkInfo = getPrimaryKeyInfo(database, stream, tableNameToTable, quoteString);
- if (pkInfo.isPresent()) {
- pairToPkInfoMap.put(pair, pkInfo.get());
- }
- });
- return pairToPkInfoMap;
- }
-
- // Returns the primary key info associated with the stream.
- private static Optional getPrimaryKeyInfo(final JdbcDatabase database,
- final ConfiguredAirbyteStream stream,
- final Map>> tableNameToTable,
- final String quoteString) {
- final String fullyQualifiedTableName =
- DbSourceDiscoverUtil.getFullyQualifiedTableName(stream.getStream().getNamespace(), (stream.getStream().getName()));
- final TableInfo> table = tableNameToTable
- .get(fullyQualifiedTableName);
- return getPrimaryKeyInfo(database, stream, table, quoteString);
- }
-
- private static Optional getPrimaryKeyInfo(final JdbcDatabase database,
- final ConfiguredAirbyteStream stream,
- final TableInfo> table,
- final String quoteString) {
- // For cursor-based syncs, we cannot always assume a primary key field exists. We need to handle the
- // case where it does not exist when we support
- // cursor-based syncs.
- if (stream.getStream().getSourceDefinedPrimaryKey().size() > 1) {
- LOGGER.info("Composite primary key detected for {namespace, stream} : {}, {}", stream.getStream().getNamespace(), stream.getStream().getName());
- }
- if (stream.getStream().getSourceDefinedPrimaryKey().isEmpty()) {
- return Optional.empty();
- }
-
- final String pkFieldName = stream.getStream().getSourceDefinedPrimaryKey().getFirst().getFirst();
- final MysqlType pkFieldType = table.getFields().stream()
- .filter(field -> field.getName().equals(pkFieldName))
- .findFirst().get().getType();
-
- final String pkMaxValue = MySqlQueryUtils.getMaxPkValueForStream(database, stream, pkFieldName, quoteString);
- return Optional.of(new PrimaryKeyInfo(pkFieldName, pkFieldType, pkMaxValue));
- }
-
- private static boolean isStreamPartiallyOrFullyCompleted(ConfiguredAirbyteStream stream, InitialLoadStreams initialLoadStreams) {
- boolean isStreamCompleted = !initialLoadStreams.streamsForInitialLoad.contains(stream);
- // A stream has been partially completed if an initial load status exists.
- boolean isStreamPartiallyCompleted = (initialLoadStreams.pairToInitialLoadStatus
- .get(new AirbyteStreamNameNamespacePair(stream.getStream().getName(), stream.getStream().getNamespace()))) != null;
- return isStreamCompleted || isStreamPartiallyCompleted;
- }
-
- public record InitialLoadStreams(List streamsForInitialLoad,
- Map pairToInitialLoadStatus) {
-
- }
-
- public record CursorBasedStreams(List streamsForCursorBased,
- Map pairToCursorBasedStatus) {
-
- }
-
- public record PrimaryKeyInfo(String pkFieldName, MysqlType fieldType, String pkMaxValue) {}
-
- public static AirbyteStreamNameNamespacePair convertNameNamespacePairFromV0(final io.airbyte.protocol.models.AirbyteStreamNameNamespacePair v1NameNamespacePair) {
- return new AirbyteStreamNameNamespacePair(v1NameNamespacePair.getName(), v1NameNamespacePair.getNamespace());
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/kotlin/MySqlSourceExceptionHandler.kt b/airbyte-integrations/connectors/source-mysql/src/main/kotlin/MySqlSourceExceptionHandler.kt
deleted file mode 100644
index 7853afc08fb4..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/kotlin/MySqlSourceExceptionHandler.kt
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
- */
-package io.airbyte.integrations.source.mysql
-
-import io.airbyte.cdk.integrations.util.ConnectorErrorProfile
-import io.airbyte.cdk.integrations.util.ConnectorExceptionHandler
-import io.airbyte.cdk.integrations.util.FailureType
-
-class MySqlSourceExceptionHandler : ConnectorExceptionHandler() {
- override fun initializeErrorDictionary() {
- // adding common error profiles
- super.initializeErrorDictionary()
- // adding connector specific error profiles
- add(
- ConnectorErrorProfile(
- errorClass = "MySQL Syntax Exception",
- regexMatchingPattern = ".*unknown column '.+' in 'field list'.*",
- failureType = FailureType.CONFIG,
- externalMessage =
- "A column needed by MySQL source connector is missing in the database",
- sampleInternalMessage = "Unknown column 'X' in 'field list'",
- ),
- )
-
- add(
- ConnectorErrorProfile(
- errorClass = "MySQL EOF Exception",
- regexMatchingPattern =
- ".*can not read response from server. expected to read [1-9]\\d* bytes.*",
- failureType = FailureType.TRANSIENT,
- externalMessage = "Can not read data from MySQL server",
- sampleInternalMessage =
- "java.io.EOFException: Can not read response from server. Expected to read X bytes, read Y bytes before connection was unexpectedly lost.",
- ),
- )
-
- add(
- ConnectorErrorProfile(
- errorClass = "MySQL Hikari Connection Error",
- regexMatchingPattern = ".*connection is not available, request timed out after*",
- failureType = FailureType.TRANSIENT,
- externalMessage = "Database read failed due to connection timeout, will retry.",
- sampleInternalMessage =
- "java.sql.SQLTransientConnectionException: HikariPool-x - Connection is not available, request timed out after xms",
- referenceLinks = listOf("https://github.com/airbytehq/airbyte/issues/41614"),
- ),
- )
-
- add(
- ConnectorErrorProfile(
- errorClass = "MySQL Timezone Error",
- regexMatchingPattern = ".*is unrecognized or represents more than one time zone*",
- failureType = FailureType.CONFIG,
- externalMessage =
- "Please configure your database with the correct timezone found in the detailed error message. " +
- "Please refer to the following documentation: https://dev.mysql.com/doc/refman/8.4/en/time-zone-support.html",
- sampleInternalMessage =
- "java.lang.RuntimeException: Connector configuration is not valid. Unable to connect: " +
- "The server time zone value 'PDT' is unrecognized or represents more than one time zone. " +
- "You must configure either the server or JDBC driver (via the 'connectionTimeZone' configuration property) to " +
- "use a more specific time zone value if you want to utilize time zone support.",
- referenceLinks =
- listOf(
- "https://github.com/airbytehq/airbyte/issues/41614",
- "https://github.com/airbytehq/oncall/issues/5250",
- ),
- ),
- )
-
- add(
- ConnectorErrorProfile(
- errorClass = "MySQL Schema change error",
- regexMatchingPattern = ".*whose schema isn't known to this connector*",
- failureType = FailureType.CONFIG,
- externalMessage =
- "Your connection could not be completed because changes were detected on an unknown table (see detailed error for the table name), " +
- "please refresh your schema or reset the connection.",
- sampleInternalMessage =
- "java.lang.RuntimeException: java.lang.RuntimeException: org.apache.kafka.connect.errors." +
- "ConnectException: An exception occurred in the change event producer. This connector will be stopped.",
- referenceLinks =
- listOf("https://github.com/airbytehq/airbyte-internal-issues/issues/7156"),
- ),
- )
-
- add(
- ConnectorErrorProfile(
- errorClass = "MySQL limit reached",
- regexMatchingPattern =
- ".*query execution was interrupted, maximum statement execution time exceeded*",
- failureType = FailureType.TRANSIENT,
- externalMessage =
- "The query took too long to return results, the database read was aborted. Will retry.",
- sampleInternalMessage =
- "java.lang.RuntimeException: java.lang.RuntimeException: java.sql.SQLException: " +
- "Query execution was interrupted, maximum statement execution time exceeded",
- referenceLinks =
- listOf("https://github.com/airbytehq/airbyte-internal-issues/issues/7155"),
- ),
- )
- }
-}
diff --git a/airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlCdcInitialSnapshotStateValue.kt b/airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlCdcInitialSnapshotStateValue.kt
similarity index 100%
rename from airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlCdcInitialSnapshotStateValue.kt
rename to airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlCdcInitialSnapshotStateValue.kt
diff --git a/airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlCdcMetaFields.kt b/airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlCdcMetaFields.kt
similarity index 100%
rename from airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlCdcMetaFields.kt
rename to airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlCdcMetaFields.kt
diff --git a/airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlJdbcEncryption.kt b/airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlJdbcEncryption.kt
similarity index 100%
rename from airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlJdbcEncryption.kt
rename to airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlJdbcEncryption.kt
diff --git a/airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlJdbcPartition.kt b/airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlJdbcPartition.kt
similarity index 100%
rename from airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlJdbcPartition.kt
rename to airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlJdbcPartition.kt
diff --git a/airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlJdbcPartitionFactory.kt b/airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlJdbcPartitionFactory.kt
similarity index 100%
rename from airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlJdbcPartitionFactory.kt
rename to airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlJdbcPartitionFactory.kt
diff --git a/airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlJdbcStreamStateValue.kt b/airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlJdbcStreamStateValue.kt
similarity index 100%
rename from airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlJdbcStreamStateValue.kt
rename to airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlJdbcStreamStateValue.kt
diff --git a/airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlSource.kt b/airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlSource.kt
similarity index 100%
rename from airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlSource.kt
rename to airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlSource.kt
diff --git a/airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlSourceConfiguration.kt b/airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlSourceConfiguration.kt
similarity index 100%
rename from airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlSourceConfiguration.kt
rename to airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlSourceConfiguration.kt
diff --git a/airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlSourceConfigurationSpecification.kt b/airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlSourceConfigurationSpecification.kt
similarity index 100%
rename from airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlSourceConfigurationSpecification.kt
rename to airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlSourceConfigurationSpecification.kt
diff --git a/airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlSourceMetadataQuerier.kt b/airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlSourceMetadataQuerier.kt
similarity index 100%
rename from airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlSourceMetadataQuerier.kt
rename to airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlSourceMetadataQuerier.kt
diff --git a/airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlSourceOperations.kt b/airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlSourceOperations.kt
similarity index 100%
rename from airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlSourceOperations.kt
rename to airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/MysqlSourceOperations.kt
diff --git a/airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/cdc/MySqlDebeziumOperations.kt b/airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/cdc/MySqlDebeziumOperations.kt
similarity index 100%
rename from airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/cdc/MySqlDebeziumOperations.kt
rename to airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/cdc/MySqlDebeziumOperations.kt
diff --git a/airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/cdc/MySqlPosition.kt b/airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/cdc/MySqlPosition.kt
similarity index 100%
rename from airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/cdc/MySqlPosition.kt
rename to airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/cdc/MySqlPosition.kt
diff --git a/airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/cdc/converters/MySQLBooleanConverter.kt b/airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/cdc/converters/MySQLBooleanConverter.kt
similarity index 100%
rename from airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/cdc/converters/MySQLBooleanConverter.kt
rename to airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/cdc/converters/MySQLBooleanConverter.kt
diff --git a/airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/cdc/converters/MySQLDateTimeConverter.kt b/airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/cdc/converters/MySQLDateTimeConverter.kt
similarity index 100%
rename from airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/cdc/converters/MySQLDateTimeConverter.kt
rename to airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/cdc/converters/MySQLDateTimeConverter.kt
diff --git a/airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/cdc/converters/MySQLNumbericConverter.kt b/airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/cdc/converters/MySQLNumbericConverter.kt
similarity index 100%
rename from airbyte-integrations/connectors/source-mysql-v2/src/main/kotlin/io/airbyte/integrations/source/mysql/cdc/converters/MySQLNumbericConverter.kt
rename to airbyte-integrations/connectors/source-mysql/src/main/kotlin/io/airbyte/integrations/source/mysql/cdc/converters/MySQLNumbericConverter.kt
diff --git a/airbyte-integrations/connectors/source-mysql-v2/src/main/resources/application.yml b/airbyte-integrations/connectors/source-mysql/src/main/resources/application.yml
similarity index 100%
rename from airbyte-integrations/connectors/source-mysql-v2/src/main/resources/application.yml
rename to airbyte-integrations/connectors/source-mysql/src/main/resources/application.yml
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/resources/internal_models/internal_models.yaml b/airbyte-integrations/connectors/source-mysql/src/main/resources/internal_models/internal_models.yaml
deleted file mode 100644
index d7c998e4c714..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/resources/internal_models/internal_models.yaml
+++ /dev/null
@@ -1,48 +0,0 @@
----
-"$schema": http://json-schema.org/draft-07/schema#
-title: MySQL Models
-type: object
-description: MySQL Models
-properties:
- state_type:
- "$ref": "#/definitions/StateType"
- primary_key_state:
- "$ref": "#/definitions/PrimaryKeyLoadStatus"
- cursor_based_state:
- "$ref": "#/definitions/CursorBasedStatus"
-definitions:
- StateType:
- description: Enum to define the sync mode of state.
- type: string
- enum:
- - cursor_based
- - primary_key
- CursorBasedStatus:
- type: object
- extends:
- type: object
- existingJavaType: "io.airbyte.cdk.integrations.source.relationaldb.models.DbStreamState"
- properties:
- state_type:
- "$ref": "#/definitions/StateType"
- version:
- description: Version of state.
- type: integer
- PrimaryKeyLoadStatus:
- type: object
- properties:
- version:
- description: Version of state.
- type: integer
- state_type:
- "$ref": "#/definitions/StateType"
- pk_name:
- description: primary key name
- type: string
- pk_val:
- description: primary key watermark
- type: string
- incremental_state:
- description: State to switch to after completion of primary key initial sync
- type: object
- existingJavaType: com.fasterxml.jackson.databind.JsonNode
diff --git a/airbyte-integrations/connectors/source-mysql/src/main/resources/spec.json b/airbyte-integrations/connectors/source-mysql/src/main/resources/spec.json
deleted file mode 100644
index 5a9304326cdd..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/main/resources/spec.json
+++ /dev/null
@@ -1,252 +0,0 @@
-{
- "documentationUrl": "https://docs.airbyte.com/integrations/sources/mysql",
- "connectionSpecification": {
- "$schema": "http://json-schema.org/draft-07/schema#",
- "title": "MySql Source Spec",
- "type": "object",
- "required": ["host", "port", "database", "username", "replication_method"],
- "properties": {
- "host": {
- "description": "The host name of the database.",
- "title": "Host",
- "type": "string",
- "order": 0
- },
- "port": {
- "description": "The port to connect to.",
- "title": "Port",
- "type": "integer",
- "minimum": 0,
- "maximum": 65536,
- "default": 3306,
- "examples": ["3306"],
- "order": 1
- },
- "database": {
- "description": "The database name.",
- "title": "Database",
- "type": "string",
- "order": 2
- },
- "username": {
- "description": "The username which is used to access the database.",
- "title": "Username",
- "type": "string",
- "order": 3
- },
- "password": {
- "description": "The password associated with the username.",
- "title": "Password",
- "type": "string",
- "airbyte_secret": true,
- "order": 4,
- "always_show": true
- },
- "jdbc_url_params": {
- "description": "Additional properties to pass to the JDBC URL string when connecting to the database formatted as 'key=value' pairs separated by the symbol '&'. (example: key1=value1&key2=value2&key3=value3). For more information read about JDBC URL parameters.",
- "title": "JDBC URL Parameters (Advanced)",
- "type": "string",
- "order": 5
- },
- "ssl": {
- "title": "SSL Connection",
- "description": "Encrypt data using SSL.",
- "type": "boolean",
- "default": true,
- "order": 6
- },
- "ssl_mode": {
- "title": "SSL modes",
- "description": "SSL connection modes. Read more in the docs.",
- "type": "object",
- "order": 7,
- "oneOf": [
- {
- "title": "preferred",
- "description": "Automatically attempt SSL connection. If the MySQL server does not support SSL, continue with a regular connection.",
- "required": ["mode"],
- "properties": {
- "mode": {
- "type": "string",
- "const": "preferred",
- "order": 0
- }
- }
- },
- {
- "title": "required",
- "description": "Always connect with SSL. If the MySQL server doesn’t support SSL, the connection will not be established. Certificate Authority (CA) and Hostname are not verified.",
- "required": ["mode"],
- "properties": {
- "mode": {
- "type": "string",
- "const": "required",
- "order": 0
- }
- }
- },
- {
- "title": "Verify CA",
- "description": "Always connect with SSL. Verifies CA, but allows connection even if Hostname does not match.",
- "required": ["mode", "ca_certificate"],
- "properties": {
- "mode": {
- "type": "string",
- "const": "verify_ca",
- "order": 0
- },
- "ca_certificate": {
- "type": "string",
- "title": "CA certificate",
- "description": "CA certificate",
- "airbyte_secret": true,
- "multiline": true,
- "order": 1
- },
- "client_certificate": {
- "type": "string",
- "title": "Client certificate",
- "description": "Client certificate (this is not a required field, but if you want to use it, you will need to add the Client key as well)",
- "airbyte_secret": true,
- "multiline": true,
- "order": 2,
- "always_show": true
- },
- "client_key": {
- "type": "string",
- "title": "Client key",
- "description": "Client key (this is not a required field, but if you want to use it, you will need to add the Client certificate as well)",
- "airbyte_secret": true,
- "multiline": true,
- "order": 3,
- "always_show": true
- },
- "client_key_password": {
- "type": "string",
- "title": "Client key password",
- "description": "Password for keystorage. This field is optional. If you do not add it - the password will be generated automatically.",
- "airbyte_secret": true,
- "order": 4
- }
- }
- },
- {
- "title": "Verify Identity",
- "description": "Always connect with SSL. Verify both CA and Hostname.",
- "required": ["mode", "ca_certificate"],
- "properties": {
- "mode": {
- "type": "string",
- "const": "verify_identity",
- "order": 0
- },
- "ca_certificate": {
- "type": "string",
- "title": "CA certificate",
- "description": "CA certificate",
- "airbyte_secret": true,
- "multiline": true,
- "order": 1
- },
- "client_certificate": {
- "type": "string",
- "title": "Client certificate",
- "description": "Client certificate (this is not a required field, but if you want to use it, you will need to add the Client key as well)",
- "airbyte_secret": true,
- "multiline": true,
- "order": 2,
- "always_show": true
- },
- "client_key": {
- "type": "string",
- "title": "Client key",
- "description": "Client key (this is not a required field, but if you want to use it, you will need to add the Client certificate as well)",
- "airbyte_secret": true,
- "multiline": true,
- "order": 3,
- "always_show": true
- },
- "client_key_password": {
- "type": "string",
- "title": "Client key password",
- "description": "Password for keystorage. This field is optional. If you do not add it - the password will be generated automatically.",
- "airbyte_secret": true,
- "order": 4
- }
- }
- }
- ]
- },
- "replication_method": {
- "type": "object",
- "title": "Update Method",
- "description": "Configures how data is extracted from the database.",
- "order": 8,
- "default": "CDC",
- "display_type": "radio",
- "oneOf": [
- {
- "title": "Read Changes using Binary Log (CDC)",
- "description": "Recommended - Incrementally reads new inserts, updates, and deletes using the MySQL binary log. This must be enabled on your database.",
- "required": ["method"],
- "properties": {
- "method": {
- "type": "string",
- "const": "CDC",
- "order": 0
- },
- "initial_waiting_seconds": {
- "type": "integer",
- "title": "Initial Waiting Time in Seconds (Advanced)",
- "description": "The amount of time the connector will wait when it launches to determine if there is new data to sync or not. Defaults to 300 seconds. Valid range: 120 seconds to 1200 seconds. Read about initial waiting time.",
- "default": 300,
- "min": 120,
- "max": 1200,
- "order": 1,
- "always_show": true
- },
- "server_time_zone": {
- "type": "string",
- "title": "Configured server timezone for the MySQL source (Advanced)",
- "description": "Enter the configured MySQL server timezone. This should only be done if the configured timezone in your MySQL instance does not conform to IANNA standard.",
- "order": 2,
- "always_show": true
- },
- "invalid_cdc_cursor_position_behavior": {
- "type": "string",
- "title": "Invalid CDC position behavior (Advanced)",
- "description": "Determines whether Airbyte should fail or re-sync data in case of an stale/invalid cursor value into the WAL. If 'Fail sync' is chosen, a user will have to manually reset the connection before being able to continue syncing data. If 'Re-sync data' is chosen, Airbyte will automatically trigger a refresh but could lead to higher cloud costs and data loss.",
- "enum": ["Fail sync", "Re-sync data"],
- "default": "Fail sync",
- "order": 3,
- "always_show": true
- },
- "initial_load_timeout_hours": {
- "type": "integer",
- "title": "Initial Load Timeout in Hours (Advanced)",
- "description": "The amount of time an initial load is allowed to continue for before catching up on CDC logs.",
- "default": 8,
- "min": 4,
- "max": 24,
- "order": 4,
- "always_show": true
- }
- }
- },
- {
- "title": "Scan Changes with User Defined Cursor",
- "description": "Incrementally detects new inserts and updates using the cursor column chosen when configuring a connection (e.g. created_at, updated_at).",
- "required": ["method"],
- "properties": {
- "method": {
- "type": "string",
- "const": "STANDARD",
- "order": 0
- }
- }
- }
- ]
- }
- }
- }
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/AbstractMySqlSourceDatatypeTest.java b/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/AbstractMySqlSourceDatatypeTest.java
deleted file mode 100644
index 8eb622e72716..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/AbstractMySqlSourceDatatypeTest.java
+++ /dev/null
@@ -1,501 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.io.airbyte.integration_tests.sources;
-
-import com.mysql.cj.MysqlType;
-import io.airbyte.cdk.integrations.standardtest.source.AbstractSourceDatabaseTypeTest;
-import io.airbyte.cdk.integrations.standardtest.source.TestDataHolder;
-import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase;
-import io.airbyte.protocol.models.JsonSchemaType;
-import java.io.File;
-import java.io.IOException;
-import java.util.Base64;
-import java.util.Set;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang3.RandomStringUtils;
-import org.apache.commons.lang3.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public abstract class AbstractMySqlSourceDatatypeTest extends AbstractSourceDatabaseTypeTest {
-
- protected static final Logger LOGGER = LoggerFactory.getLogger(AbstractMySqlSourceDatatypeTest.class);
-
- protected MySQLTestDatabase testdb;
-
- @Override
- protected String getNameSpace() {
- return testdb.getDatabaseName();
- }
-
- @Override
- protected void tearDown(final TestDestinationEnv testEnv) {
- testdb.close();
- }
-
- @Override
- protected String getImageName() {
- return "airbyte/source-mysql:dev";
- }
-
- @Override
- protected void initTests() {
- // bit defaults to bit(1), which is equivalent to boolean
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("bit")
- .airbyteType(JsonSchemaType.BOOLEAN)
- .addInsertValues("null", "1", "0")
- .addExpectedValues(null, "true", "false")
- .build());
-
- // bit(1) is equivalent to boolean
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("bit")
- .fullSourceDataType("bit(1)")
- .airbyteType(JsonSchemaType.BOOLEAN)
- .addInsertValues("null", "1", "0")
- .addExpectedValues(null, "true", "false")
- .build());
-
- // bit(>1) is binary
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("bit")
- .fullSourceDataType("bit(7)")
- .airbyteType(JsonSchemaType.STRING_BASE_64)
- // 1000001 is binary for A
- .addInsertValues("null", "b'1000001'")
- // QQo= is base64 encoding in charset UTF-8 for A
- .addExpectedValues(null, "QQ==")
- .build());
-
- // tinyint without width
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("tinyint")
- .airbyteType(JsonSchemaType.INTEGER)
- .addInsertValues("null", "-128", "127")
- .addExpectedValues(null, "-128", "127")
- .build());
-
- // tinyint(1) is equivalent to boolean
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("tinyint")
- .fullSourceDataType("tinyint(1)")
- .airbyteType(JsonSchemaType.BOOLEAN)
- .addInsertValues("null", "1", "0")
- .addExpectedValues(null, "true", "false")
- .build());
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("tinyint")
- .fullSourceDataType("tinyint(1) unsigned")
- .airbyteType(JsonSchemaType.INTEGER)
- .addInsertValues("null", "0", "1", "2", "3")
- .addExpectedValues(null, "0", "1", "2", "3")
- .build());
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("tinyint")
- .fullSourceDataType("tinyint(2)")
- .airbyteType(JsonSchemaType.INTEGER)
- .addInsertValues("null", "-128", "127")
- .addExpectedValues(null, "-128", "127")
- .build());
-
- final Set booleanTypes = Set.of("BOOLEAN", "BOOL");
- for (final String booleanType : booleanTypes) {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(booleanType)
- .airbyteType(JsonSchemaType.BOOLEAN)
- // MySql booleans are tinyint(1), and only 1 is true
- .addInsertValues("null", "1", "0")
- .addExpectedValues(null, "true", "false")
- .build());
- }
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("smallint")
- .airbyteType(JsonSchemaType.INTEGER)
- .addInsertValues("null", "-32768", "32767")
- .addExpectedValues(null, "-32768", "32767")
- .build());
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("smallint")
- .airbyteType(JsonSchemaType.INTEGER)
- .fullSourceDataType("smallint zerofill")
- .addInsertValues("1")
- .addExpectedValues("1")
- .build());
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("smallint")
- .airbyteType(JsonSchemaType.INTEGER)
- .fullSourceDataType("smallint unsigned")
- .addInsertValues("null", "0", "65535")
- .addExpectedValues(null, "0", "65535")
- .build());
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("mediumint")
- .airbyteType(JsonSchemaType.INTEGER)
- .addInsertValues("null", "-8388608", "8388607")
- .addExpectedValues(null, "-8388608", "8388607")
- .build());
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("mediumint")
- .airbyteType(JsonSchemaType.INTEGER)
- .fullSourceDataType("mediumint zerofill")
- .addInsertValues("1")
- .addExpectedValues("1")
- .build());
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("int")
- .airbyteType(JsonSchemaType.INTEGER)
- .addInsertValues("null", "-2147483648", "2147483647")
- .addExpectedValues(null, "-2147483648", "2147483647")
- .build());
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("int")
- .airbyteType(JsonSchemaType.INTEGER)
- .fullSourceDataType("int unsigned")
- .addInsertValues("3428724653")
- .addExpectedValues("3428724653")
- .build());
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("int")
- .airbyteType(JsonSchemaType.INTEGER)
- .fullSourceDataType("int zerofill")
- .addInsertValues("1")
- .addExpectedValues("1")
- .build());
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("bigint")
- .airbyteType(JsonSchemaType.INTEGER)
- .addInsertValues("null", "9223372036854775807")
- .addExpectedValues(null, "9223372036854775807")
- .build());
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("float")
- .airbyteType(JsonSchemaType.NUMBER)
- .addInsertValues("null", "10.5")
- .addExpectedValues(null, "10.5")
- .build());
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("double")
- .airbyteType(JsonSchemaType.NUMBER)
- .addInsertValues("null", "power(10, 308)", "1/power(10, 45)", "10.5")
- .addExpectedValues(null, String.valueOf(Math.pow(10, 308)), String.valueOf(1 / Math.pow(10, 45)), "10.5")
- .build());
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("decimal")
- .airbyteType(JsonSchemaType.NUMBER)
- .fullSourceDataType("decimal(10,3)")
- .addInsertValues("0.188", "null")
- .addExpectedValues("0.188", null)
- .build());
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("decimal")
- .airbyteType(JsonSchemaType.INTEGER)
- .fullSourceDataType("decimal(32,0)")
- .addInsertValues("1700000.01", "123")
- .addExpectedValues("1700000", "123")
- .build());
-
- for (final String type : Set.of("date", "date not null default '0000-00-00'")) {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("date")
- .fullSourceDataType(type)
- .airbyteType(JsonSchemaType.STRING_DATE)
- .addInsertValues("'1999-01-08'", "'2021-01-01'", "'2022/11/12'", "'1987.12.01'")
- .addExpectedValues("1999-01-08", "2021-01-01", "2022-11-12", "1987-12-01")
- .build());
- }
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("date")
- .airbyteType(JsonSchemaType.STRING_DATE)
- .addInsertValues("null")
- .addExpectedValues((String) null)
- .build());
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("date")
- .airbyteType(JsonSchemaType.STRING_DATE)
- .addInsertValues("0000-00-00")
- .addExpectedValues((String) null)
- .build());
-
- for (final String fullSourceType : Set.of("datetime", "datetime not null default now()")) {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("datetime")
- .fullSourceDataType(fullSourceType)
- .airbyteType(JsonSchemaType.STRING_TIMESTAMP_WITHOUT_TIMEZONE)
- .addInsertValues("'2005-10-10 23:22:21'", "'2013-09-05T10:10:02'", "'2013-09-06T10:10:02'")
- .addExpectedValues("2005-10-10T23:22:21", "2013-09-05T10:10:02", "2013-09-06T10:10:02")
- .build());
- }
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("datetime")
- .airbyteType(JsonSchemaType.STRING_TIMESTAMP_WITHOUT_TIMEZONE)
- .addInsertValues("null")
- .addExpectedValues((String) null)
- .build());
-
- addTimestampDataTypeTest();
-
- for (final String fullSourceType : Set.of("time", "time not null default '00:00:00'")) {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("time")
- .fullSourceDataType(fullSourceType)
- .airbyteType(JsonSchemaType.STRING_TIME_WITHOUT_TIMEZONE)
- // JDBC driver can process only "clock"(00:00:00-23:59:59) values.
- .addInsertValues("'-22:59:59'", "'23:59:59'", "'00:00:00'")
- .addExpectedValues("22:59:59", "23:59:59", "00:00:00.000000")
- .build());
-
- }
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("time")
- .airbyteType(JsonSchemaType.STRING_TIME_WITHOUT_TIMEZONE)
- // JDBC driver can process only "clock"(00:00:00-23:59:59) values.
- .addInsertValues("null")
- .addExpectedValues((String) null)
- .build());
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("year")
- .airbyteType(JsonSchemaType.INTEGER)
- // MySQL converts values in the ranges '0' - '69' to YEAR value in the range 2000 - 2069
- // and '70' - '99' to 1970 - 1999.
- .addInsertValues("null", "'1997'", "'0'", "'50'", "'70'", "'80'", "'99'", "'00'", "'000'")
- .addExpectedValues(null, "1997", "2000", "2050", "1970", "1980", "1999", "2000", "2000")
- .build());
-
- // char types can be string or binary, so they are tested separately
- final Set charTypes = Stream.of(MysqlType.CHAR, MysqlType.VARCHAR)
- .map(Enum::name)
- .collect(Collectors.toSet());
- for (final String charType : charTypes) {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(charType)
- .airbyteType(JsonSchemaType.STRING)
- .fullSourceDataType(charType + "(63)")
- .addInsertValues("null", "'Airbyte'", "'!\"#$%&\\'()*+,-./:;<=>?\\@[\\]^_\\`{|}~'")
- .addExpectedValues(null, "Airbyte", "!\"#$%&'()*+,-./:;<=>?@[]^_`{|}~")
- .build());
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(charType)
- .airbyteType(JsonSchemaType.STRING)
- .fullSourceDataType(charType + "(63) character set utf16")
- .addInsertValues("0xfffd")
- .addExpectedValues("�")
- .build());
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(charType)
- .airbyteType(JsonSchemaType.STRING)
- .fullSourceDataType(charType + "(63) character set cp1251")
- .addInsertValues("'тест'")
- .addExpectedValues("тест")
- .build());
-
- // when charset is binary, return binary in base64 encoding in charset UTF-8
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(charType)
- .airbyteType(JsonSchemaType.STRING_BASE_64)
- .fullSourceDataType(charType + "(7) character set binary")
- .addInsertValues("null", "'Airbyte'")
- .addExpectedValues(null, "QWlyYnl0ZQ==")
- .build());
- }
-
- final Set blobTypes = Stream
- .of(MysqlType.TINYBLOB, MysqlType.BLOB, MysqlType.MEDIUMBLOB, MysqlType.LONGBLOB)
- .map(Enum::name)
- .collect(Collectors.toSet());
- for (final String blobType : blobTypes) {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(blobType)
- .airbyteType(JsonSchemaType.STRING_BASE_64)
- .addInsertValues("null", "'Airbyte'")
- .addExpectedValues(null, "QWlyYnl0ZQ==")
- .build());
- }
-
- // binary appends '\0' to the end of the string
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(MysqlType.BINARY.name())
- .fullSourceDataType(MysqlType.BINARY.name() + "(10)")
- .airbyteType(JsonSchemaType.STRING_BASE_64)
- .addInsertValues("null", "'Airbyte'")
- .addExpectedValues(null, "QWlyYnl0ZQAAAA==")
- .build());
-
- // varbinary does not append '\0' to the end of the string
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(MysqlType.VARBINARY.name())
- .fullSourceDataType(MysqlType.VARBINARY.name() + "(10)")
- .airbyteType(JsonSchemaType.STRING_BASE_64)
- .addInsertValues("null", "'Airbyte'")
- .addExpectedValues(null, "QWlyYnl0ZQ==")
- .build());
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(MysqlType.VARBINARY.name())
- .airbyteType(JsonSchemaType.STRING_BASE_64)
- .fullSourceDataType(MysqlType.VARBINARY.name() + "(20000)") // size should be enough to save test.png
- .addInsertValues("null", "'test'", "'тест'", String.format("FROM_BASE64('%s')", getFileDataInBase64()))
- .addExpectedValues(null, "dGVzdA==", "0YLQtdGB0YI=", getFileDataInBase64())
- .build());
-
- final Set textTypes = Stream
- .of(MysqlType.TINYTEXT, MysqlType.TEXT, MysqlType.MEDIUMTEXT, MysqlType.LONGTEXT)
- .map(Enum::name)
- .collect(Collectors.toSet());
- final String randomText = RandomStringUtils.random(50, true, true);
- for (final String textType : textTypes) {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(textType)
- .airbyteType(JsonSchemaType.STRING)
- .addInsertValues("null", "'Airbyte'", String.format("'%s'", randomText))
- .addExpectedValues(null, "Airbyte", randomText)
- .build());
- }
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("mediumtext")
- .airbyteType(JsonSchemaType.STRING)
- .addInsertValues(getLogString(1048000), "'test'")
- .addExpectedValues(StringUtils.leftPad("0", 1048000, "0"), "test")
- .build());
-
- addJsonDataTypeTest();
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("enum")
- .fullSourceDataType("ENUM('xs', 's', 'm', 'l', 'xl')")
- .airbyteType(JsonSchemaType.STRING)
- .addInsertValues("null", "'xs'", "'m'")
- .addExpectedValues(null, "xs", "m")
- .build());
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("set")
- .fullSourceDataType("SET('xs', 's', 'm', 'l', 'xl')")
- .airbyteType(JsonSchemaType.STRING)
- .addInsertValues("null", "'xs,s'", "'m,xl'")
- .addExpectedValues(null, "xs,s", "m,xl")
- .build());
-
- addDecimalValuesTest();
- }
-
- protected void addJsonDataTypeTest() {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("json")
- .airbyteType(JsonSchemaType.STRING)
- .addInsertValues("null", "'{\"a\": 10, \"b\": 15}'", "'{\"fóo\": \"bär\"}'", "'{\"春江潮水连海平\": \"海上明月共潮生\"}'")
- .addExpectedValues(null, "{\"a\": 10, \"b\": 15}", "{\"fóo\": \"bär\"}", "{\"春江潮水连海平\": \"海上明月共潮生\"}")
- .build());
- }
-
- protected void addTimestampDataTypeTest() {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("timestamp")
- .airbyteType(JsonSchemaType.STRING_TIMESTAMP_WITH_TIMEZONE)
- .addInsertValues("null", "'2021-01-00'", "'2021-00-00'", "'0000-00-00'", "'2022-08-09T10:17:16.161342Z'")
- .addExpectedValues(null, null, null, null, "2022-08-09T10:17:16.000000Z")
- .build());
- }
-
- private String getLogString(final int length) {
- final int maxLpadLength = 262144;
- final StringBuilder stringBuilder = new StringBuilder("concat(");
- final int fullChunks = length / maxLpadLength;
- stringBuilder.append("lpad('0', 262144, '0'),".repeat(fullChunks));
- stringBuilder.append("lpad('0', ").append(length % maxLpadLength).append(", '0'))");
- return stringBuilder.toString();
- }
-
- private String getFileDataInBase64() {
- final File file = new File(getClass().getClassLoader().getResource("test.png").getFile());
- try {
- return Base64.getEncoder().encodeToString(FileUtils.readFileToByteArray(file));
- } catch (final IOException e) {
- LOGGER.error(String.format("Fail to read the file: %s. Error: %s", file.getAbsoluteFile(), e.getMessage()));
- }
- return null;
- }
-
- protected void addDecimalValuesTest() {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("decimal")
- .airbyteType(JsonSchemaType.NUMBER)
- .fullSourceDataType("decimal(19,2)")
- .addInsertValues("1700000.01", "'123'")
- .addExpectedValues("1700000.01", "123.0")
- .build());
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/AbstractSshMySqlSourceAcceptanceTest.java b/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/AbstractSshMySqlSourceAcceptanceTest.java
deleted file mode 100644
index 765495b85469..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/AbstractSshMySqlSourceAcceptanceTest.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.io.airbyte.integration_tests.sources;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.google.common.collect.Lists;
-import io.airbyte.cdk.db.jdbc.JdbcUtils;
-import io.airbyte.cdk.integrations.base.ssh.SshHelpers;
-import io.airbyte.cdk.integrations.standardtest.source.SourceAcceptanceTest;
-import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
-import io.airbyte.commons.io.IOs;
-import io.airbyte.commons.json.Jsons;
-import io.airbyte.protocol.models.Field;
-import io.airbyte.protocol.models.JsonSchemaType;
-import io.airbyte.protocol.models.v0.CatalogHelpers;
-import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog;
-import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream;
-import io.airbyte.protocol.models.v0.ConnectorSpecification;
-import io.airbyte.protocol.models.v0.DestinationSyncMode;
-import io.airbyte.protocol.models.v0.SyncMode;
-import java.nio.file.Path;
-import java.util.HashMap;
-
-public abstract class AbstractSshMySqlSourceAcceptanceTest extends SourceAcceptanceTest {
-
- private static final String STREAM_NAME = "id_and_name";
- private static final String STREAM_NAME2 = "starships";
-
- private JsonNode config;
-
- public abstract Path getConfigFilePath();
-
- @Override
- protected void setupEnvironment(final TestDestinationEnv environment) {
- config = Jsons.deserialize(IOs.readFile(getConfigFilePath()));
- }
-
- @Override
- protected void tearDown(final TestDestinationEnv testEnv) {}
-
- @Override
- protected String getImageName() {
- return "airbyte/source-mysql:dev";
- }
-
- @Override
- protected ConnectorSpecification getSpec() throws Exception {
- return SshHelpers.getSpecAndInjectSsh();
- }
-
- @Override
- protected JsonNode getConfig() {
- return config;
- }
-
- @Override
- protected ConfiguredAirbyteCatalog getConfiguredCatalog() {
- return new ConfiguredAirbyteCatalog().withStreams(Lists.newArrayList(
- new ConfiguredAirbyteStream()
- .withSyncMode(SyncMode.INCREMENTAL)
- .withCursorField(Lists.newArrayList("id"))
- .withDestinationSyncMode(DestinationSyncMode.APPEND)
- .withStream(CatalogHelpers.createAirbyteStream(
- String.format("%s", STREAM_NAME),
- Field.of("id", JsonSchemaType.NUMBER),
- Field.of("name", JsonSchemaType.STRING))
- .withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL))),
- new ConfiguredAirbyteStream()
- .withSyncMode(SyncMode.INCREMENTAL)
- .withCursorField(Lists.newArrayList("id"))
- .withDestinationSyncMode(DestinationSyncMode.APPEND)
- .withStream(CatalogHelpers.createAirbyteStream(
- String.format("%s", STREAM_NAME2),
- config.get(JdbcUtils.DATABASE_KEY).asText(),
- Field.of("id", JsonSchemaType.NUMBER),
- Field.of("name", JsonSchemaType.STRING))
- .withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL)))));
- }
-
- @Override
- protected JsonNode getState() {
- return Jsons.jsonNode(new HashMap<>());
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CDCMySqlDatatypeAccuracyTest.java b/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CDCMySqlDatatypeAccuracyTest.java
deleted file mode 100644
index 9400c4e66368..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CDCMySqlDatatypeAccuracyTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.io.airbyte.integration_tests.sources;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import io.airbyte.cdk.db.Database;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase.BaseImage;
-import org.junit.jupiter.api.Test;
-
-public class CDCMySqlDatatypeAccuracyTest extends MySqlDatatypeAccuracyTest {
-
- @Override
- protected JsonNode getConfig() {
- return testdb.integrationTestConfigBuilder()
- .withoutSsl()
- .withCdcReplication()
- .with("snapshot_mode", "initial_only")
- .build();
- }
-
- @Override
- protected Database setupDatabase() {
- testdb = MySQLTestDatabase.in(BaseImage.MYSQL_8).withoutStrictMode().withCdcPermissions();
- return testdb.getDatabase();
- }
-
- // Temporarily disable this test since it's causing trouble on GHA.
- @Override
- @Test
- public void testDataContent() {
- // Do Nothing
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CdcBinlogsMySqlSourceDatatypeTest.java b/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CdcBinlogsMySqlSourceDatatypeTest.java
deleted file mode 100644
index 82ab112d7e15..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CdcBinlogsMySqlSourceDatatypeTest.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.io.airbyte.integration_tests.sources;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.google.common.collect.Iterables;
-import io.airbyte.cdk.db.Database;
-import io.airbyte.cdk.integrations.standardtest.source.TestDataHolder;
-import io.airbyte.commons.json.Jsons;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase.BaseImage;
-import io.airbyte.protocol.models.JsonSchemaType;
-import io.airbyte.protocol.models.v0.AirbyteMessage;
-import io.airbyte.protocol.models.v0.AirbyteStateMessage;
-import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog;
-import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream;
-import java.util.List;
-
-public class CdcBinlogsMySqlSourceDatatypeTest extends AbstractMySqlSourceDatatypeTest {
-
- private JsonNode stateAfterFirstSync;
-
- @Override
- protected JsonNode getConfig() {
- return testdb.integrationTestConfigBuilder()
- .withoutSsl()
- .withCdcReplication()
- .build();
- }
-
- @Override
- protected Database setupDatabase() {
- testdb = MySQLTestDatabase.in(BaseImage.MYSQL_8).withoutStrictMode().withCdcPermissions();
- return testdb.getDatabase();
- }
-
- @Override
- protected List runRead(final ConfiguredAirbyteCatalog configuredCatalog) throws Exception {
- if (stateAfterFirstSync == null) {
- throw new RuntimeException("stateAfterFirstSync is null");
- }
- return super.runRead(configuredCatalog, stateAfterFirstSync);
- }
-
- @Override
- protected void postSetup() throws Exception {
- final var database = testdb.getDatabase();
- for (final TestDataHolder test : testDataHolders) {
- database.query(ctx -> {
- ctx.execute("TRUNCATE TABLE " + test.getNameWithTestPrefix() + ";");
- return null;
- });
- }
-
- final ConfiguredAirbyteStream dummyTableWithData = createDummyTableWithData(database);
- final ConfiguredAirbyteCatalog catalog = getConfiguredCatalog();
- catalog.getStreams().add(dummyTableWithData);
-
- final List allMessages = super.runRead(catalog);
- final List stateAfterFirstBatch = extractStateMessages(allMessages);
- stateAfterFirstSync = Jsons.jsonNode(List.of(Iterables.getLast(stateAfterFirstBatch)));
- if (stateAfterFirstSync == null) {
- throw new RuntimeException("stateAfterFirstSync should not be null");
- }
- for (final TestDataHolder test : testDataHolders) {
- database.query(ctx -> {
- test.getInsertSqlQueries().forEach(ctx::fetch);
- return null;
- });
- }
- }
-
- @Override
- public boolean testCatalog() {
- return true;
- }
-
- @Override
- protected void addTimestampDataTypeTest() {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("timestamp")
- .airbyteType(JsonSchemaType.STRING_TIMESTAMP_WITH_TIMEZONE)
- .addInsertValues("null", "'2021-01-00'", "'2021-00-00'", "'0000-00-00'", "'2022-08-09T10:17:16.161342Z'")
- .addExpectedValues(null, "1970-01-01T00:00:00.000000Z", "1970-01-01T00:00:00.000000Z", "1970-01-01T00:00:00.000000Z",
- "2022-08-09T10:17:16.000000Z")
- .build());
- }
-
- @Override
- protected void addJsonDataTypeTest() {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("json")
- .airbyteType(JsonSchemaType.STRING)
- .addInsertValues("null", "'{\"a\":10,\"b\":15}'", "'{\"fóo\":\"bär\"}'", "'{\"春江潮水连海平\":\"海上明月共潮生\"}'")
- .addExpectedValues(null, "{\"a\":10,\"b\":15}", "{\"fóo\":\"bär\"}", "{\"春江潮水连海平\":\"海上明月共潮生\"}")
- .build());
- }
-
- @Override
- protected void addDecimalValuesTest() {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType("decimal")
- .airbyteType(JsonSchemaType.NUMBER)
- .fullSourceDataType("decimal(19,2)")
- .addInsertValues("1700000.01", "'123'")
- .addExpectedValues("1700000.01", "123.00")
- .build());
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CdcInitialSnapshotMySqlSourceDatatypeTest.java b/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CdcInitialSnapshotMySqlSourceDatatypeTest.java
deleted file mode 100644
index 6b971c86927c..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CdcInitialSnapshotMySqlSourceDatatypeTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.io.airbyte.integration_tests.sources;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import io.airbyte.cdk.db.Database;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase.BaseImage;
-
-public class CdcInitialSnapshotMySqlSourceDatatypeTest extends AbstractMySqlSourceDatatypeTest {
-
- @Override
- protected JsonNode getConfig() {
- return testdb.integrationTestConfigBuilder()
- .withoutSsl()
- .withCdcReplication()
- .with("snapshot_mode", "initial_only")
- .build();
- }
-
- @Override
- protected Database setupDatabase() {
- testdb = MySQLTestDatabase.in(BaseImage.MYSQL_8).withoutStrictMode().withCdcPermissions();
- return testdb.getDatabase();
- }
-
- @Override
- public boolean testCatalog() {
- return true;
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CdcMySqlSourceAcceptanceTest.java b/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CdcMySqlSourceAcceptanceTest.java
deleted file mode 100644
index 8286c3087991..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CdcMySqlSourceAcceptanceTest.java
+++ /dev/null
@@ -1,254 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.io.airbyte.integration_tests.sources;
-
-import static io.airbyte.protocol.models.v0.SyncMode.FULL_REFRESH;
-import static io.airbyte.protocol.models.v0.SyncMode.INCREMENTAL;
-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 com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import io.airbyte.cdk.integrations.base.ssh.SshHelpers;
-import io.airbyte.cdk.integrations.standardtest.source.SourceAcceptanceTest;
-import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
-import io.airbyte.commons.json.Jsons;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase.BaseImage;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase.ContainerModifier;
-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.AirbyteRecordMessage;
-import io.airbyte.protocol.models.v0.AirbyteStateMessage;
-import io.airbyte.protocol.models.v0.CatalogHelpers;
-import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog;
-import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream;
-import io.airbyte.protocol.models.v0.ConnectorSpecification;
-import io.airbyte.protocol.models.v0.DestinationSyncMode;
-import io.airbyte.protocol.models.v0.SyncMode;
-import java.util.List;
-import java.util.Objects;
-import java.util.stream.Collectors;
-import org.apache.commons.lang3.ArrayUtils;
-import org.junit.Assert;
-import org.junit.jupiter.api.Test;
-
-public class CdcMySqlSourceAcceptanceTest extends SourceAcceptanceTest {
-
- protected static final String STREAM_NAME = "id_and_name";
- protected static final String STREAM_NAME2 = "starships";
- protected static final String STREAM_NAME3 = "stream3";
-
- protected MySQLTestDatabase testdb;
-
- @Override
- protected String getImageName() {
- return "airbyte/source-mysql:dev";
- }
-
- @Override
- protected ConnectorSpecification getSpec() throws Exception {
- return SshHelpers.getSpecAndInjectSsh();
- }
-
- @Override
- protected JsonNode getConfig() {
- return testdb.integrationTestConfigBuilder()
- .withCdcReplication()
- .withoutSsl()
- .build();
- }
-
- @Override
- protected ConfiguredAirbyteCatalog getConfiguredCatalog() {
- return new ConfiguredAirbyteCatalog().withStreams(Lists.newArrayList(
- new ConfiguredAirbyteStream()
- .withSyncMode(INCREMENTAL)
- .withDestinationSyncMode(DestinationSyncMode.APPEND)
- .withStream(CatalogHelpers.createAirbyteStream(
- STREAM_NAME, testdb.getDatabaseName(),
- Field.of("id", JsonSchemaType.NUMBER),
- Field.of("name", JsonSchemaType.STRING))
- .withSourceDefinedCursor(true)
- .withSourceDefinedPrimaryKey(List.of(List.of("id")))
- .withSupportedSyncModes(
- Lists.newArrayList(SyncMode.FULL_REFRESH, INCREMENTAL))),
- new ConfiguredAirbyteStream()
- .withSyncMode(INCREMENTAL)
- .withDestinationSyncMode(DestinationSyncMode.APPEND)
- .withStream(CatalogHelpers.createAirbyteStream(
- STREAM_NAME2, testdb.getDatabaseName(),
- Field.of("id", JsonSchemaType.NUMBER),
- Field.of("name", JsonSchemaType.STRING))
- .withSourceDefinedCursor(true)
- .withSourceDefinedPrimaryKey(List.of(List.of("id")))
- .withSupportedSyncModes(
- Lists.newArrayList(SyncMode.FULL_REFRESH, INCREMENTAL)))));
- }
-
- @Override
- protected JsonNode getState() {
- return null;
- }
-
- @Override
- protected void setupEnvironment(final TestDestinationEnv environment) {
- testdb = MySQLTestDatabase.in(BaseImage.MYSQL_8, getContainerModifiers())
- .withCdcPermissions()
- .with("CREATE TABLE id_and_name(id INTEGER, name VARCHAR(200));")
- .with("INSERT INTO id_and_name (id, name) VALUES (1,'picard'), (2, 'crusher'), (3, 'vash');")
- .with("CREATE TABLE starships(id INTEGER, name VARCHAR(200));")
- .with("INSERT INTO starships (id, name) VALUES (1,'enterprise-d'), (2, 'defiant'), (3, 'yamato');")
- .with("CREATE TABLE %s (id INTEGER PRIMARY KEY, name VARCHAR(200), userid INTEGER DEFAULT NULL);", STREAM_NAME3)
- .with("INSERT INTO %s (id, name) VALUES (4,'voyager');", STREAM_NAME3);
- }
-
- protected ContainerModifier[] getContainerModifiers() {
- return ArrayUtils.toArray();
- }
-
- @Override
- protected void tearDown(final TestDestinationEnv testEnv) {
- testdb.close();
- }
-
- @Test
- public void testIncrementalSyncShouldNotFailIfBinlogIsDeleted() throws Exception {
- final ConfiguredAirbyteCatalog configuredCatalog = withSourceDefinedCursors(getConfiguredCatalog());
- // only sync incremental streams
- configuredCatalog.setStreams(
- configuredCatalog.getStreams().stream().filter(s -> s.getSyncMode() == INCREMENTAL).collect(Collectors.toList()));
-
- final List airbyteMessages = runRead(configuredCatalog, getState());
- final List recordMessages = filterRecords(airbyteMessages);
- final List stateMessages = airbyteMessages
- .stream()
- .filter(m -> m.getType() == AirbyteMessage.Type.STATE)
- .map(AirbyteMessage::getState)
- .collect(Collectors.toList());
- assertFalse(recordMessages.isEmpty(), "Expected the first incremental sync to produce records");
- assertFalse(stateMessages.isEmpty(), "Expected incremental sync to produce STATE messages");
-
- // when we run incremental sync again there should be no new records. Run a sync with the latest
- // state message and assert no records were emitted.
- final JsonNode latestState = Jsons.jsonNode(List.of(Iterables.getLast(stateMessages)));
- // RESET MASTER removes all binary log files that are listed in the index file,
- // leaving only a single, empty binary log file with a numeric suffix of .000001
- testdb.with("RESET MASTER;");
-
- assertEquals(6, filterRecords(runRead(configuredCatalog, latestState)).size());
- }
-
- @Test
- public void testIncrementalReadSelectedColumns() throws Exception {
- final ConfiguredAirbyteCatalog catalog = getConfiguredCatalogWithPartialColumns();
- final List allMessages = runRead(catalog);
-
- final List records = filterRecords(allMessages);
- assertFalse(records.isEmpty(), "Expected a incremental sync to produce records");
- verifyFieldNotExist(records, STREAM_NAME, "name");
- verifyFieldNotExist(records, STREAM_NAME2, "name");
- }
-
- private ConfiguredAirbyteCatalog getConfiguredCatalogWithPartialColumns() {
- // We cannot strip the primary key field as that is required for a successful CDC sync
- return new ConfiguredAirbyteCatalog().withStreams(Lists.newArrayList(
- new ConfiguredAirbyteStream()
- .withSyncMode(INCREMENTAL)
- .withDestinationSyncMode(DestinationSyncMode.APPEND)
- .withStream(CatalogHelpers.createAirbyteStream(
- STREAM_NAME, testdb.getDatabaseName(),
- Field.of("id", JsonSchemaType.NUMBER)
- /* no name field */)
- .withSourceDefinedCursor(true)
- .withSourceDefinedPrimaryKey(List.of(List.of("id")))
- .withSupportedSyncModes(
- Lists.newArrayList(SyncMode.FULL_REFRESH, INCREMENTAL))),
- new ConfiguredAirbyteStream()
- .withSyncMode(INCREMENTAL)
- .withDestinationSyncMode(DestinationSyncMode.APPEND)
- .withStream(CatalogHelpers.createAirbyteStream(
- STREAM_NAME2, testdb.getDatabaseName(),
- /* no name field */
- Field.of("id", JsonSchemaType.NUMBER))
- .withSourceDefinedCursor(true)
- .withSourceDefinedPrimaryKey(List.of(List.of("id")))
- .withSupportedSyncModes(
- Lists.newArrayList(SyncMode.FULL_REFRESH, INCREMENTAL)))));
- }
-
- private void verifyFieldNotExist(final List records, final String stream, final String field) {
- assertTrue(records.stream().noneMatch(r -> r.getStream().equals(stream) && r.getData().get(field) != null),
- "Records contain unselected columns [%s:%s]".formatted(stream, field));
- }
-
- @Test
- protected void testNullValueConversion() throws Exception {
- final List configuredAirbyteStreams =
- Lists.newArrayList(new ConfiguredAirbyteStream()
- .withSyncMode(INCREMENTAL)
- .withDestinationSyncMode(DestinationSyncMode.APPEND)
- .withStream(CatalogHelpers.createAirbyteStream(STREAM_NAME3,
- testdb.getDatabaseName(),
- Field.of("id", JsonSchemaType.NUMBER),
- Field.of("name", JsonSchemaType.STRING),
- Field.of("userid", JsonSchemaType.NUMBER))
- .withSourceDefinedCursor(true)
- .withSourceDefinedPrimaryKey(List.of(List.of("id")))
- .withSupportedSyncModes(Lists.newArrayList(FULL_REFRESH, INCREMENTAL))));
-
- final ConfiguredAirbyteCatalog configuredCatalogWithOneStream =
- new ConfiguredAirbyteCatalog().withStreams(List.of(configuredAirbyteStreams.get(0)));
-
- final List airbyteMessages = runRead(configuredCatalogWithOneStream, getState());
- final List recordMessages = filterRecords(airbyteMessages);
- final List stateMessages = airbyteMessages
- .stream()
- .filter(m -> m.getType() == AirbyteMessage.Type.STATE)
- .map(AirbyteMessage::getState)
- .collect(Collectors.toList());
- Assert.assertEquals(recordMessages.size(), 1);
- assertFalse(stateMessages.isEmpty(), "Reason");
- ObjectMapper mapper = new ObjectMapper();
-
- assertEquals(cdcFieldsOmitted(recordMessages.get(0).getData()),
- mapper.readTree("{\"id\":4, \"name\":\"voyager\", \"userid\":null}"));
-
- // when we run incremental sync again there should be no new records. Run a sync with the latest
- // state message and assert no records were emitted.
- JsonNode latestState = extractLatestState(stateMessages);
-
- testdb.getDatabase().query(c -> {
- return c.query("INSERT INTO %s.%s (id, name) VALUES (5,'deep space nine');".formatted(testdb.getDatabaseName(), STREAM_NAME3));
- }).execute();
-
- assert Objects.nonNull(latestState);
- final List secondSyncRecords = filterRecords(runRead(configuredCatalogWithOneStream, latestState));
- assertFalse(
- secondSyncRecords.isEmpty(),
- "Expected the second incremental sync to produce records.");
- assertEquals(cdcFieldsOmitted(secondSyncRecords.get(0).getData()),
- mapper.readTree("{\"id\":5, \"name\":\"deep space nine\", \"userid\":null}"));
-
- }
-
- private JsonNode cdcFieldsOmitted(final JsonNode node) {
- ObjectMapper mapper = new ObjectMapper();
- ObjectNode object = mapper.createObjectNode();
- node.fieldNames().forEachRemaining(name -> {
- if (!name.toLowerCase().startsWith("_ab_cdc_")) {
- object.put(name, node.get(name));
- }
-
- });
- return object;
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CdcMySqlSslCaCertificateSourceAcceptanceTest.java b/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CdcMySqlSslCaCertificateSourceAcceptanceTest.java
deleted file mode 100644
index 98ccf8c7f50f..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CdcMySqlSslCaCertificateSourceAcceptanceTest.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.io.airbyte.integration_tests.sources;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.google.common.collect.ImmutableMap;
-import io.airbyte.cdk.db.jdbc.JdbcUtils;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase.ContainerModifier;
-import org.apache.commons.lang3.ArrayUtils;
-
-public class CdcMySqlSslCaCertificateSourceAcceptanceTest extends CdcMySqlSourceAcceptanceTest {
-
- @Override
- protected JsonNode getConfig() {
- return testdb.integrationTestConfigBuilder()
- .withCdcReplication()
- .withSsl(ImmutableMap.builder()
- .put(JdbcUtils.MODE_KEY, "verify_ca")
- .put("ca_certificate", testdb.getCertificates().caCertificate())
- .build())
- .build();
- }
-
- @Override
- protected ContainerModifier[] getContainerModifiers() {
- return ArrayUtils.toArray(ContainerModifier.ROOT_AND_SERVER_CERTIFICATES);
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CdcMySqlSslRequiredSourceAcceptanceTest.java b/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CdcMySqlSslRequiredSourceAcceptanceTest.java
deleted file mode 100644
index f508513b72d8..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CdcMySqlSslRequiredSourceAcceptanceTest.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.io.airbyte.integration_tests.sources;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.google.common.collect.ImmutableMap;
-import io.airbyte.cdk.db.jdbc.JdbcUtils;
-import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase.ContainerModifier;
-import org.apache.commons.lang3.ArrayUtils;
-
-public class CdcMySqlSslRequiredSourceAcceptanceTest extends CdcMySqlSourceAcceptanceTest {
-
- @Override
- protected JsonNode getConfig() {
- return testdb.integrationTestConfigBuilder()
- .withCdcReplication()
- .withSsl(ImmutableMap.builder().put(JdbcUtils.MODE_KEY, "required").build())
- .build();
- }
-
- @Override
- protected void setupEnvironment(final TestDestinationEnv environment) {
- super.setupEnvironment(environment);
- testdb.with("ALTER USER %s REQUIRE SSL;", testdb.getUserName());
- }
-
- @Override
- protected ContainerModifier[] getContainerModifiers() {
- return ArrayUtils.toArray(ContainerModifier.ROOT_AND_SERVER_CERTIFICATES, ContainerModifier.CLIENT_CERTITICATE);
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CloudDeploymentMySqlSourceAcceptanceTest.java b/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CloudDeploymentMySqlSourceAcceptanceTest.java
deleted file mode 100644
index 8b607cc092f8..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CloudDeploymentMySqlSourceAcceptanceTest.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.io.airbyte.integration_tests.sources;
-
-import io.airbyte.cdk.integrations.base.ssh.SshHelpers;
-import io.airbyte.commons.features.FeatureFlags;
-import io.airbyte.commons.features.FeatureFlagsWrapper;
-import io.airbyte.commons.json.Jsons;
-import io.airbyte.commons.resources.MoreResources;
-import io.airbyte.protocol.models.v0.ConnectorSpecification;
-
-public class CloudDeploymentMySqlSourceAcceptanceTest extends MySqlSslSourceAcceptanceTest {
-
- @Override
- protected FeatureFlags featureFlags() {
- return FeatureFlagsWrapper.overridingDeploymentMode(super.featureFlags(), "CLOUD");
- }
-
- @Override
- protected ConnectorSpecification getSpec() throws Exception {
- return SshHelpers.injectSshIntoSpec(Jsons.deserialize(MoreResources.readResource("expected_cloud_spec.json"), ConnectorSpecification.class));
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CloudDeploymentMySqlSslCaCertificateSourceAcceptanceTest.java b/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CloudDeploymentMySqlSslCaCertificateSourceAcceptanceTest.java
deleted file mode 100644
index 15f0b5b5a612..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CloudDeploymentMySqlSslCaCertificateSourceAcceptanceTest.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.io.airbyte.integration_tests.sources;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.google.common.collect.ImmutableMap;
-import io.airbyte.cdk.db.jdbc.JdbcUtils;
-import io.airbyte.cdk.integrations.base.ssh.SshHelpers;
-import io.airbyte.commons.features.FeatureFlags;
-import io.airbyte.commons.features.FeatureFlagsWrapper;
-import io.airbyte.commons.json.Jsons;
-import io.airbyte.commons.resources.MoreResources;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase.ContainerModifier;
-import io.airbyte.protocol.models.v0.ConnectorSpecification;
-import org.apache.commons.lang3.ArrayUtils;
-
-public class CloudDeploymentMySqlSslCaCertificateSourceAcceptanceTest extends MySqlSourceAcceptanceTest {
-
- private static final String PASSWORD = "Passw0rd";
-
- @Override
- protected FeatureFlags featureFlags() {
- return FeatureFlagsWrapper.overridingDeploymentMode(super.featureFlags(), "CLOUD");
- }
-
- @Override
- protected ContainerModifier[] getContainerModifiers() {
- return ArrayUtils.toArray(ContainerModifier.ROOT_AND_SERVER_CERTIFICATES);
- }
-
- @Override
- protected JsonNode getConfig() {
- return testdb.integrationTestConfigBuilder()
- .withStandardReplication()
- .withSsl(ImmutableMap.builder()
- .put(JdbcUtils.MODE_KEY, "verify_ca")
- .put("ca_certificate", testdb.getCaCertificate())
- .put("client_key_password", PASSWORD)
- .build())
- .build();
- }
-
- @Override
- protected ConnectorSpecification getSpec() throws Exception {
- return SshHelpers.injectSshIntoSpec(Jsons.deserialize(MoreResources.readResource("expected_cloud_spec.json"), ConnectorSpecification.class));
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CloudDeploymentMySqlSslFullCertificateSourceAcceptanceTest.java b/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CloudDeploymentMySqlSslFullCertificateSourceAcceptanceTest.java
deleted file mode 100644
index 298276ee443f..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CloudDeploymentMySqlSslFullCertificateSourceAcceptanceTest.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.io.airbyte.integration_tests.sources;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.google.common.collect.ImmutableMap;
-import io.airbyte.cdk.db.jdbc.JdbcUtils;
-import io.airbyte.cdk.integrations.base.ssh.SshHelpers;
-import io.airbyte.commons.features.FeatureFlags;
-import io.airbyte.commons.features.FeatureFlagsWrapper;
-import io.airbyte.commons.json.Jsons;
-import io.airbyte.commons.resources.MoreResources;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase.ContainerModifier;
-import io.airbyte.protocol.models.v0.ConnectorSpecification;
-import org.apache.commons.lang3.ArrayUtils;
-
-public class CloudDeploymentMySqlSslFullCertificateSourceAcceptanceTest extends MySqlSourceAcceptanceTest {
-
- private static final String PASSWORD = "Passw0rd";
-
- @Override
- protected FeatureFlags featureFlags() {
- return FeatureFlagsWrapper.overridingDeploymentMode(super.featureFlags(), "CLOUD");
- }
-
- @Override
- protected ContainerModifier[] getContainerModifiers() {
- return ArrayUtils.toArray(ContainerModifier.ROOT_AND_SERVER_CERTIFICATES, ContainerModifier.CLIENT_CERTITICATE);
- }
-
- @Override
- protected JsonNode getConfig() {
- return testdb.integrationTestConfigBuilder()
- .withStandardReplication()
- .withSsl(ImmutableMap.builder()
- .put(JdbcUtils.MODE_KEY, "verify_ca")
- .put("ca_certificate", testdb.getCertificates().caCertificate())
- .put("client_certificate", testdb.getCertificates().clientCertificate())
- .put("client_key", testdb.getCertificates().clientKey())
- .put("client_key_password", PASSWORD)
- .build())
- .build();
- }
-
- @Override
- protected ConnectorSpecification getSpec() throws Exception {
- return SshHelpers.injectSshIntoSpec(Jsons.deserialize(MoreResources.readResource("expected_cloud_spec.json"), ConnectorSpecification.class));
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/MySqlDatatypeAccuracyTest.java b/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/MySqlDatatypeAccuracyTest.java
deleted file mode 100644
index 516d6c20a425..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/MySqlDatatypeAccuracyTest.java
+++ /dev/null
@@ -1,475 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.io.airbyte.integration_tests.sources;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.mysql.cj.MysqlType;
-import io.airbyte.cdk.db.Database;
-import io.airbyte.cdk.integrations.standardtest.source.TestDataHolder;
-import io.airbyte.integrations.source.mysql.MySQLContainerFactory;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase;
-import io.airbyte.protocol.models.JsonSchemaType;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-public class MySqlDatatypeAccuracyTest extends AbstractMySqlSourceDatatypeTest {
-
- @Override
- protected JsonNode getConfig() {
- return testdb.integrationTestConfigBuilder()
- .withoutSsl()
- .withStandardReplication()
- .build();
- }
-
- @Override
- protected Database setupDatabase() {
- final var sharedContainer = new MySQLContainerFactory().shared("mysql:8.0");
- testdb = new MySQLTestDatabase(sharedContainer)
- .withConnectionProperty("zeroDateTimeBehavior", "convertToNull")
- .initialized()
- .withoutStrictMode();
- return testdb.getDatabase();
- }
-
- private final Map> charsetsCollationsMap = Map.of(
- "UTF8", Arrays.asList("UTF8_bin", "UTF8_general_ci"),
- "UTF8MB4", Arrays.asList("UTF8MB4_general_ci", "utf8mb4_0900_ai_ci"),
- "UTF16", Arrays.asList("UTF16_bin", "UTF16_general_ci"),
- "binary", Arrays.asList("binary"),
- "CP1250", Arrays.asList("CP1250_general_ci", "cp1250_czech_cs"));
-
- @Override
- public boolean testCatalog() {
- return true;
- }
-
- @Override
- protected void initTests() {
- for (final MysqlType mst : MysqlType.values()) {
- switch (mst) {
- case DECIMAL -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.INTEGER)
- .fullSourceDataType("%s(10,0)".formatted(mst.getName()))
- .build());
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.NUMBER)
- .fullSourceDataType("%s(%d,30)".formatted(mst.getName(), mst.getPrecision()))
- .build());
- }
- case DECIMAL_UNSIGNED -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.INTEGER)
- .fullSourceDataType("DECIMAL(32,0) UNSIGNED")
- .build());
-
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.NUMBER)
- .fullSourceDataType("DECIMAL(%d,30) UNSIGNED".formatted(mst.getPrecision()))
- .build());
- }
- case TINYINT -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.BOOLEAN)
- .fullSourceDataType("%s(1)".formatted(mst.getName()))
- .build());
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.INTEGER)
- .fullSourceDataType("%s(%d)".formatted(mst.getName(), mst.getPrecision()))
- .build());
- }
- case TINYINT_UNSIGNED -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.INTEGER)
- .fullSourceDataType("TINYINT(1) UNSIGNED")
- .build());
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.INTEGER)
- .fullSourceDataType("TINYINT(%d) UNSIGNED".formatted(mst.getPrecision()))
- .build());
- }
- case BOOLEAN -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.BOOLEAN)
- .fullSourceDataType("%s".formatted(mst.getName()))
- .build());
- }
- case SMALLINT, BIGINT, MEDIUMINT, INT -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.INTEGER)
- .fullSourceDataType("%s(1)".formatted(mst.getName()))
- .build());
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.INTEGER)
- .fullSourceDataType("%s(%d)".formatted(mst.getName(), mst.getPrecision()))
- .build());
- }
- case SMALLINT_UNSIGNED -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.INTEGER)
- .fullSourceDataType("SMALLINT(1) UNSIGNED")
- .build());
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.INTEGER)
- .fullSourceDataType("SMALLINT(%d) UNSIGNED".formatted(mst.getPrecision()))
- .build());
- }
- case INT_UNSIGNED -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.INTEGER)
- .fullSourceDataType("INT(1) UNSIGNED")
- .build());
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.INTEGER)
- .fullSourceDataType("INT(%d) UNSIGNED".formatted(mst.getPrecision()))
- .build());
- }
- case FLOAT -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.NUMBER)
- .fullSourceDataType("%s(0)".formatted(mst.getName()))
- .build());
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.NUMBER)
- .fullSourceDataType("%s(24)".formatted(mst.getName()))
- .build());
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.NUMBER)
- .fullSourceDataType("%s(25)".formatted(mst.getName()))
- .build());
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.NUMBER)
- .fullSourceDataType("%s(53)".formatted(mst.getName()))
- .build());
- }
- case FLOAT_UNSIGNED -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.NUMBER)
- .fullSourceDataType("FLOAT(0) UNSIGNED")
- .build());
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.NUMBER)
- .fullSourceDataType("FLOAT(24) UNSIGNED")
- .build());
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.NUMBER)
- .fullSourceDataType("FLOAT(25) UNSIGNED")
- .build());
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.NUMBER)
- .fullSourceDataType("FLOAT(53) UNSIGNED")
- .build());
-
- }
- case DOUBLE -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.NUMBER)
- .fullSourceDataType("DOUBLE PRECISION")
- .build());
- }
- case DOUBLE_UNSIGNED -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.NUMBER)
- .fullSourceDataType("DOUBLE PRECISION UNSIGNED")
- .build());
- }
- case TIMESTAMP -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.STRING_TIMESTAMP_WITH_TIMEZONE)
- .fullSourceDataType("%s(0)".formatted(mst.getName()))
- .build());
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.STRING_TIMESTAMP_WITH_TIMEZONE)
- .fullSourceDataType("%s(6)".formatted(mst.getName()))
- .build());
- }
- case BIGINT_UNSIGNED -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.INTEGER)
- .fullSourceDataType("BIGINT(1) UNSIGNED")
- .build());
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.INTEGER)
- .fullSourceDataType("BIGINT(%d) UNSIGNED".formatted(mst.getPrecision()))
- .build());
- }
- case MEDIUMINT_UNSIGNED -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.INTEGER)
- .fullSourceDataType("MEDIUMINT(1) UNSIGNED")
- .build());
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.INTEGER)
- .fullSourceDataType("MEDIUMINT(%d) UNSIGNED".formatted(mst.getPrecision()))
- .build());
- }
- case DATE -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.STRING_DATE)
- .fullSourceDataType("%s".formatted(mst.getName()))
- .build());
- }
- case TIME -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.STRING_TIME_WITHOUT_TIMEZONE)
- .fullSourceDataType("%s".formatted(mst.getName()))
- .build());
- }
- case DATETIME -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.STRING_TIMESTAMP_WITHOUT_TIMEZONE)
- .fullSourceDataType("%s".formatted(mst.getName()))
- .build());
- }
- case YEAR -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.INTEGER)
- .fullSourceDataType("%s".formatted(mst.getName()))
- .build());
- }
- case VARCHAR -> {
- for (final Entry entry : charsetsCollationsMap.entrySet()) {
- List collations = (List) entry.getValue();
- final var airbyteType = (entry.getKey() == "binary") ? JsonSchemaType.STRING_BASE_64 : JsonSchemaType.STRING;
- for (final String collation : collations) {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(airbyteType)
- .fullSourceDataType("%s(0) CHARACTER SET %s COLLATE %s".formatted(mst.getName(), entry.getKey(), collation))
- .build());
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(airbyteType)
- .fullSourceDataType("%s(60000) CHARACTER SET %s COLLATE %s".formatted(mst.getName(), entry.getKey(), collation))
- .build());
- }
- }
- }
- case VARBINARY -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.STRING_BASE_64)
- .fullSourceDataType("%s(1)".formatted(mst.getName()))
- .build());
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.STRING_BASE_64)
- .fullSourceDataType("%s(65000)".formatted(mst.getName()))
- .build());
- }
- case BIT -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.BOOLEAN)
- .fullSourceDataType("%s(1)".formatted(mst.getName()))
- .build());
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.STRING_BASE_64)
- .fullSourceDataType("%s(64)".formatted(mst.getName()))
- .build());
-
- }
- case JSON -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.STRING)
- .fullSourceDataType("%s".formatted(mst.getName()))
- .build());
- }
- case ENUM, SET -> {
- for (final Entry entry : charsetsCollationsMap.entrySet()) {
- List collations = (List) entry.getValue();
- for (final String collation : collations) {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.STRING)
- .fullSourceDataType(
- "%s('value1', 'value2', 'value3') CHARACTER SET %s COLLATE %s".formatted(mst.getName(), entry.getKey(), collation))
- .build());
- }
- }
- }
- case TINYBLOB, MEDIUMBLOB, LONGBLOB, GEOMETRY -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.STRING_BASE_64)
- .fullSourceDataType("%s".formatted(mst.getName()))
- .build());
- }
- case TINYTEXT, MEDIUMTEXT, LONGTEXT -> {
- for (final Entry entry : charsetsCollationsMap.entrySet()) {
- final var airbyteType = (entry.getKey() == "binary") ? JsonSchemaType.STRING_BASE_64 : JsonSchemaType.STRING;
- List collations = (List) entry.getValue();
- for (final String collation : collations) {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(airbyteType)
- .fullSourceDataType("%s CHARACTER SET %s COLLATE %s".formatted(mst.getName(), entry.getKey(), collation))
- .build());
- }
- }
- }
- case BLOB -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.STRING_BASE_64)
- .fullSourceDataType("%s(0)".formatted(mst.getName()))
- .build());
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.STRING_BASE_64)
- .fullSourceDataType("%s(65000)".formatted(mst.getName()))
- .build());
- }
- case TEXT -> {
- for (final Entry entry : charsetsCollationsMap.entrySet()) {
- final var airbyteType = (entry.getKey() == "binary") ? JsonSchemaType.STRING_BASE_64 : JsonSchemaType.STRING;
- List collations = (List) entry.getValue();
- for (final String collation : collations) {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(airbyteType)
- .fullSourceDataType("%s(0) CHARACTER SET %s COLLATE %s".formatted(mst.getName(), entry.getKey(), collation))
- .build());
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(airbyteType)
- .fullSourceDataType("%s(65000) CHARACTER SET %s COLLATE %s".formatted(mst.getName(), entry.getKey(), collation))
- .build());
- }
- }
- }
- case CHAR -> {
- for (final Entry entry : charsetsCollationsMap.entrySet()) {
- final var airbyteType = (entry.getKey() == "binary") ? JsonSchemaType.STRING_BASE_64 : JsonSchemaType.STRING;
- List collations = (List) entry.getValue();
- for (final String collation : collations) {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(airbyteType)
- .fullSourceDataType("%s(0) CHARACTER SET %s COLLATE %s".formatted(mst.getName(), entry.getKey(), collation))
- .build());
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(airbyteType)
- .fullSourceDataType("%s(255) CHARACTER SET %s COLLATE %s".formatted(mst.getName(), entry.getKey(), collation))
- .build());
- }
- }
- }
- case BINARY -> {
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.STRING_BASE_64)
- .fullSourceDataType("%s(0)".formatted(mst.getName()))
- .build());
- addDataTypeTestData(
- TestDataHolder.builder()
- .sourceType(mst.name())
- .airbyteType(JsonSchemaType.STRING_BASE_64)
- .fullSourceDataType("%s(255)".formatted(mst.getName()))
- .build());
- }
- case NULL, UNKNOWN -> {
- // no-op
- }
- default -> throw new IllegalStateException("Unexpected value: " + mst);
- }
- }
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/MySqlSourceAcceptanceTest.java b/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/MySqlSourceAcceptanceTest.java
deleted file mode 100644
index 6044c66cf9cb..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/MySqlSourceAcceptanceTest.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.io.airbyte.integration_tests.sources;
-
-import static io.airbyte.protocol.models.v0.SyncMode.INCREMENTAL;
-import static org.junit.Assert.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.collect.Lists;
-import io.airbyte.cdk.integrations.base.ssh.SshHelpers;
-import io.airbyte.cdk.integrations.standardtest.source.SourceAcceptanceTest;
-import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
-import io.airbyte.commons.json.Jsons;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase.BaseImage;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase.ContainerModifier;
-import io.airbyte.protocol.models.Field;
-import io.airbyte.protocol.models.JsonSchemaType;
-import io.airbyte.protocol.models.v0.*;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Objects;
-import java.util.stream.Collectors;
-import org.apache.commons.lang3.ArrayUtils;
-import org.junit.jupiter.api.Test;
-
-public class MySqlSourceAcceptanceTest extends SourceAcceptanceTest {
-
- protected MySQLTestDatabase testdb;
-
- private static final String STREAM_NAME = "id_and_name";
- private static final String STREAM_NAME2 = "public.starships";
-
- @Override
- protected void setupEnvironment(final TestDestinationEnv environment) throws Exception {
- testdb = MySQLTestDatabase.in(BaseImage.MYSQL_8, getContainerModifiers())
- .with("CREATE TABLE id_and_name(id INTEGER, name VARCHAR(200));")
- .with("INSERT INTO id_and_name (id, name) VALUES (1,'picard'), (2, 'crusher'), (3, 'vash');")
- .with("CREATE TABLE starships(id INTEGER, name VARCHAR(200));")
- .with("INSERT INTO starships (id, name) VALUES (1,'enterprise-d'), (2, 'defiant'), (3, 'yamato');");
- }
-
- protected ContainerModifier[] getContainerModifiers() {
- return ArrayUtils.toArray();
- }
-
- @Override
- protected void tearDown(final TestDestinationEnv testEnv) {
- testdb.close();
- }
-
- @Override
- protected String getImageName() {
- return "airbyte/source-mysql:dev";
- }
-
- @Override
- protected ConnectorSpecification getSpec() throws Exception {
- return SshHelpers.getSpecAndInjectSsh();
- }
-
- @Override
- protected JsonNode getConfig() {
- return testdb.integrationTestConfigBuilder()
- .withStandardReplication()
- .withoutSsl()
- .build();
- }
-
- @Override
- protected ConfiguredAirbyteCatalog getConfiguredCatalog() {
- return new ConfiguredAirbyteCatalog().withStreams(Lists.newArrayList(
- new ConfiguredAirbyteStream()
- .withSyncMode(SyncMode.INCREMENTAL)
- .withCursorField(Lists.newArrayList("id"))
- .withDestinationSyncMode(DestinationSyncMode.APPEND)
- .withStream(CatalogHelpers.createAirbyteStream(
- STREAM_NAME, testdb.getDatabaseName(),
- Field.of("id", JsonSchemaType.NUMBER),
- Field.of("name", JsonSchemaType.STRING))
- .withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL))),
- new ConfiguredAirbyteStream()
- .withSyncMode(SyncMode.INCREMENTAL)
- .withCursorField(Lists.newArrayList("id"))
- .withDestinationSyncMode(DestinationSyncMode.APPEND)
- .withStream(CatalogHelpers.createAirbyteStream(
- STREAM_NAME2, testdb.getDatabaseName(),
- Field.of("id", JsonSchemaType.NUMBER),
- Field.of("name", JsonSchemaType.STRING))
- .withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL)))));
- }
-
- @Override
- protected JsonNode getState() {
- return Jsons.jsonNode(new HashMap<>());
- }
-
- @Test
- protected void testNullValueConversion() throws Exception {
- final String STREAM_NAME3 = "stream3";
- testdb.getDatabase().query(c -> {
- return c.query("""
- CREATE TABLE %s.%s (id INTEGER PRIMARY KEY, name VARCHAR(200), userid INTEGER DEFAULT NULL);
- """.formatted(testdb.getDatabaseName(), STREAM_NAME3));
- }).execute();
-
- testdb.getDatabase().query(c -> {
- return c.query("""
- INSERT INTO %s.%s (id, name) VALUES (4,'voyager');
- """.formatted(testdb.getDatabaseName(), STREAM_NAME3));
- }).execute();
-
- final List configuredAirbyteStreams =
- Lists.newArrayList(CatalogHelpers.createConfiguredAirbyteStream(STREAM_NAME3,
- testdb.getDatabaseName(),
- Field.of("id", JsonSchemaType.NUMBER),
- Field.of("name", JsonSchemaType.STRING),
- Field.of("userid", JsonSchemaType.NUMBER))
- .withDestinationSyncMode(DestinationSyncMode.APPEND)
- .withSyncMode(INCREMENTAL)
- .withCursorField(List.of("id")));
- final ConfiguredAirbyteCatalog configuredCatalogWithOneStream =
- new ConfiguredAirbyteCatalog().withStreams(List.of(configuredAirbyteStreams.get(0)));
-
- final List airbyteMessages = runRead(configuredCatalogWithOneStream, getState());
- final List recordMessages = filterRecords(airbyteMessages);
- final List stateMessages = airbyteMessages
- .stream()
- .filter(m -> m.getType() == AirbyteMessage.Type.STATE)
- .map(AirbyteMessage::getState)
- .collect(Collectors.toList());
- assertEquals(recordMessages.size(), 1);
- assertFalse(stateMessages.isEmpty(), "Reason");
- ObjectMapper mapper = new ObjectMapper();
-
- assertEquals(recordMessages.get(0).getData(),
- mapper.readTree("{\"id\":4, \"name\":\"voyager\", \"userid\":null}"));
-
- // when we run incremental sync again there should be no new records. Run a sync with the latest
- // state message and assert no records were emitted.
- JsonNode latestState = extractLatestState(stateMessages);
-
- testdb.getDatabase().query(c -> {
- return c.query("INSERT INTO %s.%s (id, name) VALUES (5,'deep space nine');".formatted(testdb.getDatabaseName(), STREAM_NAME3));
- }).execute();
-
- assert Objects.nonNull(latestState);
- final List secondSyncRecords = filterRecords(runRead(configuredCatalogWithOneStream, latestState));
- assertFalse(
- secondSyncRecords.isEmpty(),
- "Expected the second incremental sync to produce records.");
- assertEquals(secondSyncRecords.get(0).getData(),
- mapper.readTree("{\"id\":5, \"name\":\"deep space nine\", \"userid\":null}"));
-
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/MySqlSourceDatatypeTest.java b/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/MySqlSourceDatatypeTest.java
deleted file mode 100644
index cbfa689562dc..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/MySqlSourceDatatypeTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.io.airbyte.integration_tests.sources;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import io.airbyte.cdk.db.Database;
-import io.airbyte.integrations.source.mysql.MySQLContainerFactory;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase;
-
-public class MySqlSourceDatatypeTest extends AbstractMySqlSourceDatatypeTest {
-
- @Override
- protected JsonNode getConfig() {
- return testdb.integrationTestConfigBuilder()
- .withoutSsl()
- .withStandardReplication()
- .build();
- }
-
- @Override
- protected Database setupDatabase() {
- final var sharedContainer = new MySQLContainerFactory().shared("mysql:8.0");
- testdb = new MySQLTestDatabase(sharedContainer)
- .withConnectionProperty("zeroDateTimeBehavior", "convertToNull")
- .initialized()
- .withoutStrictMode();
- return testdb.getDatabase();
- }
-
- @Override
- public boolean testCatalog() {
- return true;
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/MySqlSslCaCertificateSourceAcceptanceTest.java b/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/MySqlSslCaCertificateSourceAcceptanceTest.java
deleted file mode 100644
index 71f36aa027f4..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/MySqlSslCaCertificateSourceAcceptanceTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.io.airbyte.integration_tests.sources;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.google.common.collect.ImmutableMap;
-import io.airbyte.cdk.db.jdbc.JdbcUtils;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase.ContainerModifier;
-import org.apache.commons.lang3.ArrayUtils;
-
-public class MySqlSslCaCertificateSourceAcceptanceTest extends MySqlSourceAcceptanceTest {
-
- private static final String PASSWORD = "Passw0rd";
-
- @Override
- protected ContainerModifier[] getContainerModifiers() {
- return ArrayUtils.toArray(ContainerModifier.ROOT_AND_SERVER_CERTIFICATES);
- }
-
- @Override
- protected JsonNode getConfig() {
- return testdb.integrationTestConfigBuilder()
- .withStandardReplication()
- .withSsl(ImmutableMap.builder()
- .put(JdbcUtils.MODE_KEY, "verify_ca")
- .put("ca_certificate", testdb.getCaCertificate())
- .put("client_key_password", PASSWORD)
- .build())
- .build();
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/MySqlSslFullCertificateSourceAcceptanceTest.java b/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/MySqlSslFullCertificateSourceAcceptanceTest.java
deleted file mode 100644
index d9f325d9db31..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/MySqlSslFullCertificateSourceAcceptanceTest.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.io.airbyte.integration_tests.sources;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.google.common.collect.ImmutableMap;
-import io.airbyte.cdk.db.jdbc.JdbcUtils;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase.ContainerModifier;
-import org.apache.commons.lang3.ArrayUtils;
-
-public class MySqlSslFullCertificateSourceAcceptanceTest extends MySqlSourceAcceptanceTest {
-
- private static final String PASSWORD = "Passw0rd";
-
- @Override
- protected ContainerModifier[] getContainerModifiers() {
- return ArrayUtils.toArray(ContainerModifier.ROOT_AND_SERVER_CERTIFICATES, ContainerModifier.CLIENT_CERTITICATE);
- }
-
- @Override
- protected JsonNode getConfig() {
- return testdb.integrationTestConfigBuilder()
- .withStandardReplication()
- .withSsl(ImmutableMap.builder()
- .put(JdbcUtils.MODE_KEY, "verify_ca")
- .put("ca_certificate", testdb.getCertificates().caCertificate())
- .put("client_certificate", testdb.getCertificates().clientCertificate())
- .put("client_key", testdb.getCertificates().clientKey())
- .put("client_key_password", PASSWORD)
- .build())
- .build();
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/MySqlSslSourceAcceptanceTest.java b/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/MySqlSslSourceAcceptanceTest.java
deleted file mode 100644
index 5f46e43808e4..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/MySqlSslSourceAcceptanceTest.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.io.airbyte.integration_tests.sources;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.google.common.collect.ImmutableMap;
-import io.airbyte.cdk.db.jdbc.JdbcUtils;
-
-public class MySqlSslSourceAcceptanceTest extends MySqlSourceAcceptanceTest {
-
- @Override
- protected JsonNode getConfig() {
- return testdb.integrationTestConfigBuilder()
- .withStandardReplication()
- .withSsl(ImmutableMap.builder().put(JdbcUtils.MODE_KEY, "required").build())
- .build();
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/SshKeyMySqlSourceAcceptanceTest.java b/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/SshKeyMySqlSourceAcceptanceTest.java
deleted file mode 100644
index 7d5f060f34c2..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/SshKeyMySqlSourceAcceptanceTest.java
+++ /dev/null
@@ -1,16 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.io.airbyte.integration_tests.sources;
-
-import java.nio.file.Path;
-
-public class SshKeyMySqlSourceAcceptanceTest extends AbstractSshMySqlSourceAcceptanceTest {
-
- @Override
- public Path getConfigFilePath() {
- return Path.of("secrets/ssh-key-repl-config.json");
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/SshPasswordMySqlSourceAcceptanceTest.java b/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/SshPasswordMySqlSourceAcceptanceTest.java
deleted file mode 100644
index 998e304d7145..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/SshPasswordMySqlSourceAcceptanceTest.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.integrations.io.airbyte.integration_tests.sources;
-
-import static org.junit.jupiter.api.Assertions.assertThrows;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import io.airbyte.cdk.integrations.base.Source;
-import io.airbyte.cdk.integrations.base.ssh.SshBastionContainer;
-import io.airbyte.cdk.integrations.base.ssh.SshTunnel;
-import io.airbyte.commons.exceptions.ConfigErrorException;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase.BaseImage;
-import io.airbyte.integrations.source.mysql.MySQLTestDatabase.ContainerModifier;
-import io.airbyte.integrations.source.mysql.MySqlSource;
-import java.nio.file.Path;
-import org.junit.jupiter.api.Test;
-
-public class SshPasswordMySqlSourceAcceptanceTest extends AbstractSshMySqlSourceAcceptanceTest {
-
- @Override
- public Path getConfigFilePath() {
- return Path.of("secrets/ssh-pwd-repl-config.json");
- }
-
- @Test
- public void sshTimeoutExceptionMarkAsConfigErrorTest() throws Exception {
- try (final var testdb = MySQLTestDatabase.in(BaseImage.MYSQL_8, ContainerModifier.NETWORK)) {
- final SshBastionContainer bastion = new SshBastionContainer();
- bastion.initAndStartBastion(testdb.getContainer().getNetwork());
- final var config = testdb.integrationTestConfigBuilder()
- .withoutSsl()
- .with("tunnel_method", bastion.getTunnelMethod(SshTunnel.TunnelMethod.SSH_PASSWORD_AUTH, true))
- .build();
- bastion.stopAndClose();
-
- final Source sshWrappedSource = MySqlSource.sshWrappedSource(new MySqlSource());
- final Exception exception = assertThrows(ConfigErrorException.class, () -> sshWrappedSource.discover(config));
-
- final String expectedMessage =
- "Timed out while opening a SSH Tunnel. Please double check the given SSH configurations and try again.";
- final String actualMessage = exception.getMessage();
- assertTrue(actualMessage.contains(expectedMessage));
- }
- }
-
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/resources/dummy_config.json b/airbyte-integrations/connectors/source-mysql/src/test-integration/resources/dummy_config.json
deleted file mode 100644
index e17733f16b23..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/test-integration/resources/dummy_config.json
+++ /dev/null
@@ -1,7 +0,0 @@
-{
- "host": "default",
- "port": 5555,
- "database": "default",
- "username": "default",
- "replication_method": { "method": "STANDARD" }
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/resources/expected_cloud_spec.json b/airbyte-integrations/connectors/source-mysql/src/test-integration/resources/expected_cloud_spec.json
deleted file mode 100644
index b76358180e65..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/test-integration/resources/expected_cloud_spec.json
+++ /dev/null
@@ -1,343 +0,0 @@
-{
- "documentationUrl": "https://docs.airbyte.com/integrations/sources/mysql",
- "connectionSpecification": {
- "$schema": "http://json-schema.org/draft-07/schema#",
- "title": "MySql Source Spec",
- "type": "object",
- "required": ["host", "port", "database", "username", "replication_method"],
- "properties": {
- "host": {
- "description": "The host name of the database.",
- "title": "Host",
- "type": "string",
- "order": 0
- },
- "port": {
- "description": "The port to connect to.",
- "title": "Port",
- "type": "integer",
- "minimum": 0,
- "maximum": 65536,
- "default": 3306,
- "examples": ["3306"],
- "order": 1
- },
- "database": {
- "description": "The database name.",
- "title": "Database",
- "type": "string",
- "order": 2
- },
- "username": {
- "description": "The username which is used to access the database.",
- "title": "Username",
- "type": "string",
- "order": 3
- },
- "password": {
- "description": "The password associated with the username.",
- "title": "Password",
- "type": "string",
- "airbyte_secret": true,
- "order": 4,
- "always_show": true
- },
- "jdbc_url_params": {
- "description": "Additional properties to pass to the JDBC URL string when connecting to the database formatted as 'key=value' pairs separated by the symbol '&'. (example: key1=value1&key2=value2&key3=value3). For more information read about JDBC URL parameters.",
- "title": "JDBC URL Parameters (Advanced)",
- "type": "string",
- "order": 5
- },
- "ssl_mode": {
- "title": "SSL modes",
- "description": "SSL connection modes. Read more in the docs.",
- "type": "object",
- "order": 7,
- "oneOf": [
- {
- "title": "preferred",
- "description": "Automatically attempt SSL connection. If the MySQL server does not support SSL, continue with a regular connection.",
- "required": ["mode"],
- "properties": {
- "mode": { "type": "string", "const": "preferred", "order": 0 }
- }
- },
- {
- "title": "required",
- "description": "Always connect with SSL. If the MySQL server doesn’t support SSL, the connection will not be established. Certificate Authority (CA) and Hostname are not verified.",
- "required": ["mode"],
- "properties": {
- "mode": { "type": "string", "const": "required", "order": 0 }
- }
- },
- {
- "title": "Verify CA",
- "description": "Always connect with SSL. Verifies CA, but allows connection even if Hostname does not match.",
- "required": ["mode", "ca_certificate"],
- "properties": {
- "mode": { "type": "string", "const": "verify_ca", "order": 0 },
- "ca_certificate": {
- "type": "string",
- "title": "CA certificate",
- "description": "CA certificate",
- "airbyte_secret": true,
- "multiline": true,
- "order": 1
- },
- "client_certificate": {
- "type": "string",
- "title": "Client certificate",
- "description": "Client certificate (this is not a required field, but if you want to use it, you will need to add the Client key as well)",
- "airbyte_secret": true,
- "multiline": true,
- "order": 2,
- "always_show": true
- },
- "client_key": {
- "type": "string",
- "title": "Client key",
- "description": "Client key (this is not a required field, but if you want to use it, you will need to add the Client certificate as well)",
- "airbyte_secret": true,
- "multiline": true,
- "order": 3,
- "always_show": true
- },
- "client_key_password": {
- "type": "string",
- "title": "Client key password",
- "description": "Password for keystorage. This field is optional. If you do not add it - the password will be generated automatically.",
- "airbyte_secret": true,
- "order": 4
- }
- }
- },
- {
- "title": "Verify Identity",
- "description": "Always connect with SSL. Verify both CA and Hostname.",
- "required": ["mode", "ca_certificate"],
- "properties": {
- "mode": {
- "type": "string",
- "const": "verify_identity",
- "order": 0
- },
- "ca_certificate": {
- "type": "string",
- "title": "CA certificate",
- "description": "CA certificate",
- "airbyte_secret": true,
- "multiline": true,
- "order": 1
- },
- "client_certificate": {
- "type": "string",
- "title": "Client certificate",
- "description": "Client certificate (this is not a required field, but if you want to use it, you will need to add the Client key as well)",
- "airbyte_secret": true,
- "multiline": true,
- "order": 2,
- "always_show": true
- },
- "client_key": {
- "type": "string",
- "title": "Client key",
- "description": "Client key (this is not a required field, but if you want to use it, you will need to add the Client certificate as well)",
- "airbyte_secret": true,
- "multiline": true,
- "order": 3,
- "always_show": true
- },
- "client_key_password": {
- "type": "string",
- "title": "Client key password",
- "description": "Password for keystorage. This field is optional. If you do not add it - the password will be generated automatically.",
- "airbyte_secret": true,
- "order": 4
- }
- }
- }
- ],
- "default": "required"
- },
- "replication_method": {
- "type": "object",
- "title": "Update Method",
- "description": "Configures how data is extracted from the database.",
- "order": 8,
- "default": "CDC",
- "display_type": "radio",
- "oneOf": [
- {
- "title": "Read Changes using Binary Log (CDC)",
- "description": "Recommended - Incrementally reads new inserts, updates, and deletes using the MySQL binary log. This must be enabled on your database.",
- "required": ["method"],
- "properties": {
- "method": { "type": "string", "const": "CDC", "order": 0 },
- "initial_waiting_seconds": {
- "type": "integer",
- "title": "Initial Waiting Time in Seconds (Advanced)",
- "description": "The amount of time the connector will wait when it launches to determine if there is new data to sync or not. Defaults to 300 seconds. Valid range: 120 seconds to 1200 seconds. Read about initial waiting time.",
- "default": 300,
- "min": 120,
- "max": 1200,
- "order": 1,
- "always_show": true
- },
- "server_time_zone": {
- "type": "string",
- "title": "Configured server timezone for the MySQL source (Advanced)",
- "description": "Enter the configured MySQL server timezone. This should only be done if the configured timezone in your MySQL instance does not conform to IANNA standard.",
- "order": 2,
- "always_show": true
- },
- "invalid_cdc_cursor_position_behavior": {
- "type": "string",
- "title": "Invalid CDC position behavior (Advanced)",
- "description": "Determines whether Airbyte should fail or re-sync data in case of an stale/invalid cursor value into the WAL. If 'Fail sync' is chosen, a user will have to manually reset the connection before being able to continue syncing data. If 'Re-sync data' is chosen, Airbyte will automatically trigger a refresh but could lead to higher cloud costs and data loss.",
- "enum": ["Fail sync", "Re-sync data"],
- "default": "Fail sync",
- "order": 3,
- "always_show": true
- },
- "initial_load_timeout_hours": {
- "type": "integer",
- "title": "Initial Load Timeout in Hours (Advanced)",
- "description": "The amount of time an initial load is allowed to continue for before catching up on CDC logs.",
- "default": 8,
- "min": 4,
- "max": 24,
- "order": 4,
- "always_show": true
- }
- }
- },
- {
- "title": "Scan Changes with User Defined Cursor",
- "description": "Incrementally detects new inserts and updates using the cursor column chosen when configuring a connection (e.g. created_at, updated_at).",
- "required": ["method"],
- "properties": {
- "method": { "type": "string", "const": "STANDARD", "order": 0 }
- }
- }
- ]
- },
- "tunnel_method": {
- "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
- }
- }
- }
- ]
- }
- }
- },
- "supportsNormalization": false,
- "supportsDBT": false,
- "supported_destination_sync_modes": []
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/resources/expected_oss_spec.json b/airbyte-integrations/connectors/source-mysql/src/test-integration/resources/expected_oss_spec.json
deleted file mode 100644
index d45898990ba5..000000000000
--- a/airbyte-integrations/connectors/source-mysql/src/test-integration/resources/expected_oss_spec.json
+++ /dev/null
@@ -1,367 +0,0 @@
-{
- "documentationUrl": "https://docs.airbyte.com/integrations/sources/mysql",
- "connectionSpecification": {
- "$schema": "http://json-schema.org/draft-07/schema#",
- "title": "MySql Source Spec",
- "type": "object",
- "required": ["host", "port", "database", "username", "replication_method"],
- "properties": {
- "host": {
- "description": "The host name of the database.",
- "title": "Host",
- "type": "string",
- "order": 0
- },
- "port": {
- "description": "The port to connect to.",
- "title": "Port",
- "type": "integer",
- "minimum": 0,
- "maximum": 65536,
- "default": 3306,
- "examples": ["3306"],
- "order": 1
- },
- "database": {
- "description": "The database name.",
- "title": "Database",
- "type": "string",
- "order": 2
- },
- "username": {
- "description": "The username which is used to access the database.",
- "title": "Username",
- "type": "string",
- "order": 3
- },
- "password": {
- "description": "The password associated with the username.",
- "title": "Password",
- "type": "string",
- "airbyte_secret": true,
- "order": 4,
- "always_show": true
- },
- "jdbc_url_params": {
- "description": "Additional properties to pass to the JDBC URL string when connecting to the database formatted as 'key=value' pairs separated by the symbol '&'. (example: key1=value1&key2=value2&key3=value3). For more information read about JDBC URL parameters.",
- "title": "JDBC URL Parameters (Advanced)",
- "type": "string",
- "order": 5
- },
- "ssl": {
- "title": "SSL Connection",
- "description": "Encrypt data using SSL.",
- "type": "boolean",
- "default": true,
- "order": 6
- },
- "ssl_mode": {
- "title": "SSL modes",
- "description": "SSL connection modes. Read more in the docs.",
- "type": "object",
- "order": 7,
- "oneOf": [
- {
- "title": "preferred",
- "description": "Automatically attempt SSL connection. If the MySQL server does not support SSL, continue with a regular connection.",
- "required": ["mode"],
- "properties": {
- "mode": {
- "type": "string",
- "const": "preferred",
- "order": 0
- }
- }
- },
- {
- "title": "required",
- "description": "Always connect with SSL. If the MySQL server doesn’t support SSL, the connection will not be established. Certificate Authority (CA) and Hostname are not verified.",
- "required": ["mode"],
- "properties": {
- "mode": {
- "type": "string",
- "const": "required",
- "order": 0
- }
- }
- },
- {
- "title": "Verify CA",
- "description": "Always connect with SSL. Verifies CA, but allows connection even if Hostname does not match.",
- "required": ["mode", "ca_certificate"],
- "properties": {
- "mode": {
- "type": "string",
- "const": "verify_ca",
- "order": 0
- },
- "ca_certificate": {
- "type": "string",
- "title": "CA certificate",
- "description": "CA certificate",
- "airbyte_secret": true,
- "multiline": true,
- "order": 1
- },
- "client_certificate": {
- "type": "string",
- "title": "Client certificate",
- "description": "Client certificate (this is not a required field, but if you want to use it, you will need to add the Client key as well)",
- "airbyte_secret": true,
- "multiline": true,
- "order": 2,
- "always_show": true
- },
- "client_key": {
- "type": "string",
- "title": "Client key",
- "description": "Client key (this is not a required field, but if you want to use it, you will need to add the Client certificate as well)",
- "airbyte_secret": true,
- "multiline": true,
- "order": 3,
- "always_show": true
- },
- "client_key_password": {
- "type": "string",
- "title": "Client key password",
- "description": "Password for keystorage. This field is optional. If you do not add it - the password will be generated automatically.",
- "airbyte_secret": true,
- "order": 4
- }
- }
- },
- {
- "title": "Verify Identity",
- "description": "Always connect with SSL. Verify both CA and Hostname.",
- "required": ["mode", "ca_certificate"],
- "properties": {
- "mode": {
- "type": "string",
- "const": "verify_identity",
- "order": 0
- },
- "ca_certificate": {
- "type": "string",
- "title": "CA certificate",
- "description": "CA certificate",
- "airbyte_secret": true,
- "multiline": true,
- "order": 1
- },
- "client_certificate": {
- "type": "string",
- "title": "Client certificate",
- "description": "Client certificate (this is not a required field, but if you want to use it, you will need to add the Client key as well)",
- "airbyte_secret": true,
- "multiline": true,
- "order": 2,
- "always_show": true
- },
- "client_key": {
- "type": "string",
- "title": "Client key",
- "description": "Client key (this is not a required field, but if you want to use it, you will need to add the Client certificate as well)",
- "airbyte_secret": true,
- "multiline": true,
- "order": 3,
- "always_show": true
- },
- "client_key_password": {
- "type": "string",
- "title": "Client key password",
- "description": "Password for keystorage. This field is optional. If you do not add it - the password will be generated automatically.",
- "airbyte_secret": true,
- "order": 4
- }
- }
- }
- ]
- },
- "replication_method": {
- "type": "object",
- "title": "Update Method",
- "description": "Configures how data is extracted from the database.",
- "order": 8,
- "default": "CDC",
- "display_type": "radio",
- "oneOf": [
- {
- "title": "Read Changes using Binary Log (CDC)",
- "description": "Recommended - Incrementally reads new inserts, updates, and deletes using the MySQL binary log. This must be enabled on your database.",
- "required": ["method"],
- "properties": {
- "method": {
- "type": "string",
- "const": "CDC",
- "order": 0
- },
- "initial_waiting_seconds": {
- "type": "integer",
- "title": "Initial Waiting Time in Seconds (Advanced)",
- "description": "The amount of time the connector will wait when it launches to determine if there is new data to sync or not. Defaults to 300 seconds. Valid range: 120 seconds to 1200 seconds. Read about initial waiting time.",
- "default": 300,
- "min": 120,
- "max": 1200,
- "order": 1,
- "always_show": true
- },
- "server_time_zone": {
- "type": "string",
- "title": "Configured server timezone for the MySQL source (Advanced)",
- "description": "Enter the configured MySQL server timezone. This should only be done if the configured timezone in your MySQL instance does not conform to IANNA standard.",
- "order": 2,
- "always_show": true
- },
- "invalid_cdc_cursor_position_behavior": {
- "type": "string",
- "title": "Invalid CDC position behavior (Advanced)",
- "description": "Determines whether Airbyte should fail or re-sync data in case of an stale/invalid cursor value into the WAL. If 'Fail sync' is chosen, a user will have to manually reset the connection before being able to continue syncing data. If 'Re-sync data' is chosen, Airbyte will automatically trigger a refresh but could lead to higher cloud costs and data loss.",
- "enum": ["Fail sync", "Re-sync data"],
- "default": "Fail sync",
- "order": 3,
- "always_show": true
- },
- "initial_load_timeout_hours": {
- "type": "integer",
- "title": "Initial Load Timeout in Hours (Advanced)",
- "description": "The amount of time an initial load is allowed to continue for before catching up on CDC logs.",
- "default": 8,
- "min": 4,
- "max": 24,
- "order": 4,
- "always_show": true
- }
- }
- },
- {
- "title": "Scan Changes with User Defined Cursor",
- "description": "Incrementally detects new inserts and updates using the cursor column chosen when configuring a connection (e.g. created_at, updated_at).",
- "required": ["method"],
- "properties": {
- "method": {
- "type": "string",
- "const": "STANDARD",
- "order": 0
- }
- }
- }
- ]
- },
- "tunnel_method": {
- "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
- }
- }
- }
- ]
- }
- }
- },
- "supported_destination_sync_modes": []
-}
diff --git a/airbyte-integrations/connectors/source-mysql/src/test-integration/resources/test.png b/airbyte-integrations/connectors/source-mysql/src/test-integration/resources/test.png
deleted file mode 100644
index ca452bd25e3ceabaac85a0cbb061cd49e6ab727f..0000000000000000000000000000000000000000
GIT binary patch
literal 0
HcmV?d00001
literal 17018
zcmV)eK&HQmP)T!BumJ${8cq>sHnM
z&bR&l-}#PU+qMM2LS2l{3H;wTJiYwEK7RY`XFqE{^75yBiJ%lN`VyfA1Xa@4NmBMZ8T0$yTge0lVLu-}RpR_L2YGJDZ#Oy`iP!w@86}cXr8&
zn4+x({E|>CnTSOt(p{ibF(9f2`lTKATzR_G;qNp@lNU!7zwnRoc>J3W-uSa=ybVTO
z3;637_xYrRez&XzUMQy1g`wQ3Tl{w5&*sz9>r>Jcf{KR3f`#@D8JV<+Vi|N_M7eAu
zm9h|tn+TR$a9w0A=E_BUfAD$8nvKTB@Y&{g{6EGbzNa6&@ue*4crXErsMGP0zI|oU
zgkhKSA3wbK@@Gb-&weQ)HLTak!^T1?l8YuZRfMLADCKR41KI&G$WS#ARp}bTnRX&_Z&wr{d
z+}7QwHNle7Fi2aCiilu9glz(*E