source-mysql, source-mssql: parallelize test execution (#32772)
Co-authored-by: postamar <postamar@users.noreply.github.com>
This commit is contained in:
@@ -156,6 +156,7 @@ MavenLocal debugging steps:
|
||||
|
||||
| Version | Date | Pull Request | Subject |
|
||||
| :------ | :--------- | :--------------------------------------------------------- | :------------------------------------------------------------------------------------------------------------------------------------------------------------- |
|
||||
| 0.5.0 | 2023-11-22 | [\#32656](https://github.com/airbytehq/airbyte/pull/32656) | Introduce TestDatabase test fixture, refactor database source test base classes. |
|
||||
| 0.4.11 | 2023-11-14 | [\#32526](https://github.com/airbytehq/airbyte/pull/32526) | Clean up memory manager logs. |
|
||||
| 0.4.10 | 2023-11-13 | [\#32285](https://github.com/airbytehq/airbyte/pull/32285) | Fix UUID codec ordering for MongoDB connector |
|
||||
| 0.4.9 | 2023-11-13 | [\#32468](https://github.com/airbytehq/airbyte/pull/32468) | Further error grouping improvements for DV2 connectors |
|
||||
|
||||
@@ -77,6 +77,7 @@ dependencies {
|
||||
testImplementation libs.testcontainers.jdbc
|
||||
testImplementation libs.testcontainers.mysql
|
||||
testImplementation libs.testcontainers.postgresql
|
||||
testImplementation libs.testcontainers.mssqlserver
|
||||
implementation 'org.codehaus.plexus:plexus-utils:3.4.2'
|
||||
|
||||
// bouncycastle is pinned to version-match the transitive dependency from kubernetes client-java
|
||||
|
||||
@@ -1,74 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.cdk.db;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.io.IOException;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
|
||||
public class MySqlUtils {
|
||||
|
||||
@VisibleForTesting
|
||||
public static Certificate getCertificate(final MySQLContainer<?> container,
|
||||
final boolean useAllCertificates)
|
||||
throws IOException, InterruptedException {
|
||||
// add root and server certificates to config file
|
||||
container.execInContainer("sh", "-c", "sed -i '31 a ssl' /etc/my.cnf");
|
||||
container.execInContainer("sh", "-c", "sed -i '32 a ssl-ca=/var/lib/mysql/ca.pem' /etc/my.cnf");
|
||||
container.execInContainer("sh", "-c", "sed -i '33 a ssl-cert=/var/lib/mysql/server-cert.pem' /etc/my.cnf");
|
||||
container.execInContainer("sh", "-c", "sed -i '34 a ssl-key=/var/lib/mysql/server-key.pem' /etc/my.cnf");
|
||||
container.execInContainer("sh", "-c", "sed -i '35 a require_secure_transport=ON' /etc/my.cnf");
|
||||
// add client certificates to config file
|
||||
if (useAllCertificates) {
|
||||
container.execInContainer("sh", "-c", "sed -i '39 a [client]' /etc/mysql/my.cnf");
|
||||
container.execInContainer("sh", "-c", "sed -i '40 a ssl-ca=/var/lib/mysql/ca.pem' /etc/my.cnf");
|
||||
container.execInContainer("sh", "-c", "sed -i '41 a ssl-cert=/var/lib/mysql/client-cert.pem' /etc/my.cnf");
|
||||
container.execInContainer("sh", "-c", "sed -i '42 a ssl-key=/var/lib/mysql/client-key.pem' /etc/my.cnf");
|
||||
}
|
||||
// copy root certificate and client certificates
|
||||
var caCert = container.execInContainer("sh", "-c", "cat /var/lib/mysql/ca.pem").getStdout().trim();
|
||||
|
||||
if (useAllCertificates) {
|
||||
var clientKey = container.execInContainer("sh", "-c", "cat /var/lib/mysql/client-key.pem").getStdout().trim();
|
||||
var clientCert = container.execInContainer("sh", "-c", "cat /var/lib/mysql/client-cert.pem").getStdout().trim();
|
||||
return new Certificate(caCert, clientCert, clientKey);
|
||||
} else {
|
||||
return new Certificate(caCert);
|
||||
}
|
||||
}
|
||||
|
||||
public static class Certificate {
|
||||
|
||||
private final String caCertificate;
|
||||
private final String clientCertificate;
|
||||
private final String clientKey;
|
||||
|
||||
public Certificate(final String caCertificate) {
|
||||
this.caCertificate = caCertificate;
|
||||
this.clientCertificate = null;
|
||||
this.clientKey = null;
|
||||
}
|
||||
|
||||
public Certificate(final String caCertificate, final String clientCertificate, final String clientKey) {
|
||||
this.caCertificate = caCertificate;
|
||||
this.clientCertificate = clientCertificate;
|
||||
this.clientKey = clientKey;
|
||||
}
|
||||
|
||||
public String getCaCertificate() {
|
||||
return caCertificate;
|
||||
}
|
||||
|
||||
public String getClientCertificate() {
|
||||
return clientCertificate;
|
||||
}
|
||||
|
||||
public String getClientKey() {
|
||||
return clientKey;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
@@ -5,14 +5,11 @@
|
||||
package io.airbyte.cdk.db;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcDatabase;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import java.io.IOException;
|
||||
import java.sql.SQLException;
|
||||
import java.util.List;
|
||||
import org.testcontainers.containers.PostgreSQLContainer;
|
||||
|
||||
public class PostgresUtils {
|
||||
|
||||
@@ -26,74 +23,4 @@ public class PostgresUtils {
|
||||
return PgLsn.fromPgString(jsonNodes.get(0).get("pg_current_wal_lsn").asText());
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public static Certificate getCertificate(final PostgreSQLContainer<?> container) throws IOException, InterruptedException {
|
||||
container.execInContainer("su", "-c", "psql -U test -c \"CREATE USER postgres WITH PASSWORD 'postgres';\"");
|
||||
container.execInContainer("su", "-c", "psql -U test -c \"GRANT CONNECT ON DATABASE \"test\" TO postgres;\"");
|
||||
container.execInContainer("su", "-c", "psql -U test -c \"ALTER USER postgres WITH SUPERUSER;\"");
|
||||
|
||||
container.execInContainer("su", "-c", "openssl ecparam -name prime256v1 -genkey -noout -out ca.key");
|
||||
container.execInContainer("su", "-c", "openssl req -new -x509 -sha256 -key ca.key -out ca.crt -subj \"/CN=127.0.0.1\"");
|
||||
container.execInContainer("su", "-c", "openssl ecparam -name prime256v1 -genkey -noout -out server.key");
|
||||
container.execInContainer("su", "-c", "openssl req -new -sha256 -key server.key -out server.csr -subj \"/CN=localhost\"");
|
||||
container.execInContainer("su", "-c",
|
||||
"openssl x509 -req -in server.csr -CA ca.crt -CAkey ca.key -CAcreateserial -out server.crt -days 365 -sha256");
|
||||
container.execInContainer("su", "-c", "cp server.key /etc/ssl/private/");
|
||||
container.execInContainer("su", "-c", "cp server.crt /etc/ssl/private/");
|
||||
container.execInContainer("su", "-c", "cp ca.crt /etc/ssl/private/");
|
||||
container.execInContainer("su", "-c", "chmod og-rwx /etc/ssl/private/server.* /etc/ssl/private/ca.*");
|
||||
container.execInContainer("su", "-c", "chown postgres:postgres /etc/ssl/private/server.crt /etc/ssl/private/server.key /etc/ssl/private/ca.crt");
|
||||
container.execInContainer("su", "-c", "echo \"ssl = on\" >> /var/lib/postgresql/data/postgresql.conf");
|
||||
container.execInContainer("su", "-c", "echo \"ssl_cert_file = '/etc/ssl/private/server.crt'\" >> /var/lib/postgresql/data/postgresql.conf");
|
||||
container.execInContainer("su", "-c", "echo \"ssl_key_file = '/etc/ssl/private/server.key'\" >> /var/lib/postgresql/data/postgresql.conf");
|
||||
container.execInContainer("su", "-c", "echo \"ssl_ca_file = '/etc/ssl/private/ca.crt'\" >> /var/lib/postgresql/data/postgresql.conf");
|
||||
container.execInContainer("su", "-c", "mkdir root/.postgresql");
|
||||
container.execInContainer("su", "-c",
|
||||
"echo \"hostssl all all 127.0.0.1/32 cert clientcert=verify-full\" >> /var/lib/postgresql/data/pg_hba.conf");
|
||||
|
||||
final var caCert = container.execInContainer("su", "-c", "cat ca.crt").getStdout().trim();
|
||||
|
||||
container.execInContainer("su", "-c", "openssl ecparam -name prime256v1 -genkey -noout -out client.key");
|
||||
container.execInContainer("su", "-c", "openssl req -new -sha256 -key client.key -out client.csr -subj \"/CN=postgres\"");
|
||||
container.execInContainer("su", "-c",
|
||||
"openssl x509 -req -in client.csr -CA ca.crt -CAkey ca.key -CAcreateserial -out client.crt -days 365 -sha256");
|
||||
container.execInContainer("su", "-c", "cp client.crt ~/.postgresql/postgresql.crt");
|
||||
container.execInContainer("su", "-c", "cp client.key ~/.postgresql/postgresql.key");
|
||||
container.execInContainer("su", "-c", "chmod 0600 ~/.postgresql/postgresql.crt ~/.postgresql/postgresql.key");
|
||||
container.execInContainer("su", "-c", "cp ca.crt root/.postgresql/ca.crt");
|
||||
container.execInContainer("su", "-c", "chown postgres:postgres ~/.postgresql/ca.crt");
|
||||
|
||||
container.execInContainer("su", "-c", "psql -U test -c \"SELECT pg_reload_conf();\"");
|
||||
|
||||
final var clientKey = container.execInContainer("su", "-c", "cat client.key").getStdout().trim();
|
||||
final var clientCert = container.execInContainer("su", "-c", "cat client.crt").getStdout().trim();
|
||||
return new Certificate(caCert, clientCert, clientKey);
|
||||
}
|
||||
|
||||
public static class Certificate {
|
||||
|
||||
private final String caCertificate;
|
||||
private final String clientCertificate;
|
||||
private final String clientKey;
|
||||
|
||||
public Certificate(final String caCertificate, final String clientCertificate, final String clientKey) {
|
||||
this.caCertificate = caCertificate;
|
||||
this.clientCertificate = clientCertificate;
|
||||
this.clientKey = clientKey;
|
||||
}
|
||||
|
||||
public String getCaCertificate() {
|
||||
return caCertificate;
|
||||
}
|
||||
|
||||
public String getClientCertificate() {
|
||||
return clientCertificate;
|
||||
}
|
||||
|
||||
public String getClientKey() {
|
||||
return clientKey;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -11,8 +11,10 @@ import com.zaxxer.hikari.HikariConfig;
|
||||
import com.zaxxer.hikari.HikariDataSource;
|
||||
import java.io.Closeable;
|
||||
import java.time.Duration;
|
||||
import java.time.temporal.ChronoUnit;
|
||||
import java.time.temporal.TemporalUnit;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import javax.sql.DataSource;
|
||||
|
||||
/**
|
||||
@@ -188,10 +190,10 @@ public class DataSourceFactory {
|
||||
private DataSourceBuilder() {}
|
||||
|
||||
/**
|
||||
* Retrieves connectionTimeout value from connection properties in seconds, default minimum timeout
|
||||
* Retrieves connectionTimeout value from connection properties in millis, default minimum timeout
|
||||
* is 60 seconds since Hikari default of 30 seconds is not enough for acceptance tests. In the case
|
||||
* the value is 0, pass the value along as Hikari and Postgres use default max value for 0 timeout
|
||||
* value
|
||||
* value.
|
||||
*
|
||||
* NOTE: HikariCP uses milliseconds for all time values:
|
||||
* https://github.com/brettwooldridge/HikariCP#gear-configuration-knobs-baby whereas Postgres is
|
||||
@@ -203,27 +205,32 @@ public class DataSourceFactory {
|
||||
* @return DataSourceBuilder class used to create dynamic fields for DataSource
|
||||
*/
|
||||
private static long getConnectionTimeoutMs(final Map<String, String> connectionProperties, String driverClassName) {
|
||||
// TODO: the usage of CONNECT_TIMEOUT is Postgres specific, may need to extend for other databases
|
||||
if (driverClassName.equals(DatabaseDriver.POSTGRESQL.getDriverClassName())) {
|
||||
final String pgPropertyConnectTimeout = CONNECT_TIMEOUT.getName();
|
||||
// If the PGProperty.CONNECT_TIMEOUT was set by the user, then take its value, if not take the
|
||||
// default
|
||||
if (connectionProperties.containsKey(pgPropertyConnectTimeout)
|
||||
&& (Long.parseLong(connectionProperties.get(pgPropertyConnectTimeout)) >= 0)) {
|
||||
return Duration.ofSeconds(Long.parseLong(connectionProperties.get(pgPropertyConnectTimeout))).toMillis();
|
||||
} else {
|
||||
return Duration.ofSeconds(Long.parseLong(Objects.requireNonNull(CONNECT_TIMEOUT.getDefaultValue()))).toMillis();
|
||||
}
|
||||
final Optional<Duration> parsedConnectionTimeout = switch (DatabaseDriver.findByDriverClassName(driverClassName)) {
|
||||
case POSTGRESQL -> maybeParseDuration(connectionProperties.get(CONNECT_TIMEOUT.getName()), ChronoUnit.SECONDS)
|
||||
.or(() -> maybeParseDuration(CONNECT_TIMEOUT.getDefaultValue(), ChronoUnit.SECONDS));
|
||||
case MYSQL -> maybeParseDuration(connectionProperties.get("connectTimeout"), ChronoUnit.MILLIS);
|
||||
case MSSQLSERVER -> maybeParseDuration(connectionProperties.get("loginTimeout"), ChronoUnit.SECONDS);
|
||||
default -> maybeParseDuration(connectionProperties.get(CONNECT_TIMEOUT_KEY), ChronoUnit.SECONDS)
|
||||
// Enforce minimum timeout duration for unspecified data sources.
|
||||
.filter(d -> d.compareTo(CONNECT_TIMEOUT_DEFAULT) >= 0);
|
||||
};
|
||||
return parsedConnectionTimeout.orElse(CONNECT_TIMEOUT_DEFAULT).toMillis();
|
||||
}
|
||||
|
||||
private static Optional<Duration> maybeParseDuration(final String stringValue, TemporalUnit unit) {
|
||||
if (stringValue == null) {
|
||||
return Optional.empty();
|
||||
}
|
||||
final Duration connectionTimeout;
|
||||
connectionTimeout =
|
||||
connectionProperties.containsKey(CONNECT_TIMEOUT_KEY) ? Duration.ofSeconds(Long.parseLong(connectionProperties.get(CONNECT_TIMEOUT_KEY)))
|
||||
: CONNECT_TIMEOUT_DEFAULT;
|
||||
if (connectionTimeout.getSeconds() == 0) {
|
||||
return connectionTimeout.toMillis();
|
||||
} else {
|
||||
return (connectionTimeout.compareTo(CONNECT_TIMEOUT_DEFAULT) > 0 ? connectionTimeout : CONNECT_TIMEOUT_DEFAULT).toMillis();
|
||||
final long number;
|
||||
try {
|
||||
number = Long.parseLong(stringValue);
|
||||
} catch (NumberFormatException __) {
|
||||
return Optional.empty();
|
||||
}
|
||||
if (number < 0) {
|
||||
return Optional.empty();
|
||||
}
|
||||
return Optional.of(Duration.of(number, unit));
|
||||
}
|
||||
|
||||
public DataSourceBuilder withConnectionProperties(final Map<String, String> connectionProperties) {
|
||||
|
||||
@@ -21,7 +21,7 @@ import org.testcontainers.containers.JdbcDatabaseContainer;
|
||||
import org.testcontainers.containers.Network;
|
||||
import org.testcontainers.images.builder.ImageFromDockerfile;
|
||||
|
||||
public class SshBastionContainer {
|
||||
public class SshBastionContainer implements AutoCloseable {
|
||||
|
||||
private static final String SSH_USER = "sshuser";
|
||||
private static final String SSH_PASSWORD = "secret";
|
||||
@@ -36,21 +36,27 @@ public class SshBastionContainer {
|
||||
bastion.start();
|
||||
}
|
||||
|
||||
public JsonNode getTunnelMethod(final SshTunnel.TunnelMethod tunnelMethod,
|
||||
final boolean innerAddress)
|
||||
throws IOException, InterruptedException {
|
||||
final var containerAddress = innerAddress ? getInnerContainerAddress(bastion) : getOuterContainerAddress(bastion);
|
||||
return Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("tunnel_host",
|
||||
Objects.requireNonNull(containerAddress.left))
|
||||
.put("tunnel_method", tunnelMethod)
|
||||
.put("tunnel_port", containerAddress.right)
|
||||
.put("tunnel_user", SSH_USER)
|
||||
.put("tunnel_user_password", tunnelMethod.equals(SSH_PASSWORD_AUTH) ? SSH_PASSWORD : "")
|
||||
.put("ssh_key", tunnelMethod.equals(SSH_KEY_AUTH) ? bastion.execInContainer("cat", "var/bastion/id_rsa").getStdout() : "")
|
||||
.build());
|
||||
}
|
||||
|
||||
public JsonNode getTunnelConfig(final SshTunnel.TunnelMethod tunnelMethod,
|
||||
final ImmutableMap.Builder<Object, Object> builderWithSchema,
|
||||
final boolean innerAddress)
|
||||
throws IOException, InterruptedException {
|
||||
final var containerAddress = innerAddress ? getInnerContainerAddress(bastion) : getOuterContainerAddress(bastion);
|
||||
return Jsons.jsonNode(builderWithSchema
|
||||
.put("tunnel_method", Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("tunnel_host",
|
||||
Objects.requireNonNull(containerAddress.left))
|
||||
.put("tunnel_method", tunnelMethod)
|
||||
.put("tunnel_port", containerAddress.right)
|
||||
.put("tunnel_user", SSH_USER)
|
||||
.put("tunnel_user_password", tunnelMethod.equals(SSH_PASSWORD_AUTH) ? SSH_PASSWORD : "")
|
||||
.put("ssh_key", tunnelMethod.equals(SSH_KEY_AUTH) ? bastion.execInContainer("cat", "var/bastion/id_rsa").getStdout() : "")
|
||||
.build()))
|
||||
.put("tunnel_method", getTunnelMethod(tunnelMethod, innerAddress))
|
||||
.build());
|
||||
}
|
||||
|
||||
@@ -83,6 +89,11 @@ public class SshBastionContainer {
|
||||
bastion.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
stopAndClose();
|
||||
}
|
||||
|
||||
public GenericContainer getContainer() {
|
||||
return bastion;
|
||||
}
|
||||
|
||||
@@ -1 +1 @@
|
||||
version=0.4.11
|
||||
version=0.5.0
|
||||
@@ -17,6 +17,7 @@ import javax.sql.DataSource;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.testcontainers.containers.MSSQLServerContainer;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
|
||||
/**
|
||||
@@ -80,7 +81,7 @@ class DataSourceFactoryTest extends CommonFactoryTest {
|
||||
try (MySQLContainer<?> mySQLContainer = new MySQLContainer<>("mysql:8.0")) {
|
||||
mySQLContainer.start();
|
||||
final Map<String, String> connectionProperties = Map.of(
|
||||
CONNECT_TIMEOUT, "30");
|
||||
CONNECT_TIMEOUT, "5000");
|
||||
final DataSource dataSource = DataSourceFactory.create(
|
||||
mySQLContainer.getUsername(),
|
||||
mySQLContainer.getPassword(),
|
||||
@@ -89,7 +90,23 @@ class DataSourceFactoryTest extends CommonFactoryTest {
|
||||
connectionProperties);
|
||||
assertNotNull(dataSource);
|
||||
assertEquals(HikariDataSource.class, dataSource.getClass());
|
||||
assertEquals(60000, ((HikariDataSource) dataSource).getHikariConfigMXBean().getConnectionTimeout());
|
||||
assertEquals(5000, ((HikariDataSource) dataSource).getHikariConfigMXBean().getConnectionTimeout());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testCreatingMsSQLServerDataSourceWithConnectionTimeoutSetBelowDefault() {
|
||||
try (var mssqlServerContainer = new MSSQLServerContainer<>("mcr.microsoft.com/mssql/server:2019-latest").acceptLicense()) {
|
||||
mssqlServerContainer.start();
|
||||
final DataSource dataSource = DataSourceFactory.create(
|
||||
mssqlServerContainer.getUsername(),
|
||||
mssqlServerContainer.getPassword(),
|
||||
mssqlServerContainer.getDriverClassName(),
|
||||
mssqlServerContainer.getJdbcUrl(),
|
||||
Map.of("loginTimeout", "5"));
|
||||
assertNotNull(dataSource);
|
||||
assertEquals(HikariDataSource.class, dataSource.getClass());
|
||||
assertEquals(5000, ((HikariDataSource) dataSource).getHikariConfigMXBean().getConnectionTimeout());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -32,7 +32,6 @@ import io.debezium.engine.ChangeEvent;
|
||||
import io.debezium.engine.DebeziumEngine;
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
import java.util.Collections;
|
||||
import java.util.Optional;
|
||||
import java.util.OptionalInt;
|
||||
import java.util.Properties;
|
||||
@@ -57,18 +56,20 @@ public class AirbyteDebeziumHandler<T> {
|
||||
private final JsonNode config;
|
||||
private final CdcTargetPosition<T> targetPosition;
|
||||
private final boolean trackSchemaHistory;
|
||||
private final Duration firstRecordWaitTime;
|
||||
private final Duration firstRecordWaitTime, subsequentRecordWaitTime;
|
||||
private final OptionalInt queueSize;
|
||||
|
||||
public AirbyteDebeziumHandler(final JsonNode config,
|
||||
final CdcTargetPosition<T> targetPosition,
|
||||
final boolean trackSchemaHistory,
|
||||
final Duration firstRecordWaitTime,
|
||||
final Duration subsequentRecordWaitTime,
|
||||
final OptionalInt queueSize) {
|
||||
this.config = config;
|
||||
this.targetPosition = targetPosition;
|
||||
this.trackSchemaHistory = trackSchemaHistory;
|
||||
this.firstRecordWaitTime = firstRecordWaitTime;
|
||||
this.subsequentRecordWaitTime = subsequentRecordWaitTime;
|
||||
this.queueSize = queueSize;
|
||||
}
|
||||
|
||||
@@ -97,7 +98,8 @@ public class AirbyteDebeziumHandler<T> {
|
||||
targetPosition,
|
||||
tableSnapshotPublisher::hasClosed,
|
||||
new DebeziumShutdownProcedure<>(queue, tableSnapshotPublisher::close, tableSnapshotPublisher::hasClosed),
|
||||
firstRecordWaitTime);
|
||||
firstRecordWaitTime,
|
||||
subsequentRecordWaitTime);
|
||||
|
||||
return AutoCloseableIterators.concatWithEagerClose(AutoCloseableIterators
|
||||
.transform(
|
||||
@@ -108,10 +110,6 @@ public class AirbyteDebeziumHandler<T> {
|
||||
.fromIterator(MoreIterators.singletonIteratorFromSupplier(cdcStateHandler::saveStateAfterCompletionOfSnapshotOfNewStreams)));
|
||||
}
|
||||
|
||||
/**
|
||||
* In the default case here, we don't know for sure whether the Debezium Engine will produce records
|
||||
* or not. We therefore pass {@link canShortCircuitDebeziumEngine} = false.
|
||||
*/
|
||||
public AutoCloseableIterator<AirbyteMessage> getIncrementalIterators(final ConfiguredAirbyteCatalog catalog,
|
||||
final CdcSavedInfoFetcher cdcSavedInfoFetcher,
|
||||
final CdcStateHandler cdcStateHandler,
|
||||
@@ -120,32 +118,6 @@ public class AirbyteDebeziumHandler<T> {
|
||||
final DebeziumPropertiesManager.DebeziumConnectorType debeziumConnectorType,
|
||||
final Instant emittedAt,
|
||||
final boolean addDbNameToState) {
|
||||
return getIncrementalIterators(
|
||||
catalog,
|
||||
cdcSavedInfoFetcher,
|
||||
cdcStateHandler,
|
||||
cdcMetadataInjector,
|
||||
connectorProperties,
|
||||
debeziumConnectorType,
|
||||
emittedAt, addDbNameToState,
|
||||
false);
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @param canShortCircuitDebeziumEngine This argument may be set to true in cases where we already
|
||||
* know that the Debezium Engine is not going to be producing any change events. In this
|
||||
* case, this method skips provisioning a Debezium Engine altogether.
|
||||
*/
|
||||
public AutoCloseableIterator<AirbyteMessage> getIncrementalIterators(final ConfiguredAirbyteCatalog catalog,
|
||||
final CdcSavedInfoFetcher cdcSavedInfoFetcher,
|
||||
final CdcStateHandler cdcStateHandler,
|
||||
final CdcMetadataInjector cdcMetadataInjector,
|
||||
final Properties connectorProperties,
|
||||
final DebeziumPropertiesManager.DebeziumConnectorType debeziumConnectorType,
|
||||
final Instant emittedAt,
|
||||
final boolean addDbNameToState,
|
||||
final boolean canShortCircuitDebeziumEngine) {
|
||||
LOGGER.info("Using CDC: {}", true);
|
||||
LOGGER.info("Using DBZ version: {}", DebeziumEngine.class.getPackage().getImplementationVersion());
|
||||
final AirbyteFileOffsetBackingStore offsetManager = AirbyteFileOffsetBackingStore.initializeState(
|
||||
@@ -157,23 +129,18 @@ public class AirbyteDebeziumHandler<T> {
|
||||
cdcStateHandler.compressSchemaHistoryForState())
|
||||
: Optional.empty();
|
||||
|
||||
final AutoCloseableIterator<ChangeEventWithMetadata> eventIterator;
|
||||
if (!canShortCircuitDebeziumEngine) {
|
||||
final var publisher = new DebeziumRecordPublisher(
|
||||
connectorProperties, config, catalog, offsetManager, schemaHistoryManager, debeziumConnectorType);
|
||||
final var queue = new LinkedBlockingQueue<ChangeEvent<String, String>>(queueSize.orElse(QUEUE_CAPACITY));
|
||||
publisher.start(queue);
|
||||
// handle state machine around pub/sub logic.
|
||||
eventIterator = new DebeziumRecordIterator<>(
|
||||
queue,
|
||||
targetPosition,
|
||||
publisher::hasClosed,
|
||||
new DebeziumShutdownProcedure<>(queue, publisher::close, publisher::hasClosed),
|
||||
firstRecordWaitTime);
|
||||
} else {
|
||||
LOGGER.info("Short-circuiting Debezium Engine: nothing of interest in target replication stream interval.");
|
||||
eventIterator = AutoCloseableIterators.fromIterator(Collections.emptyIterator());
|
||||
}
|
||||
final var publisher = new DebeziumRecordPublisher(
|
||||
connectorProperties, config, catalog, offsetManager, schemaHistoryManager, debeziumConnectorType);
|
||||
final var queue = new LinkedBlockingQueue<ChangeEvent<String, String>>(queueSize.orElse(QUEUE_CAPACITY));
|
||||
publisher.start(queue);
|
||||
// handle state machine around pub/sub logic.
|
||||
final AutoCloseableIterator<ChangeEventWithMetadata> eventIterator = new DebeziumRecordIterator<>(
|
||||
queue,
|
||||
targetPosition,
|
||||
publisher::hasClosed,
|
||||
new DebeziumShutdownProcedure<>(queue, publisher::close, publisher::hasClosed),
|
||||
firstRecordWaitTime,
|
||||
subsequentRecordWaitTime);
|
||||
|
||||
final Duration syncCheckpointDuration =
|
||||
config.get(SYNC_CHECKPOINT_DURATION_PROPERTY) != null ? Duration.ofSeconds(config.get(SYNC_CHECKPOINT_DURATION_PROPERTY).asLong())
|
||||
|
||||
@@ -39,13 +39,11 @@ public class DebeziumRecordIterator<T> extends AbstractIterator<ChangeEventWithM
|
||||
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(DebeziumRecordIterator.class);
|
||||
|
||||
private static final Duration SUBSEQUENT_RECORD_WAIT_TIME = Duration.ofMinutes(1);
|
||||
|
||||
private final Map<Class<? extends ChangeEvent>, Field> heartbeatEventSourceField;
|
||||
private final LinkedBlockingQueue<ChangeEvent<String, String>> queue;
|
||||
private final CdcTargetPosition<T> targetPosition;
|
||||
private final Supplier<Boolean> publisherStatusSupplier;
|
||||
private final Duration firstRecordWaitTime;
|
||||
private final Duration firstRecordWaitTime, subsequentRecordWaitTime;
|
||||
private final DebeziumShutdownProcedure<ChangeEvent<String, String>> debeziumShutdownProcedure;
|
||||
|
||||
private boolean receivedFirstRecord;
|
||||
@@ -59,12 +57,14 @@ public class DebeziumRecordIterator<T> extends AbstractIterator<ChangeEventWithM
|
||||
final CdcTargetPosition<T> targetPosition,
|
||||
final Supplier<Boolean> publisherStatusSupplier,
|
||||
final DebeziumShutdownProcedure<ChangeEvent<String, String>> debeziumShutdownProcedure,
|
||||
final Duration firstRecordWaitTime) {
|
||||
final Duration firstRecordWaitTime,
|
||||
final Duration subsequentRecordWaitTime) {
|
||||
this.queue = queue;
|
||||
this.targetPosition = targetPosition;
|
||||
this.publisherStatusSupplier = publisherStatusSupplier;
|
||||
this.debeziumShutdownProcedure = debeziumShutdownProcedure;
|
||||
this.firstRecordWaitTime = firstRecordWaitTime;
|
||||
this.subsequentRecordWaitTime = subsequentRecordWaitTime;
|
||||
this.heartbeatEventSourceField = new HashMap<>(1);
|
||||
|
||||
this.receivedFirstRecord = false;
|
||||
@@ -90,7 +90,7 @@ public class DebeziumRecordIterator<T> extends AbstractIterator<ChangeEventWithM
|
||||
while (!MoreBooleans.isTruthy(publisherStatusSupplier.get()) || !queue.isEmpty()) {
|
||||
final ChangeEvent<String, String> next;
|
||||
|
||||
final Duration waitTime = receivedFirstRecord ? SUBSEQUENT_RECORD_WAIT_TIME : this.firstRecordWaitTime;
|
||||
final Duration waitTime = receivedFirstRecord ? this.subsequentRecordWaitTime : this.firstRecordWaitTime;
|
||||
try {
|
||||
next = queue.poll(waitTime.getSeconds(), TimeUnit.SECONDS);
|
||||
} catch (final InterruptedException e) {
|
||||
|
||||
@@ -10,13 +10,14 @@ import java.util.Optional;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
public class FirstRecordWaitTimeUtil {
|
||||
public class RecordWaitTimeUtil {
|
||||
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(FirstRecordWaitTimeUtil.class);
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(RecordWaitTimeUtil.class);
|
||||
|
||||
public static final Duration MIN_FIRST_RECORD_WAIT_TIME = Duration.ofMinutes(2);
|
||||
public static final Duration MAX_FIRST_RECORD_WAIT_TIME = Duration.ofMinutes(20);
|
||||
public static final Duration DEFAULT_FIRST_RECORD_WAIT_TIME = Duration.ofMinutes(5);
|
||||
public static final Duration DEFAULT_SUBSEQUENT_RECORD_WAIT_TIME = Duration.ofMinutes(1);
|
||||
|
||||
public static void checkFirstRecordWaitTime(final JsonNode config) {
|
||||
// we need to skip the check because in tests, we set initial_waiting_seconds
|
||||
@@ -59,6 +60,18 @@ public class FirstRecordWaitTimeUtil {
|
||||
return firstRecordWaitTime;
|
||||
}
|
||||
|
||||
public static Duration getSubsequentRecordWaitTime(final JsonNode config) {
|
||||
Duration subsequentRecordWaitTime = DEFAULT_SUBSEQUENT_RECORD_WAIT_TIME;
|
||||
final boolean isTest = config.has("is_test") && config.get("is_test").asBoolean();
|
||||
final Optional<Integer> firstRecordWaitSeconds = getFirstRecordWaitSeconds(config);
|
||||
if (isTest && firstRecordWaitSeconds.isPresent()) {
|
||||
// In tests, reuse the initial_waiting_seconds property to speed things up.
|
||||
subsequentRecordWaitTime = Duration.ofSeconds(firstRecordWaitSeconds.get());
|
||||
}
|
||||
LOGGER.info("Subsequent record waiting time: {} seconds", subsequentRecordWaitTime.getSeconds());
|
||||
return subsequentRecordWaitTime;
|
||||
}
|
||||
|
||||
public static Optional<Integer> getFirstRecordWaitSeconds(final JsonNode config) {
|
||||
final JsonNode replicationMethod = config.get("replication_method");
|
||||
if (replicationMethod != null && replicationMethod.has("initial_waiting_seconds")) {
|
||||
@@ -124,48 +124,6 @@ public class PostgresDebeziumStateUtil implements DebeziumStateUtil {
|
||||
}
|
||||
}
|
||||
|
||||
public boolean maybeReplicationStreamIntervalHasRecords(final JsonNode jdbcConfig,
|
||||
final String slotName,
|
||||
final String publicationName,
|
||||
final String plugin,
|
||||
final long startOffset,
|
||||
final long endOffset) {
|
||||
try (final BaseConnection pgConnection = (BaseConnection) PostgresReplicationConnection.createConnection(jdbcConfig)) {
|
||||
ChainedLogicalStreamBuilder streamBuilder = pgConnection
|
||||
.getReplicationAPI()
|
||||
.replicationStream()
|
||||
.logical()
|
||||
.withSlotName("\"" + slotName + "\"")
|
||||
.withStartPosition(LogSequenceNumber.valueOf(startOffset));
|
||||
streamBuilder = addSlotOption(publicationName, plugin, pgConnection, streamBuilder);
|
||||
|
||||
try (final PGReplicationStream stream = streamBuilder.start()) {
|
||||
LogSequenceNumber current = stream.getLastReceiveLSN();
|
||||
final LogSequenceNumber end = LogSequenceNumber.valueOf(endOffset);
|
||||
// Attempt to read from the stream.
|
||||
// This will advance the stream past any bookkeeping entries, until:
|
||||
// - either the end of the stream is reached,
|
||||
// - or a meaningful entry is read.
|
||||
// In the first case, we can update the current position and conclude that the stream contains
|
||||
// nothing of
|
||||
// interest to us between the starting position and the current position.
|
||||
final var msg = stream.readPending();
|
||||
if (msg == null) {
|
||||
current = stream.getLastReceiveLSN();
|
||||
}
|
||||
if (current.compareTo(end) >= 0) {
|
||||
// If we've reached or gone past the end of the interval which interests us,
|
||||
// then there's nothing in it that we could possibly care about.
|
||||
return false;
|
||||
}
|
||||
}
|
||||
} catch (SQLException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
// In all other cases, we can't draw any conclusions as to the contents of the stream interval.
|
||||
return true;
|
||||
}
|
||||
|
||||
private ChainedLogicalStreamBuilder addSlotOption(final String publicationName,
|
||||
final String plugin,
|
||||
final BaseConnection pgConnection,
|
||||
|
||||
@@ -36,6 +36,7 @@ public class DebeziumRecordIteratorTest {
|
||||
},
|
||||
() -> false,
|
||||
mock(DebeziumShutdownProcedure.class),
|
||||
Duration.ZERO,
|
||||
Duration.ZERO);
|
||||
final Long lsn = debeziumRecordIterator.getHeartbeatPosition(new ChangeEvent<String, String>() {
|
||||
|
||||
|
||||
@@ -1,51 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.cdk.integrations.debezium.internals;
|
||||
|
||||
import static io.airbyte.cdk.integrations.debezium.internals.FirstRecordWaitTimeUtil.MAX_FIRST_RECORD_WAIT_TIME;
|
||||
import static io.airbyte.cdk.integrations.debezium.internals.FirstRecordWaitTimeUtil.MIN_FIRST_RECORD_WAIT_TIME;
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import java.time.Duration;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
public class FirstRecordWaitTimeUtilTest {
|
||||
|
||||
@Test
|
||||
void testGetFirstRecordWaitTime() {
|
||||
final JsonNode emptyConfig = Jsons.jsonNode(Collections.emptyMap());
|
||||
assertDoesNotThrow(() -> FirstRecordWaitTimeUtil.checkFirstRecordWaitTime(emptyConfig));
|
||||
assertEquals(Optional.empty(), FirstRecordWaitTimeUtil.getFirstRecordWaitSeconds(emptyConfig));
|
||||
assertEquals(FirstRecordWaitTimeUtil.DEFAULT_FIRST_RECORD_WAIT_TIME, FirstRecordWaitTimeUtil.getFirstRecordWaitTime(emptyConfig));
|
||||
|
||||
final JsonNode normalConfig = Jsons.jsonNode(Map.of("replication_method",
|
||||
Map.of("method", "CDC", "initial_waiting_seconds", 500)));
|
||||
assertDoesNotThrow(() -> FirstRecordWaitTimeUtil.checkFirstRecordWaitTime(normalConfig));
|
||||
assertEquals(Optional.of(500), FirstRecordWaitTimeUtil.getFirstRecordWaitSeconds(normalConfig));
|
||||
assertEquals(Duration.ofSeconds(500), FirstRecordWaitTimeUtil.getFirstRecordWaitTime(normalConfig));
|
||||
|
||||
final int tooShortTimeout = (int) MIN_FIRST_RECORD_WAIT_TIME.getSeconds() - 1;
|
||||
final JsonNode tooShortConfig = Jsons.jsonNode(Map.of("replication_method",
|
||||
Map.of("method", "CDC", "initial_waiting_seconds", tooShortTimeout)));
|
||||
assertThrows(IllegalArgumentException.class, () -> FirstRecordWaitTimeUtil.checkFirstRecordWaitTime(tooShortConfig));
|
||||
assertEquals(Optional.of(tooShortTimeout), FirstRecordWaitTimeUtil.getFirstRecordWaitSeconds(tooShortConfig));
|
||||
assertEquals(MIN_FIRST_RECORD_WAIT_TIME, FirstRecordWaitTimeUtil.getFirstRecordWaitTime(tooShortConfig));
|
||||
|
||||
final int tooLongTimeout = (int) MAX_FIRST_RECORD_WAIT_TIME.getSeconds() + 1;
|
||||
final JsonNode tooLongConfig = Jsons.jsonNode(Map.of("replication_method",
|
||||
Map.of("method", "CDC", "initial_waiting_seconds", tooLongTimeout)));
|
||||
assertThrows(IllegalArgumentException.class, () -> FirstRecordWaitTimeUtil.checkFirstRecordWaitTime(tooLongConfig));
|
||||
assertEquals(Optional.of(tooLongTimeout), FirstRecordWaitTimeUtil.getFirstRecordWaitSeconds(tooLongConfig));
|
||||
assertEquals(MAX_FIRST_RECORD_WAIT_TIME, FirstRecordWaitTimeUtil.getFirstRecordWaitTime(tooLongConfig));
|
||||
}
|
||||
|
||||
}
|
||||
@@ -26,7 +26,6 @@ import java.util.Map;
|
||||
import java.util.OptionalLong;
|
||||
import java.util.Properties;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.Disabled;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
@@ -146,10 +145,9 @@ public class PostgresDebeziumStateUtilTest {
|
||||
Assertions.assertTrue(savedOffsetAfterReplicationSlotLSN);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@Disabled
|
||||
@ValueSource(strings = {"pgoutput", "wal2json"})
|
||||
public void LsnCommitTest(final String plugin) throws SQLException {
|
||||
@Test
|
||||
public void LsnCommitTest() throws SQLException {
|
||||
final String plugin = "pgoutput";
|
||||
final DockerImageName myImage = DockerImageName.parse("debezium/postgres:13-alpine").asCompatibleSubstituteFor("postgres");
|
||||
final String dbName = Strings.addRandomSuffix("db", "_", 10).toLowerCase();
|
||||
final String fullReplicationSlot = "debezium_slot" + "_" + dbName;
|
||||
@@ -200,45 +198,6 @@ public class PostgresDebeziumStateUtilTest {
|
||||
Assertions.assertEquals(targetLsn, lsnAfterCommit.asLong());
|
||||
Assertions.assertNotEquals(slotStateAtTheBeginning, slotStateAfterCommit);
|
||||
|
||||
// Now check that maybeReplicationStreamIntervalHasRecords behaves as expected.
|
||||
|
||||
final long lsnBeforeBookkeepingStatements = PostgresUtils.getLsn(database).asLong();
|
||||
|
||||
database.execute("SELECT txid_current();");
|
||||
database.execute("CHECKPOINT");
|
||||
final long lsnAfterBookkeepingStatements = PostgresUtils.getLsn(database).asLong();
|
||||
Assertions.assertNotEquals(lsnBeforeBookkeepingStatements, lsnAfterBookkeepingStatements);
|
||||
|
||||
Assertions.assertFalse(postgresDebeziumStateUtil.maybeReplicationStreamIntervalHasRecords(
|
||||
Jsons.jsonNode(databaseConfig),
|
||||
fullReplicationSlot,
|
||||
publication,
|
||||
plugin,
|
||||
lsnBeforeBookkeepingStatements,
|
||||
lsnAfterBookkeepingStatements));
|
||||
|
||||
database.execute("INSERT INTO public.test_table VALUES (3, 'baz');");
|
||||
final long lsnAfterMeaningfulStatement = PostgresUtils.getLsn(database).asLong();
|
||||
Assertions.assertNotEquals(lsnBeforeBookkeepingStatements, lsnAfterMeaningfulStatement);
|
||||
|
||||
Assertions.assertTrue(postgresDebeziumStateUtil.maybeReplicationStreamIntervalHasRecords(
|
||||
Jsons.jsonNode(databaseConfig),
|
||||
fullReplicationSlot,
|
||||
publication,
|
||||
plugin,
|
||||
lsnBeforeBookkeepingStatements,
|
||||
lsnAfterMeaningfulStatement));
|
||||
Assertions.assertTrue(postgresDebeziumStateUtil.maybeReplicationStreamIntervalHasRecords(
|
||||
Jsons.jsonNode(databaseConfig),
|
||||
fullReplicationSlot,
|
||||
publication,
|
||||
plugin,
|
||||
lsnAfterBookkeepingStatements,
|
||||
lsnAfterMeaningfulStatement));
|
||||
|
||||
final var slotStateAtTheEnd = getReplicationSlot(database, fullReplicationSlot, plugin, dbName);
|
||||
Assertions.assertEquals(slotStateAfterCommit, slotStateAtTheEnd);
|
||||
|
||||
container.stop();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,51 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.cdk.integrations.debezium.internals;
|
||||
|
||||
import static io.airbyte.cdk.integrations.debezium.internals.RecordWaitTimeUtil.MAX_FIRST_RECORD_WAIT_TIME;
|
||||
import static io.airbyte.cdk.integrations.debezium.internals.RecordWaitTimeUtil.MIN_FIRST_RECORD_WAIT_TIME;
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import java.time.Duration;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
public class RecordWaitTimeUtilTest {
|
||||
|
||||
@Test
|
||||
void testGetFirstRecordWaitTime() {
|
||||
final JsonNode emptyConfig = Jsons.jsonNode(Collections.emptyMap());
|
||||
assertDoesNotThrow(() -> RecordWaitTimeUtil.checkFirstRecordWaitTime(emptyConfig));
|
||||
assertEquals(Optional.empty(), RecordWaitTimeUtil.getFirstRecordWaitSeconds(emptyConfig));
|
||||
assertEquals(RecordWaitTimeUtil.DEFAULT_FIRST_RECORD_WAIT_TIME, RecordWaitTimeUtil.getFirstRecordWaitTime(emptyConfig));
|
||||
|
||||
final JsonNode normalConfig = Jsons.jsonNode(Map.of("replication_method",
|
||||
Map.of("method", "CDC", "initial_waiting_seconds", 500)));
|
||||
assertDoesNotThrow(() -> RecordWaitTimeUtil.checkFirstRecordWaitTime(normalConfig));
|
||||
assertEquals(Optional.of(500), RecordWaitTimeUtil.getFirstRecordWaitSeconds(normalConfig));
|
||||
assertEquals(Duration.ofSeconds(500), RecordWaitTimeUtil.getFirstRecordWaitTime(normalConfig));
|
||||
|
||||
final int tooShortTimeout = (int) MIN_FIRST_RECORD_WAIT_TIME.getSeconds() - 1;
|
||||
final JsonNode tooShortConfig = Jsons.jsonNode(Map.of("replication_method",
|
||||
Map.of("method", "CDC", "initial_waiting_seconds", tooShortTimeout)));
|
||||
assertThrows(IllegalArgumentException.class, () -> RecordWaitTimeUtil.checkFirstRecordWaitTime(tooShortConfig));
|
||||
assertEquals(Optional.of(tooShortTimeout), RecordWaitTimeUtil.getFirstRecordWaitSeconds(tooShortConfig));
|
||||
assertEquals(MIN_FIRST_RECORD_WAIT_TIME, RecordWaitTimeUtil.getFirstRecordWaitTime(tooShortConfig));
|
||||
|
||||
final int tooLongTimeout = (int) MAX_FIRST_RECORD_WAIT_TIME.getSeconds() + 1;
|
||||
final JsonNode tooLongConfig = Jsons.jsonNode(Map.of("replication_method",
|
||||
Map.of("method", "CDC", "initial_waiting_seconds", tooLongTimeout)));
|
||||
assertThrows(IllegalArgumentException.class, () -> RecordWaitTimeUtil.checkFirstRecordWaitTime(tooLongConfig));
|
||||
assertEquals(Optional.of(tooLongTimeout), RecordWaitTimeUtil.getFirstRecordWaitSeconds(tooLongConfig));
|
||||
assertEquals(MAX_FIRST_RECORD_WAIT_TIME, RecordWaitTimeUtil.getFirstRecordWaitTime(tooLongConfig));
|
||||
}
|
||||
|
||||
}
|
||||
@@ -16,72 +16,57 @@ import io.airbyte.cdk.integrations.base.IntegrationRunner;
|
||||
import io.airbyte.cdk.integrations.base.Source;
|
||||
import io.airbyte.cdk.integrations.source.jdbc.test.JdbcSourceAcceptanceTest;
|
||||
import io.airbyte.cdk.integrations.util.HostPortResolver;
|
||||
import io.airbyte.cdk.testutils.PostgreSQLContainerHelper;
|
||||
import io.airbyte.cdk.testutils.TestDatabase;
|
||||
import io.airbyte.commons.features.EnvVariableFeatureFlags;
|
||||
import io.airbyte.commons.io.IOs;
|
||||
import io.airbyte.commons.features.FeatureFlagsWrapper;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.commons.string.Strings;
|
||||
import io.airbyte.protocol.models.v0.AirbyteStateMessage.AirbyteStateType;
|
||||
import java.sql.JDBCType;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Stream;
|
||||
import org.jooq.SQLDialect;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.testcontainers.containers.PostgreSQLContainer;
|
||||
import org.testcontainers.utility.MountableFile;
|
||||
import uk.org.webcompere.systemstubs.environment.EnvironmentVariables;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStub;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStubsExtension;
|
||||
|
||||
/**
|
||||
* Runs the acceptance tests in the source-jdbc test module. We want this module to run these tests
|
||||
* itself as a sanity check. The trade off here is that this class is duplicated from the one used
|
||||
* in source-postgres.
|
||||
*/
|
||||
@ExtendWith(SystemStubsExtension.class)
|
||||
class DefaultJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest {
|
||||
class DefaultJdbcSourceAcceptanceTest
|
||||
extends JdbcSourceAcceptanceTest<DefaultJdbcSourceAcceptanceTest.PostgresTestSource, DefaultJdbcSourceAcceptanceTest.BareBonesTestDatabase> {
|
||||
|
||||
@SystemStub
|
||||
private EnvironmentVariables environmentVariables;
|
||||
|
||||
private static PostgreSQLContainer<?> PSQL_DB;
|
||||
|
||||
private JsonNode config;
|
||||
private String dbName;
|
||||
private static PostgreSQLContainer<?> PSQL_CONTAINER;
|
||||
|
||||
@BeforeAll
|
||||
static void init() {
|
||||
PSQL_DB = new PostgreSQLContainer<>("postgres:13-alpine");
|
||||
PSQL_DB.start();
|
||||
PSQL_CONTAINER = new PostgreSQLContainer<>("postgres:13-alpine");
|
||||
PSQL_CONTAINER.start();
|
||||
CREATE_TABLE_WITHOUT_CURSOR_TYPE_QUERY = "CREATE TABLE %s (%s BIT(3) NOT NULL);";
|
||||
INSERT_TABLE_WITHOUT_CURSOR_TYPE_QUERY = "INSERT INTO %s VALUES(B'101');";
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
public void setup() throws Exception {
|
||||
dbName = Strings.addRandomSuffix("db", "_", 10).toLowerCase();
|
||||
@Override
|
||||
protected JsonNode config() {
|
||||
return testdb.testConfigBuilder().build();
|
||||
}
|
||||
|
||||
config = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, PSQL_DB.getHost())
|
||||
.put(JdbcUtils.PORT_KEY, PSQL_DB.getFirstMappedPort())
|
||||
.put(JdbcUtils.DATABASE_KEY, dbName)
|
||||
.put(JdbcUtils.USERNAME_KEY, PSQL_DB.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, PSQL_DB.getPassword())
|
||||
.build());
|
||||
@Override
|
||||
protected PostgresTestSource source() {
|
||||
final var source = new PostgresTestSource();
|
||||
source.setFeatureFlags(FeatureFlagsWrapper.overridingUseStreamCapableState(new EnvVariableFeatureFlags(), true));
|
||||
return source;
|
||||
}
|
||||
|
||||
environmentVariables.set(EnvVariableFeatureFlags.USE_STREAM_CAPABLE_STATE, "true");
|
||||
|
||||
final String initScriptName = "init_" + dbName.concat(".sql");
|
||||
final String tmpFilePath = IOs.writeFileToRandomTmpDir(initScriptName, "CREATE DATABASE " + dbName + ";");
|
||||
PostgreSQLContainerHelper.runSqlScript(MountableFile.forHostPath(tmpFilePath), PSQL_DB);
|
||||
|
||||
super.setup();
|
||||
@Override
|
||||
protected BareBonesTestDatabase createTestDatabase() {
|
||||
return new BareBonesTestDatabase(PSQL_CONTAINER).initialized();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -89,16 +74,6 @@ class DefaultJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractJdbcSource<JDBCType> getJdbcSource() {
|
||||
return new PostgresTestSource();
|
||||
}
|
||||
|
||||
@Override
|
||||
public JsonNode getConfig() {
|
||||
return config;
|
||||
}
|
||||
|
||||
public JsonNode getConfigWithConnectionProperties(final PostgreSQLContainer<?> psqlDb, final String dbName, final String additionalParameters) {
|
||||
return Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, HostPortResolver.resolveHost(psqlDb))
|
||||
@@ -111,11 +86,6 @@ class DefaultJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest {
|
||||
.build());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDriverClass() {
|
||||
return PostgresTestSource.DRIVER_CLASS;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean supportsPerStream() {
|
||||
return true;
|
||||
@@ -123,10 +93,10 @@ class DefaultJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest {
|
||||
|
||||
@AfterAll
|
||||
static void cleanUp() {
|
||||
PSQL_DB.close();
|
||||
PSQL_CONTAINER.close();
|
||||
}
|
||||
|
||||
private static class PostgresTestSource extends AbstractJdbcSource<JDBCType> implements Source {
|
||||
public static class PostgresTestSource extends AbstractJdbcSource<JDBCType> implements Source {
|
||||
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(PostgresTestSource.class);
|
||||
|
||||
@@ -171,10 +141,63 @@ class DefaultJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest {
|
||||
|
||||
}
|
||||
|
||||
static protected class BareBonesTestDatabase
|
||||
extends TestDatabase<PostgreSQLContainer<?>, BareBonesTestDatabase, BareBonesTestDatabase.BareBonesConfigBuilder> {
|
||||
|
||||
public BareBonesTestDatabase(PostgreSQLContainer<?> container) {
|
||||
super(container);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Stream<Stream<String>> inContainerBootstrapCmd() {
|
||||
final var sql = Stream.of(
|
||||
String.format("CREATE DATABASE %s", getDatabaseName()),
|
||||
String.format("CREATE USER %s PASSWORD '%s'", getUserName(), getPassword()),
|
||||
String.format("GRANT ALL PRIVILEGES ON DATABASE %s TO %s", getDatabaseName(), getUserName()),
|
||||
String.format("ALTER USER %s WITH SUPERUSER", getUserName()));
|
||||
return Stream.of(Stream.concat(
|
||||
Stream.of("psql",
|
||||
"-d", getContainer().getDatabaseName(),
|
||||
"-U", getContainer().getUsername(),
|
||||
"-v", "ON_ERROR_STOP=1",
|
||||
"-a"),
|
||||
sql.flatMap(stmt -> Stream.of("-c", stmt))));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Stream<String> inContainerUndoBootstrapCmd() {
|
||||
return Stream.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DatabaseDriver getDatabaseDriver() {
|
||||
return DatabaseDriver.POSTGRESQL;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SQLDialect getSqlDialect() {
|
||||
return SQLDialect.POSTGRES;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BareBonesConfigBuilder configBuilder() {
|
||||
return new BareBonesConfigBuilder(this);
|
||||
}
|
||||
|
||||
static protected class BareBonesConfigBuilder extends TestDatabase.ConfigBuilder<BareBonesTestDatabase, BareBonesConfigBuilder> {
|
||||
|
||||
private BareBonesConfigBuilder(BareBonesTestDatabase testDatabase) {
|
||||
super(testDatabase);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
void testCustomParametersOverwriteDefaultParametersExpectException() {
|
||||
final String connectionPropertiesUrl = "ssl=false";
|
||||
final JsonNode config = getConfigWithConnectionProperties(PSQL_DB, dbName, connectionPropertiesUrl);
|
||||
final JsonNode config = getConfigWithConnectionProperties(PSQL_CONTAINER, testdb.getDatabaseName(), connectionPropertiesUrl);
|
||||
final Map<String, String> customParameters = JdbcUtils.parseJdbcParameters(config, JdbcUtils.CONNECTION_PROPERTIES_KEY, "&");
|
||||
final Map<String, String> defaultParameters = Map.of(
|
||||
"ssl", "true",
|
||||
|
||||
@@ -16,8 +16,8 @@ import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.collect.Streams;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.integrations.base.Source;
|
||||
import io.airbyte.cdk.testutils.TestDatabase;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.commons.util.AutoCloseableIterator;
|
||||
import io.airbyte.commons.util.AutoCloseableIterators;
|
||||
@@ -36,7 +36,6 @@ 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.sql.SQLException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
@@ -48,65 +47,24 @@ import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.DisplayName;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
public abstract class CdcSourceTest {
|
||||
public abstract class CdcSourceTest<S extends Source, T extends TestDatabase<?, T, ?>> {
|
||||
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(CdcSourceTest.class);
|
||||
static private final Logger LOGGER = LoggerFactory.getLogger(CdcSourceTest.class);
|
||||
|
||||
protected static final String MODELS_SCHEMA = "models_schema";
|
||||
protected static final String MODELS_STREAM_NAME = "models";
|
||||
protected static final Set<String> STREAM_NAMES = Sets
|
||||
.newHashSet(MODELS_STREAM_NAME);
|
||||
protected static final String COL_ID = "id";
|
||||
protected static final String COL_MAKE_ID = "make_id";
|
||||
protected static final String COL_MODEL = "model";
|
||||
protected static final int INITIAL_WAITING_SECONDS = 5;
|
||||
static protected final String MODELS_STREAM_NAME = "models";
|
||||
static protected final Set<String> STREAM_NAMES = Set.of(MODELS_STREAM_NAME);
|
||||
static protected final String COL_ID = "id";
|
||||
static protected final String COL_MAKE_ID = "make_id";
|
||||
static protected final String COL_MODEL = "model";
|
||||
|
||||
protected final List<JsonNode> MODEL_RECORDS_RANDOM = ImmutableList.of(
|
||||
Jsons
|
||||
.jsonNode(ImmutableMap
|
||||
.of(COL_ID + "_random", 11000, COL_MAKE_ID + "_random", 1, COL_MODEL + "_random",
|
||||
"Fiesta-random")),
|
||||
Jsons.jsonNode(ImmutableMap
|
||||
.of(COL_ID + "_random", 12000, COL_MAKE_ID + "_random", 1, COL_MODEL + "_random",
|
||||
"Focus-random")),
|
||||
Jsons
|
||||
.jsonNode(ImmutableMap
|
||||
.of(COL_ID + "_random", 13000, COL_MAKE_ID + "_random", 1, COL_MODEL + "_random",
|
||||
"Ranger-random")),
|
||||
Jsons.jsonNode(ImmutableMap
|
||||
.of(COL_ID + "_random", 14000, COL_MAKE_ID + "_random", 2, COL_MODEL + "_random",
|
||||
"GLA-random")),
|
||||
Jsons.jsonNode(ImmutableMap
|
||||
.of(COL_ID + "_random", 15000, COL_MAKE_ID + "_random", 2, COL_MODEL + "_random",
|
||||
"A 220-random")),
|
||||
Jsons
|
||||
.jsonNode(ImmutableMap
|
||||
.of(COL_ID + "_random", 16000, COL_MAKE_ID + "_random", 2, COL_MODEL + "_random",
|
||||
"E 350-random")));
|
||||
|
||||
protected static final AirbyteCatalog CATALOG = new AirbyteCatalog().withStreams(List.of(
|
||||
CatalogHelpers.createAirbyteStream(
|
||||
MODELS_STREAM_NAME,
|
||||
MODELS_SCHEMA,
|
||||
Field.of(COL_ID, JsonSchemaType.INTEGER),
|
||||
Field.of(COL_MAKE_ID, JsonSchemaType.INTEGER),
|
||||
Field.of(COL_MODEL, JsonSchemaType.STRING))
|
||||
.withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL))
|
||||
.withSourceDefinedPrimaryKey(List.of(List.of(COL_ID)))));
|
||||
protected static final ConfiguredAirbyteCatalog CONFIGURED_CATALOG = CatalogHelpers
|
||||
.toDefaultConfiguredCatalog(CATALOG);
|
||||
|
||||
// set all streams to incremental.
|
||||
static {
|
||||
CONFIGURED_CATALOG.getStreams().forEach(s -> s.setSyncMode(SyncMode.INCREMENTAL));
|
||||
}
|
||||
|
||||
protected static final List<JsonNode> MODEL_RECORDS = ImmutableList.of(
|
||||
static protected final List<JsonNode> MODEL_RECORDS = ImmutableList.of(
|
||||
Jsons.jsonNode(ImmutableMap.of(COL_ID, 11, COL_MAKE_ID, 1, COL_MODEL, "Fiesta")),
|
||||
Jsons.jsonNode(ImmutableMap.of(COL_ID, 12, COL_MAKE_ID, 1, COL_MODEL, "Focus")),
|
||||
Jsons.jsonNode(ImmutableMap.of(COL_ID, 13, COL_MAKE_ID, 1, COL_MODEL, "Ranger")),
|
||||
@@ -114,26 +72,115 @@ public abstract class CdcSourceTest {
|
||||
Jsons.jsonNode(ImmutableMap.of(COL_ID, 15, COL_MAKE_ID, 2, COL_MODEL, "A 220")),
|
||||
Jsons.jsonNode(ImmutableMap.of(COL_ID, 16, COL_MAKE_ID, 2, COL_MODEL, "E 350")));
|
||||
|
||||
protected void setup() throws SQLException {
|
||||
createAndPopulateTables();
|
||||
static protected final String RANDOM_TABLE_NAME = MODELS_STREAM_NAME + "_random";
|
||||
|
||||
static protected final List<JsonNode> MODEL_RECORDS_RANDOM = MODEL_RECORDS.stream()
|
||||
.map(r -> Jsons.jsonNode(ImmutableMap.of(
|
||||
COL_ID + "_random", r.get(COL_ID).asInt() * 1000,
|
||||
COL_MAKE_ID + "_random", r.get(COL_MAKE_ID),
|
||||
COL_MODEL + "_random", r.get(COL_MODEL).asText() + "-random")))
|
||||
.toList();
|
||||
|
||||
protected T testdb;
|
||||
|
||||
protected String createTableSqlFmt() {
|
||||
return "CREATE TABLE %s.%s(%s);";
|
||||
}
|
||||
|
||||
private void createAndPopulateTables() {
|
||||
createAndPopulateActualTable();
|
||||
createAndPopulateRandomTable();
|
||||
protected String createSchemaSqlFmt() {
|
||||
return "CREATE SCHEMA %s;";
|
||||
}
|
||||
|
||||
protected void executeQuery(final String query) {
|
||||
try {
|
||||
getDatabase().query(
|
||||
ctx -> ctx
|
||||
.execute(query));
|
||||
} catch (final SQLException e) {
|
||||
throw new RuntimeException(e);
|
||||
protected String modelsSchema() {
|
||||
return "models_schema";
|
||||
}
|
||||
|
||||
/**
|
||||
* The schema of a random table which is used as a new table in snapshot test
|
||||
*/
|
||||
protected String randomSchema() {
|
||||
return "models_schema_random";
|
||||
}
|
||||
|
||||
protected AirbyteCatalog getCatalog() {
|
||||
return new AirbyteCatalog().withStreams(List.of(
|
||||
CatalogHelpers.createAirbyteStream(
|
||||
MODELS_STREAM_NAME,
|
||||
modelsSchema(),
|
||||
Field.of(COL_ID, JsonSchemaType.INTEGER),
|
||||
Field.of(COL_MAKE_ID, JsonSchemaType.INTEGER),
|
||||
Field.of(COL_MODEL, JsonSchemaType.STRING))
|
||||
.withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL))
|
||||
.withSourceDefinedPrimaryKey(List.of(List.of(COL_ID)))));
|
||||
}
|
||||
|
||||
protected ConfiguredAirbyteCatalog getConfiguredCatalog() {
|
||||
final var configuredCatalog = CatalogHelpers.toDefaultConfiguredCatalog(getCatalog());
|
||||
configuredCatalog.getStreams().forEach(s -> s.setSyncMode(SyncMode.INCREMENTAL));
|
||||
return configuredCatalog;
|
||||
}
|
||||
|
||||
protected abstract T createTestDatabase();
|
||||
|
||||
protected abstract S source();
|
||||
|
||||
protected abstract JsonNode config();
|
||||
|
||||
protected abstract CdcTargetPosition<?> cdcLatestTargetPosition();
|
||||
|
||||
protected abstract CdcTargetPosition<?> extractPosition(final JsonNode record);
|
||||
|
||||
protected abstract void assertNullCdcMetaData(final JsonNode data);
|
||||
|
||||
protected abstract void assertCdcMetaData(final JsonNode data, final boolean deletedAtNull);
|
||||
|
||||
protected abstract void removeCDCColumns(final ObjectNode data);
|
||||
|
||||
protected abstract void addCdcMetadataColumns(final AirbyteStream stream);
|
||||
|
||||
protected abstract void addCdcDefaultCursorField(final AirbyteStream stream);
|
||||
|
||||
protected abstract void assertExpectedStateMessages(final List<AirbyteStateMessage> stateMessages);
|
||||
|
||||
@BeforeEach
|
||||
protected void setup() {
|
||||
testdb = createTestDatabase();
|
||||
|
||||
// create and populate actual table
|
||||
final var actualColumns = ImmutableMap.of(
|
||||
COL_ID, "INTEGER",
|
||||
COL_MAKE_ID, "INTEGER",
|
||||
COL_MODEL, "VARCHAR(200)");
|
||||
testdb
|
||||
.with(createSchemaSqlFmt(), modelsSchema())
|
||||
.with(createTableSqlFmt(), modelsSchema(), MODELS_STREAM_NAME, columnClause(actualColumns, Optional.of(COL_ID)));
|
||||
for (final JsonNode recordJson : MODEL_RECORDS) {
|
||||
writeModelRecord(recordJson);
|
||||
}
|
||||
|
||||
// Create and populate random table.
|
||||
// This table is not part of Airbyte sync. It is being created just to make sure the schemas not
|
||||
// being synced by Airbyte are not causing issues with our debezium logic.
|
||||
final var randomColumns = ImmutableMap.of(
|
||||
COL_ID + "_random", "INTEGER",
|
||||
COL_MAKE_ID + "_random", "INTEGER",
|
||||
COL_MODEL + "_random", "VARCHAR(200)");
|
||||
if (!randomSchema().equals(modelsSchema())) {
|
||||
testdb.with(createSchemaSqlFmt(), randomSchema());
|
||||
}
|
||||
testdb.with(createTableSqlFmt(), randomSchema(), RANDOM_TABLE_NAME, columnClause(randomColumns, Optional.of(COL_ID + "_random")));
|
||||
for (final JsonNode recordJson : MODEL_RECORDS_RANDOM) {
|
||||
writeRecords(recordJson, randomSchema(), RANDOM_TABLE_NAME,
|
||||
COL_ID + "_random", COL_MAKE_ID + "_random", COL_MODEL + "_random");
|
||||
}
|
||||
}
|
||||
|
||||
public String columnClause(final Map<String, String> columnsWithDataType, final Optional<String> primaryKey) {
|
||||
@AfterEach
|
||||
protected void tearDown() {
|
||||
testdb.close();
|
||||
}
|
||||
|
||||
protected String columnClause(final Map<String, String> columnsWithDataType, final Optional<String> primaryKey) {
|
||||
final StringBuilder columnClause = new StringBuilder();
|
||||
int i = 0;
|
||||
for (final Map.Entry<String, String> column : columnsWithDataType.entrySet()) {
|
||||
@@ -151,50 +198,8 @@ public abstract class CdcSourceTest {
|
||||
return columnClause.toString();
|
||||
}
|
||||
|
||||
public void createTable(final String schemaName, final String tableName, final String columnClause) {
|
||||
executeQuery(createTableQuery(schemaName, tableName, columnClause));
|
||||
}
|
||||
|
||||
public String createTableQuery(final String schemaName, final String tableName, final String columnClause) {
|
||||
return String.format("CREATE TABLE %s.%s(%s);", schemaName, tableName, columnClause);
|
||||
}
|
||||
|
||||
public void createSchema(final String schemaName) {
|
||||
executeQuery(createSchemaQuery(schemaName));
|
||||
}
|
||||
|
||||
public String createSchemaQuery(final String schemaName) {
|
||||
return "CREATE DATABASE " + schemaName + ";";
|
||||
}
|
||||
|
||||
private void createAndPopulateActualTable() {
|
||||
createSchema(MODELS_SCHEMA);
|
||||
createTable(MODELS_SCHEMA, MODELS_STREAM_NAME,
|
||||
columnClause(ImmutableMap.of(COL_ID, "INTEGER", COL_MAKE_ID, "INTEGER", COL_MODEL, "VARCHAR(200)"), Optional.of(COL_ID)));
|
||||
for (final JsonNode recordJson : MODEL_RECORDS) {
|
||||
writeModelRecord(recordJson);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This database and table is not part of Airbyte sync. It is being created just to make sure the
|
||||
* databases not being synced by Airbyte are not causing issues with our debezium logic
|
||||
*/
|
||||
private void createAndPopulateRandomTable() {
|
||||
if (!randomTableSchema().equals(MODELS_SCHEMA)) {
|
||||
createSchema(randomTableSchema());
|
||||
}
|
||||
createTable(randomTableSchema(), MODELS_STREAM_NAME + "_random",
|
||||
columnClause(ImmutableMap.of(COL_ID + "_random", "INTEGER", COL_MAKE_ID + "_random", "INTEGER", COL_MODEL + "_random", "VARCHAR(200)"),
|
||||
Optional.of(COL_ID + "_random")));
|
||||
for (final JsonNode recordJson : MODEL_RECORDS_RANDOM) {
|
||||
writeRecords(recordJson, randomTableSchema(), MODELS_STREAM_NAME + "_random",
|
||||
COL_ID + "_random", COL_MAKE_ID + "_random", COL_MODEL + "_random");
|
||||
}
|
||||
}
|
||||
|
||||
protected void writeModelRecord(final JsonNode recordJson) {
|
||||
writeRecords(recordJson, MODELS_SCHEMA, MODELS_STREAM_NAME, COL_ID, COL_MAKE_ID, COL_MODEL);
|
||||
writeRecords(recordJson, modelsSchema(), MODELS_STREAM_NAME, COL_ID, COL_MAKE_ID, COL_MODEL);
|
||||
}
|
||||
|
||||
protected void writeRecords(
|
||||
@@ -204,14 +209,13 @@ public abstract class CdcSourceTest {
|
||||
final String idCol,
|
||||
final String makeIdCol,
|
||||
final String modelCol) {
|
||||
executeQuery(
|
||||
String.format("INSERT INTO %s.%s (%s, %s, %s) VALUES (%s, %s, '%s');", dbName, streamName,
|
||||
idCol, makeIdCol, modelCol,
|
||||
recordJson.get(idCol).asInt(), recordJson.get(makeIdCol).asInt(),
|
||||
recordJson.get(modelCol).asText()));
|
||||
testdb.with("INSERT INTO %s.%s (%s, %s, %s) VALUES (%s, %s, '%s');", dbName, streamName,
|
||||
idCol, makeIdCol, modelCol,
|
||||
recordJson.get(idCol).asInt(), recordJson.get(makeIdCol).asInt(),
|
||||
recordJson.get(modelCol).asText());
|
||||
}
|
||||
|
||||
protected static Set<AirbyteRecordMessage> removeDuplicates(final Set<AirbyteRecordMessage> messages) {
|
||||
static protected Set<AirbyteRecordMessage> removeDuplicates(final Set<AirbyteRecordMessage> messages) {
|
||||
final Set<JsonNode> existingDataRecordsWithoutUpdated = new HashSet<>();
|
||||
final Set<AirbyteRecordMessage> output = new HashSet<>();
|
||||
|
||||
@@ -272,7 +276,7 @@ public abstract class CdcSourceTest {
|
||||
private void assertExpectedRecords(final Set<JsonNode> expectedRecords,
|
||||
final Set<AirbyteRecordMessage> actualRecords,
|
||||
final Set<String> cdcStreams) {
|
||||
assertExpectedRecords(expectedRecords, actualRecords, cdcStreams, STREAM_NAMES, MODELS_SCHEMA);
|
||||
assertExpectedRecords(expectedRecords, actualRecords, cdcStreams, STREAM_NAMES, modelsSchema());
|
||||
}
|
||||
|
||||
protected void assertExpectedRecords(final Set<JsonNode> expectedRecords,
|
||||
@@ -309,7 +313,7 @@ public abstract class CdcSourceTest {
|
||||
@DisplayName("On the first sync, produce returns records that exist in the database.")
|
||||
void testExistingData() throws Exception {
|
||||
final CdcTargetPosition targetPosition = cdcLatestTargetPosition();
|
||||
final AutoCloseableIterator<AirbyteMessage> read = getSource().read(getConfig(), CONFIGURED_CATALOG, null);
|
||||
final AutoCloseableIterator<AirbyteMessage> read = source().read(config(), getConfiguredCatalog(), null);
|
||||
final List<AirbyteMessage> actualRecords = AutoCloseableIterators.toListAndClose(read);
|
||||
|
||||
final Set<AirbyteRecordMessage> recordMessages = extractRecordMessages(actualRecords);
|
||||
@@ -332,19 +336,17 @@ public abstract class CdcSourceTest {
|
||||
@Test
|
||||
@DisplayName("When a record is deleted, produces a deletion record.")
|
||||
void testDelete() throws Exception {
|
||||
final AutoCloseableIterator<AirbyteMessage> read1 = getSource()
|
||||
.read(getConfig(), CONFIGURED_CATALOG, null);
|
||||
final AutoCloseableIterator<AirbyteMessage> read1 = source()
|
||||
.read(config(), getConfiguredCatalog(), null);
|
||||
final List<AirbyteMessage> actualRecords1 = AutoCloseableIterators.toListAndClose(read1);
|
||||
final List<AirbyteStateMessage> stateMessages1 = extractStateMessages(actualRecords1);
|
||||
assertExpectedStateMessages(stateMessages1);
|
||||
|
||||
executeQuery(String
|
||||
.format("DELETE FROM %s.%s WHERE %s = %s", MODELS_SCHEMA, MODELS_STREAM_NAME, COL_ID,
|
||||
11));
|
||||
testdb.with("DELETE FROM %s.%s WHERE %s = %s", modelsSchema(), MODELS_STREAM_NAME, COL_ID, 11);
|
||||
|
||||
final JsonNode state = Jsons.jsonNode(Collections.singletonList(stateMessages1.get(stateMessages1.size() - 1)));
|
||||
final AutoCloseableIterator<AirbyteMessage> read2 = getSource()
|
||||
.read(getConfig(), CONFIGURED_CATALOG, state);
|
||||
final AutoCloseableIterator<AirbyteMessage> read2 = source()
|
||||
.read(config(), getConfiguredCatalog(), state);
|
||||
final List<AirbyteMessage> actualRecords2 = AutoCloseableIterators.toListAndClose(read2);
|
||||
final List<AirbyteRecordMessage> recordMessages2 = new ArrayList<>(
|
||||
extractRecordMessages(actualRecords2));
|
||||
@@ -363,19 +365,18 @@ public abstract class CdcSourceTest {
|
||||
@DisplayName("When a record is updated, produces an update record.")
|
||||
void testUpdate() throws Exception {
|
||||
final String updatedModel = "Explorer";
|
||||
final AutoCloseableIterator<AirbyteMessage> read1 = getSource()
|
||||
.read(getConfig(), CONFIGURED_CATALOG, null);
|
||||
final AutoCloseableIterator<AirbyteMessage> read1 = source()
|
||||
.read(config(), getConfiguredCatalog(), null);
|
||||
final List<AirbyteMessage> actualRecords1 = AutoCloseableIterators.toListAndClose(read1);
|
||||
final List<AirbyteStateMessage> stateMessages1 = extractStateMessages(actualRecords1);
|
||||
assertExpectedStateMessages(stateMessages1);
|
||||
|
||||
executeQuery(String
|
||||
.format("UPDATE %s.%s SET %s = '%s' WHERE %s = %s", MODELS_SCHEMA, MODELS_STREAM_NAME,
|
||||
COL_MODEL, updatedModel, COL_ID, 11));
|
||||
testdb.with("UPDATE %s.%s SET %s = '%s' WHERE %s = %s", modelsSchema(), MODELS_STREAM_NAME,
|
||||
COL_MODEL, updatedModel, COL_ID, 11);
|
||||
|
||||
final JsonNode state = Jsons.jsonNode(Collections.singletonList(stateMessages1.get(stateMessages1.size() - 1)));
|
||||
final AutoCloseableIterator<AirbyteMessage> read2 = getSource()
|
||||
.read(getConfig(), CONFIGURED_CATALOG, state);
|
||||
final AutoCloseableIterator<AirbyteMessage> read2 = source()
|
||||
.read(config(), getConfiguredCatalog(), state);
|
||||
final List<AirbyteMessage> actualRecords2 = AutoCloseableIterators.toListAndClose(read2);
|
||||
final List<AirbyteRecordMessage> recordMessages2 = new ArrayList<>(
|
||||
extractRecordMessages(actualRecords2));
|
||||
@@ -402,8 +403,8 @@ public abstract class CdcSourceTest {
|
||||
writeModelRecord(record);
|
||||
}
|
||||
|
||||
final AutoCloseableIterator<AirbyteMessage> firstBatchIterator = getSource()
|
||||
.read(getConfig(), CONFIGURED_CATALOG, null);
|
||||
final AutoCloseableIterator<AirbyteMessage> firstBatchIterator = source()
|
||||
.read(config(), getConfiguredCatalog(), null);
|
||||
final List<AirbyteMessage> dataFromFirstBatch = AutoCloseableIterators
|
||||
.toListAndClose(firstBatchIterator);
|
||||
final List<AirbyteStateMessage> stateAfterFirstBatch = extractStateMessages(dataFromFirstBatch);
|
||||
@@ -422,8 +423,8 @@ public abstract class CdcSourceTest {
|
||||
}
|
||||
|
||||
final JsonNode state = Jsons.jsonNode(Collections.singletonList(stateAfterFirstBatch.get(stateAfterFirstBatch.size() - 1)));
|
||||
final AutoCloseableIterator<AirbyteMessage> secondBatchIterator = getSource()
|
||||
.read(getConfig(), CONFIGURED_CATALOG, state);
|
||||
final AutoCloseableIterator<AirbyteMessage> secondBatchIterator = source()
|
||||
.read(config(), getConfiguredCatalog(), state);
|
||||
final List<AirbyteMessage> dataFromSecondBatch = AutoCloseableIterators
|
||||
.toListAndClose(secondBatchIterator);
|
||||
|
||||
@@ -457,7 +458,7 @@ public abstract class CdcSourceTest {
|
||||
@Test
|
||||
@DisplayName("When both incremental CDC and full refresh are configured for different streams in a sync, the data is replicated as expected.")
|
||||
void testCdcAndFullRefreshInSameSync() throws Exception {
|
||||
final ConfiguredAirbyteCatalog configuredCatalog = Jsons.clone(CONFIGURED_CATALOG);
|
||||
final ConfiguredAirbyteCatalog configuredCatalog = Jsons.clone(getConfiguredCatalog());
|
||||
|
||||
final List<JsonNode> MODEL_RECORDS_2 = ImmutableList.of(
|
||||
Jsons.jsonNode(ImmutableMap.of(COL_ID, 110, COL_MAKE_ID, 1, COL_MODEL, "Fiesta-2")),
|
||||
@@ -467,18 +468,17 @@ public abstract class CdcSourceTest {
|
||||
Jsons.jsonNode(ImmutableMap.of(COL_ID, 150, COL_MAKE_ID, 2, COL_MODEL, "A 220-2")),
|
||||
Jsons.jsonNode(ImmutableMap.of(COL_ID, 160, COL_MAKE_ID, 2, COL_MODEL, "E 350-2")));
|
||||
|
||||
createTable(MODELS_SCHEMA, MODELS_STREAM_NAME + "_2",
|
||||
columnClause(ImmutableMap.of(COL_ID, "INTEGER", COL_MAKE_ID, "INTEGER", COL_MODEL, "VARCHAR(200)"), Optional.of(COL_ID)));
|
||||
final var columns = ImmutableMap.of(COL_ID, "INTEGER", COL_MAKE_ID, "INTEGER", COL_MODEL, "VARCHAR(200)");
|
||||
testdb.with(createTableSqlFmt(), modelsSchema(), MODELS_STREAM_NAME + "_2", columnClause(columns, Optional.of(COL_ID)));
|
||||
|
||||
for (final JsonNode recordJson : MODEL_RECORDS_2) {
|
||||
writeRecords(recordJson, MODELS_SCHEMA, MODELS_STREAM_NAME + "_2", COL_ID,
|
||||
COL_MAKE_ID, COL_MODEL);
|
||||
writeRecords(recordJson, modelsSchema(), MODELS_STREAM_NAME + "_2", COL_ID, COL_MAKE_ID, COL_MODEL);
|
||||
}
|
||||
|
||||
final ConfiguredAirbyteStream airbyteStream = new ConfiguredAirbyteStream()
|
||||
.withStream(CatalogHelpers.createAirbyteStream(
|
||||
MODELS_STREAM_NAME + "_2",
|
||||
MODELS_SCHEMA,
|
||||
modelsSchema(),
|
||||
Field.of(COL_ID, JsonSchemaType.INTEGER),
|
||||
Field.of(COL_MAKE_ID, JsonSchemaType.INTEGER),
|
||||
Field.of(COL_MODEL, JsonSchemaType.STRING))
|
||||
@@ -491,8 +491,8 @@ public abstract class CdcSourceTest {
|
||||
streams.add(airbyteStream);
|
||||
configuredCatalog.withStreams(streams);
|
||||
|
||||
final AutoCloseableIterator<AirbyteMessage> read1 = getSource()
|
||||
.read(getConfig(), configuredCatalog, null);
|
||||
final AutoCloseableIterator<AirbyteMessage> read1 = source()
|
||||
.read(config(), configuredCatalog, null);
|
||||
final List<AirbyteMessage> actualRecords1 = AutoCloseableIterators.toListAndClose(read1);
|
||||
|
||||
final Set<AirbyteRecordMessage> recordMessages1 = extractRecordMessages(actualRecords1);
|
||||
@@ -505,15 +505,15 @@ public abstract class CdcSourceTest {
|
||||
recordMessages1,
|
||||
Collections.singleton(MODELS_STREAM_NAME),
|
||||
names,
|
||||
MODELS_SCHEMA);
|
||||
modelsSchema());
|
||||
|
||||
final JsonNode puntoRecord = Jsons
|
||||
.jsonNode(ImmutableMap.of(COL_ID, 100, COL_MAKE_ID, 3, COL_MODEL, "Punto"));
|
||||
writeModelRecord(puntoRecord);
|
||||
|
||||
final JsonNode state = Jsons.jsonNode(Collections.singletonList(stateMessages1.get(stateMessages1.size() - 1)));
|
||||
final AutoCloseableIterator<AirbyteMessage> read2 = getSource()
|
||||
.read(getConfig(), configuredCatalog, state);
|
||||
final AutoCloseableIterator<AirbyteMessage> read2 = source()
|
||||
.read(config(), configuredCatalog, state);
|
||||
final List<AirbyteMessage> actualRecords2 = AutoCloseableIterators.toListAndClose(read2);
|
||||
|
||||
final Set<AirbyteRecordMessage> recordMessages2 = extractRecordMessages(actualRecords2);
|
||||
@@ -525,17 +525,16 @@ public abstract class CdcSourceTest {
|
||||
recordMessages2,
|
||||
Collections.singleton(MODELS_STREAM_NAME),
|
||||
names,
|
||||
MODELS_SCHEMA);
|
||||
modelsSchema());
|
||||
}
|
||||
|
||||
@Test
|
||||
@DisplayName("When no records exist, no records are returned.")
|
||||
void testNoData() throws Exception {
|
||||
|
||||
executeQuery(String.format("DELETE FROM %s.%s", MODELS_SCHEMA, MODELS_STREAM_NAME));
|
||||
testdb.with("DELETE FROM %s.%s", modelsSchema(), MODELS_STREAM_NAME);
|
||||
|
||||
final AutoCloseableIterator<AirbyteMessage> read = getSource()
|
||||
.read(getConfig(), CONFIGURED_CATALOG, null);
|
||||
final AutoCloseableIterator<AirbyteMessage> read = source().read(config(), getConfiguredCatalog(), null);
|
||||
final List<AirbyteMessage> actualRecords = AutoCloseableIterators.toListAndClose(read);
|
||||
|
||||
final Set<AirbyteRecordMessage> recordMessages = extractRecordMessages(actualRecords);
|
||||
@@ -551,14 +550,14 @@ public abstract class CdcSourceTest {
|
||||
@Test
|
||||
@DisplayName("When no changes have been made to the database since the previous sync, no records are returned.")
|
||||
void testNoDataOnSecondSync() throws Exception {
|
||||
final AutoCloseableIterator<AirbyteMessage> read1 = getSource()
|
||||
.read(getConfig(), CONFIGURED_CATALOG, null);
|
||||
final AutoCloseableIterator<AirbyteMessage> read1 = source()
|
||||
.read(config(), getConfiguredCatalog(), null);
|
||||
final List<AirbyteMessage> actualRecords1 = AutoCloseableIterators.toListAndClose(read1);
|
||||
final List<AirbyteStateMessage> stateMessagesFromFirstSync = extractStateMessages(actualRecords1);
|
||||
final JsonNode state = Jsons.jsonNode(Collections.singletonList(stateMessagesFromFirstSync.get(stateMessagesFromFirstSync.size() - 1)));
|
||||
|
||||
final AutoCloseableIterator<AirbyteMessage> read2 = getSource()
|
||||
.read(getConfig(), CONFIGURED_CATALOG, state);
|
||||
final AutoCloseableIterator<AirbyteMessage> read2 = source()
|
||||
.read(config(), getConfiguredCatalog(), state);
|
||||
final List<AirbyteMessage> actualRecords2 = AutoCloseableIterators.toListAndClose(read2);
|
||||
|
||||
final Set<AirbyteRecordMessage> recordMessages2 = extractRecordMessages(actualRecords2);
|
||||
@@ -570,14 +569,14 @@ public abstract class CdcSourceTest {
|
||||
|
||||
@Test
|
||||
void testCheck() throws Exception {
|
||||
final AirbyteConnectionStatus status = getSource().check(getConfig());
|
||||
final AirbyteConnectionStatus status = source().check(config());
|
||||
assertEquals(status.getStatus(), AirbyteConnectionStatus.Status.SUCCEEDED);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testDiscover() throws Exception {
|
||||
final AirbyteCatalog expectedCatalog = expectedCatalogForDiscover();
|
||||
final AirbyteCatalog actualCatalog = getSource().discover(getConfig());
|
||||
final AirbyteCatalog actualCatalog = source().discover(config());
|
||||
|
||||
assertEquals(
|
||||
expectedCatalog.getStreams().stream().sorted(Comparator.comparing(AirbyteStream::getName))
|
||||
@@ -588,8 +587,8 @@ public abstract class CdcSourceTest {
|
||||
|
||||
@Test
|
||||
public void newTableSnapshotTest() throws Exception {
|
||||
final AutoCloseableIterator<AirbyteMessage> firstBatchIterator = getSource()
|
||||
.read(getConfig(), CONFIGURED_CATALOG, null);
|
||||
final AutoCloseableIterator<AirbyteMessage> firstBatchIterator = source()
|
||||
.read(config(), getConfiguredCatalog(), null);
|
||||
final List<AirbyteMessage> dataFromFirstBatch = AutoCloseableIterators
|
||||
.toListAndClose(firstBatchIterator);
|
||||
final Set<AirbyteRecordMessage> recordsFromFirstBatch = extractRecordMessages(
|
||||
@@ -605,7 +604,7 @@ public abstract class CdcSourceTest {
|
||||
.map(AirbyteStreamState::getStreamDescriptor)
|
||||
.collect(Collectors.toSet());
|
||||
assertEquals(1, streamsInStateAfterFirstSyncCompletion.size());
|
||||
assertTrue(streamsInStateAfterFirstSyncCompletion.contains(new StreamDescriptor().withName(MODELS_STREAM_NAME).withNamespace(MODELS_SCHEMA)));
|
||||
assertTrue(streamsInStateAfterFirstSyncCompletion.contains(new StreamDescriptor().withName(MODELS_STREAM_NAME).withNamespace(modelsSchema())));
|
||||
assertNotNull(stateMessageEmittedAfterFirstSyncCompletion.getData());
|
||||
|
||||
assertEquals((MODEL_RECORDS.size()), recordsFromFirstBatch.size());
|
||||
@@ -616,8 +615,8 @@ public abstract class CdcSourceTest {
|
||||
final ConfiguredAirbyteCatalog newTables = CatalogHelpers
|
||||
.toDefaultConfiguredCatalog(new AirbyteCatalog().withStreams(List.of(
|
||||
CatalogHelpers.createAirbyteStream(
|
||||
MODELS_STREAM_NAME + "_random",
|
||||
randomTableSchema(),
|
||||
RANDOM_TABLE_NAME,
|
||||
randomSchema(),
|
||||
Field.of(COL_ID + "_random", JsonSchemaType.NUMBER),
|
||||
Field.of(COL_MAKE_ID + "_random", JsonSchemaType.NUMBER),
|
||||
Field.of(COL_MODEL + "_random", JsonSchemaType.STRING))
|
||||
@@ -626,7 +625,7 @@ public abstract class CdcSourceTest {
|
||||
|
||||
newTables.getStreams().forEach(s -> s.setSyncMode(SyncMode.INCREMENTAL));
|
||||
final List<ConfiguredAirbyteStream> combinedStreams = new ArrayList<>();
|
||||
combinedStreams.addAll(CONFIGURED_CATALOG.getStreams());
|
||||
combinedStreams.addAll(getConfiguredCatalog().getStreams());
|
||||
combinedStreams.addAll(newTables.getStreams());
|
||||
|
||||
final ConfiguredAirbyteCatalog updatedCatalog = new ConfiguredAirbyteCatalog().withStreams(combinedStreams);
|
||||
@@ -644,8 +643,8 @@ public abstract class CdcSourceTest {
|
||||
writeModelRecord(record);
|
||||
}
|
||||
|
||||
final AutoCloseableIterator<AirbyteMessage> secondBatchIterator = getSource()
|
||||
.read(getConfig(), updatedCatalog, state);
|
||||
final AutoCloseableIterator<AirbyteMessage> secondBatchIterator = source()
|
||||
.read(config(), updatedCatalog, state);
|
||||
final List<AirbyteMessage> dataFromSecondBatch = AutoCloseableIterators
|
||||
.toListAndClose(secondBatchIterator);
|
||||
|
||||
@@ -654,10 +653,10 @@ public abstract class CdcSourceTest {
|
||||
|
||||
final Map<String, Set<AirbyteRecordMessage>> recordsStreamWise = extractRecordMessagesStreamWise(dataFromSecondBatch);
|
||||
assertTrue(recordsStreamWise.containsKey(MODELS_STREAM_NAME));
|
||||
assertTrue(recordsStreamWise.containsKey(MODELS_STREAM_NAME + "_random"));
|
||||
assertTrue(recordsStreamWise.containsKey(RANDOM_TABLE_NAME));
|
||||
|
||||
final Set<AirbyteRecordMessage> recordsForModelsStreamFromSecondBatch = recordsStreamWise.get(MODELS_STREAM_NAME);
|
||||
final Set<AirbyteRecordMessage> recordsForModelsRandomStreamFromSecondBatch = recordsStreamWise.get(MODELS_STREAM_NAME + "_random");
|
||||
final Set<AirbyteRecordMessage> recordsForModelsRandomStreamFromSecondBatch = recordsStreamWise.get(RANDOM_TABLE_NAME);
|
||||
|
||||
assertEquals((MODEL_RECORDS_RANDOM.size()), recordsForModelsRandomStreamFromSecondBatch.size());
|
||||
assertEquals(20, recordsForModelsStreamFromSecondBatch.size());
|
||||
@@ -665,8 +664,8 @@ public abstract class CdcSourceTest {
|
||||
recordsForModelsRandomStreamFromSecondBatch.stream().map(AirbyteRecordMessage::getStream).collect(
|
||||
Collectors.toSet()),
|
||||
Sets
|
||||
.newHashSet(MODELS_STREAM_NAME + "_random"),
|
||||
randomTableSchema());
|
||||
.newHashSet(RANDOM_TABLE_NAME),
|
||||
randomSchema());
|
||||
assertExpectedRecords(recordsWritten, recordsForModelsStreamFromSecondBatch);
|
||||
|
||||
/*
|
||||
@@ -686,14 +685,14 @@ public abstract class CdcSourceTest {
|
||||
.jsonNode(ImmutableMap
|
||||
.of(COL_ID + "_random", 11000 + recordsCreated, COL_MAKE_ID + "_random", 1 + recordsCreated, COL_MODEL + "_random",
|
||||
"Fiesta-random" + recordsCreated));
|
||||
writeRecords(record2, randomTableSchema(), MODELS_STREAM_NAME + "_random",
|
||||
writeRecords(record2, randomSchema(), RANDOM_TABLE_NAME,
|
||||
COL_ID + "_random", COL_MAKE_ID + "_random", COL_MODEL + "_random");
|
||||
recordsWrittenInRandomTable.add(record2);
|
||||
}
|
||||
|
||||
final JsonNode state2 = stateAfterSecondBatch.get(stateAfterSecondBatch.size() - 1).getData();
|
||||
final AutoCloseableIterator<AirbyteMessage> thirdBatchIterator = getSource()
|
||||
.read(getConfig(), updatedCatalog, state2);
|
||||
final AutoCloseableIterator<AirbyteMessage> thirdBatchIterator = source()
|
||||
.read(config(), updatedCatalog, state2);
|
||||
final List<AirbyteMessage> dataFromThirdBatch = AutoCloseableIterators
|
||||
.toListAndClose(thirdBatchIterator);
|
||||
|
||||
@@ -710,16 +709,17 @@ public abstract class CdcSourceTest {
|
||||
.collect(Collectors.toSet());
|
||||
assertTrue(
|
||||
streamsInSyncCompletionStateAfterThirdSync.contains(
|
||||
new StreamDescriptor().withName(MODELS_STREAM_NAME + "_random").withNamespace(randomTableSchema())));
|
||||
assertTrue(streamsInSyncCompletionStateAfterThirdSync.contains(new StreamDescriptor().withName(MODELS_STREAM_NAME).withNamespace(MODELS_SCHEMA)));
|
||||
new StreamDescriptor().withName(RANDOM_TABLE_NAME).withNamespace(randomSchema())));
|
||||
assertTrue(
|
||||
streamsInSyncCompletionStateAfterThirdSync.contains(new StreamDescriptor().withName(MODELS_STREAM_NAME).withNamespace(modelsSchema())));
|
||||
assertNotNull(stateMessageEmittedAfterThirdSyncCompletion.getData());
|
||||
|
||||
final Map<String, Set<AirbyteRecordMessage>> recordsStreamWiseFromThirdBatch = extractRecordMessagesStreamWise(dataFromThirdBatch);
|
||||
assertTrue(recordsStreamWiseFromThirdBatch.containsKey(MODELS_STREAM_NAME));
|
||||
assertTrue(recordsStreamWiseFromThirdBatch.containsKey(MODELS_STREAM_NAME + "_random"));
|
||||
assertTrue(recordsStreamWiseFromThirdBatch.containsKey(RANDOM_TABLE_NAME));
|
||||
|
||||
final Set<AirbyteRecordMessage> recordsForModelsStreamFromThirdBatch = recordsStreamWiseFromThirdBatch.get(MODELS_STREAM_NAME);
|
||||
final Set<AirbyteRecordMessage> recordsForModelsRandomStreamFromThirdBatch = recordsStreamWiseFromThirdBatch.get(MODELS_STREAM_NAME + "_random");
|
||||
final Set<AirbyteRecordMessage> recordsForModelsRandomStreamFromThirdBatch = recordsStreamWiseFromThirdBatch.get(RANDOM_TABLE_NAME);
|
||||
|
||||
assertEquals(20, recordsForModelsStreamFromThirdBatch.size());
|
||||
assertEquals(20, recordsForModelsRandomStreamFromThirdBatch.size());
|
||||
@@ -728,8 +728,8 @@ public abstract class CdcSourceTest {
|
||||
recordsForModelsRandomStreamFromThirdBatch.stream().map(AirbyteRecordMessage::getStream).collect(
|
||||
Collectors.toSet()),
|
||||
Sets
|
||||
.newHashSet(MODELS_STREAM_NAME + "_random"),
|
||||
randomTableSchema());
|
||||
.newHashSet(RANDOM_TABLE_NAME),
|
||||
randomSchema());
|
||||
}
|
||||
|
||||
protected void assertStateMessagesForNewTableSnapshotTest(final List<AirbyteStateMessage> stateMessages,
|
||||
@@ -745,8 +745,8 @@ public abstract class CdcSourceTest {
|
||||
.collect(Collectors.toSet());
|
||||
assertEquals(2, streamsInSnapshotState.size());
|
||||
assertTrue(
|
||||
streamsInSnapshotState.contains(new StreamDescriptor().withName(MODELS_STREAM_NAME + "_random").withNamespace(randomTableSchema())));
|
||||
assertTrue(streamsInSnapshotState.contains(new StreamDescriptor().withName(MODELS_STREAM_NAME).withNamespace(MODELS_SCHEMA)));
|
||||
streamsInSnapshotState.contains(new StreamDescriptor().withName(RANDOM_TABLE_NAME).withNamespace(randomSchema())));
|
||||
assertTrue(streamsInSnapshotState.contains(new StreamDescriptor().withName(MODELS_STREAM_NAME).withNamespace(modelsSchema())));
|
||||
assertNotNull(stateMessageEmittedAfterSnapshotCompletionInSecondSync.getData());
|
||||
|
||||
final AirbyteStateMessage stateMessageEmittedAfterSecondSyncCompletion = stateMessages.get(1);
|
||||
@@ -760,16 +760,16 @@ public abstract class CdcSourceTest {
|
||||
assertEquals(2, streamsInSnapshotState.size());
|
||||
assertTrue(
|
||||
streamsInSyncCompletionState.contains(
|
||||
new StreamDescriptor().withName(MODELS_STREAM_NAME + "_random").withNamespace(randomTableSchema())));
|
||||
assertTrue(streamsInSyncCompletionState.contains(new StreamDescriptor().withName(MODELS_STREAM_NAME).withNamespace(MODELS_SCHEMA)));
|
||||
new StreamDescriptor().withName(RANDOM_TABLE_NAME).withNamespace(randomSchema())));
|
||||
assertTrue(streamsInSyncCompletionState.contains(new StreamDescriptor().withName(MODELS_STREAM_NAME).withNamespace(modelsSchema())));
|
||||
assertNotNull(stateMessageEmittedAfterSecondSyncCompletion.getData());
|
||||
}
|
||||
|
||||
protected AirbyteCatalog expectedCatalogForDiscover() {
|
||||
final AirbyteCatalog expectedCatalog = Jsons.clone(CATALOG);
|
||||
final AirbyteCatalog expectedCatalog = Jsons.clone(getCatalog());
|
||||
|
||||
createTable(MODELS_SCHEMA, MODELS_STREAM_NAME + "_2",
|
||||
columnClause(ImmutableMap.of(COL_ID, "INTEGER", COL_MAKE_ID, "INTEGER", COL_MODEL, "VARCHAR(200)"), Optional.empty()));
|
||||
final var columns = ImmutableMap.of(COL_ID, "INTEGER", COL_MAKE_ID, "INTEGER", COL_MODEL, "VARCHAR(200)");
|
||||
testdb.with(createTableSqlFmt(), modelsSchema(), MODELS_STREAM_NAME + "_2", columnClause(columns, Optional.empty()));
|
||||
|
||||
final List<AirbyteStream> streams = expectedCatalog.getStreams();
|
||||
// stream with PK
|
||||
@@ -779,7 +779,7 @@ public abstract class CdcSourceTest {
|
||||
|
||||
final AirbyteStream streamWithoutPK = CatalogHelpers.createAirbyteStream(
|
||||
MODELS_STREAM_NAME + "_2",
|
||||
MODELS_SCHEMA,
|
||||
modelsSchema(),
|
||||
Field.of(COL_ID, JsonSchemaType.INTEGER),
|
||||
Field.of(COL_MAKE_ID, JsonSchemaType.INTEGER),
|
||||
Field.of(COL_MODEL, JsonSchemaType.STRING));
|
||||
@@ -789,8 +789,8 @@ public abstract class CdcSourceTest {
|
||||
addCdcMetadataColumns(streamWithoutPK);
|
||||
|
||||
final AirbyteStream randomStream = CatalogHelpers.createAirbyteStream(
|
||||
MODELS_STREAM_NAME + "_random",
|
||||
randomTableSchema(),
|
||||
RANDOM_TABLE_NAME,
|
||||
randomSchema(),
|
||||
Field.of(COL_ID + "_random", JsonSchemaType.INTEGER),
|
||||
Field.of(COL_MAKE_ID + "_random", JsonSchemaType.INTEGER),
|
||||
Field.of(COL_MODEL + "_random", JsonSchemaType.STRING))
|
||||
@@ -807,31 +807,4 @@ public abstract class CdcSourceTest {
|
||||
return expectedCatalog;
|
||||
}
|
||||
|
||||
/**
|
||||
* The schema of a random table which is used as a new table in snapshot test
|
||||
*/
|
||||
protected abstract String randomTableSchema();
|
||||
|
||||
protected abstract CdcTargetPosition cdcLatestTargetPosition();
|
||||
|
||||
protected abstract CdcTargetPosition extractPosition(final JsonNode record);
|
||||
|
||||
protected abstract void assertNullCdcMetaData(final JsonNode data);
|
||||
|
||||
protected abstract void assertCdcMetaData(final JsonNode data, final boolean deletedAtNull);
|
||||
|
||||
protected abstract void removeCDCColumns(final ObjectNode data);
|
||||
|
||||
protected abstract void addCdcMetadataColumns(final AirbyteStream stream);
|
||||
|
||||
protected abstract void addCdcDefaultCursorField(final AirbyteStream stream);
|
||||
|
||||
protected abstract Source getSource();
|
||||
|
||||
protected abstract JsonNode getConfig();
|
||||
|
||||
protected abstract Database getDatabase();
|
||||
|
||||
protected abstract void assertExpectedStateMessages(final List<AirbyteStateMessage> stateMessages);
|
||||
|
||||
}
|
||||
|
||||
@@ -4,7 +4,6 @@
|
||||
|
||||
package io.airbyte.cdk.integrations.source.jdbc.test;
|
||||
|
||||
import static io.airbyte.cdk.db.jdbc.JdbcUtils.getDefaultSourceOperations;
|
||||
import static io.airbyte.cdk.integrations.source.relationaldb.RelationalDbQueryUtils.enquoteIdentifier;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
@@ -17,20 +16,15 @@ import static org.mockito.Mockito.spy;
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
|
||||
import io.airbyte.cdk.db.factory.DataSourceFactory;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcDatabase;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcSourceOperations;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.db.jdbc.StreamingJdbcDatabase;
|
||||
import io.airbyte.cdk.db.jdbc.streaming.AdaptiveStreamingQueryConfig;
|
||||
import io.airbyte.cdk.integrations.base.Source;
|
||||
import io.airbyte.cdk.integrations.source.jdbc.AbstractJdbcSource;
|
||||
import io.airbyte.cdk.integrations.source.relationaldb.RelationalDbQueryUtils;
|
||||
import io.airbyte.cdk.integrations.source.relationaldb.models.DbState;
|
||||
import io.airbyte.cdk.integrations.source.relationaldb.models.DbStreamState;
|
||||
import io.airbyte.cdk.testutils.TestDatabase;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.commons.resources.MoreResources;
|
||||
import io.airbyte.commons.string.Strings;
|
||||
import io.airbyte.commons.util.MoreIterators;
|
||||
import io.airbyte.protocol.models.Field;
|
||||
import io.airbyte.protocol.models.JsonSchemaType;
|
||||
@@ -60,76 +54,86 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import javax.sql.DataSource;
|
||||
import org.hamcrest.Matchers;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
/**
|
||||
* Tests that should be run on all Sources that extend the AbstractJdbcSource.
|
||||
*/
|
||||
// How leverage these tests:
|
||||
// 1. Extend this class in the test module of the Source.
|
||||
// 2. From the class that extends this one, you MUST call super.setup() in a @BeforeEach method.
|
||||
// Otherwise you'll see many NPE issues. Your before each should also handle providing a fresh
|
||||
// database between each test.
|
||||
// 3. From the class that extends this one, implement a @AfterEach that cleans out the database
|
||||
// between each test.
|
||||
// 4. Then implement the abstract methods documented below.
|
||||
@SuppressFBWarnings(
|
||||
value = {"MS_SHOULD_BE_FINAL"},
|
||||
justification = "The static variables are updated in sub classes for convenience, and cannot be final.")
|
||||
public abstract class JdbcSourceAcceptanceTest {
|
||||
justification = "The static variables are updated in subclasses for convenience, and cannot be final.")
|
||||
abstract public class JdbcSourceAcceptanceTest<S extends Source, T extends TestDatabase<?, T, ?>> {
|
||||
|
||||
// schema name must be randomized for each test run,
|
||||
// otherwise parallel runs can interfere with each other
|
||||
public static String SCHEMA_NAME = Strings.addRandomSuffix("jdbc_integration_test1", "_", 5).toLowerCase();
|
||||
public static String SCHEMA_NAME2 = Strings.addRandomSuffix("jdbc_integration_test2", "_", 5).toLowerCase();
|
||||
public static Set<String> TEST_SCHEMAS = Set.of(SCHEMA_NAME, SCHEMA_NAME2);
|
||||
static protected String SCHEMA_NAME = "jdbc_integration_test1";
|
||||
static protected String SCHEMA_NAME2 = "jdbc_integration_test2";
|
||||
static protected Set<String> TEST_SCHEMAS = Set.of(SCHEMA_NAME, SCHEMA_NAME2);
|
||||
|
||||
public static String TABLE_NAME = "id_and_name";
|
||||
public static String TABLE_NAME_WITH_SPACES = "id and name";
|
||||
public static String TABLE_NAME_WITHOUT_PK = "id_and_name_without_pk";
|
||||
public static String TABLE_NAME_COMPOSITE_PK = "full_name_composite_pk";
|
||||
public static String TABLE_NAME_WITHOUT_CURSOR_TYPE = "table_without_cursor_type";
|
||||
public static String TABLE_NAME_WITH_NULLABLE_CURSOR_TYPE = "table_with_null_cursor_type";
|
||||
static protected String TABLE_NAME = "id_and_name";
|
||||
static protected String TABLE_NAME_WITH_SPACES = "id and name";
|
||||
static protected String TABLE_NAME_WITHOUT_PK = "id_and_name_without_pk";
|
||||
static protected String TABLE_NAME_COMPOSITE_PK = "full_name_composite_pk";
|
||||
static protected String TABLE_NAME_WITHOUT_CURSOR_TYPE = "table_without_cursor_type";
|
||||
static protected String TABLE_NAME_WITH_NULLABLE_CURSOR_TYPE = "table_with_null_cursor_type";
|
||||
// this table is used in testing incremental sync with concurrent insertions
|
||||
public static String TABLE_NAME_AND_TIMESTAMP = "name_and_timestamp";
|
||||
static protected String TABLE_NAME_AND_TIMESTAMP = "name_and_timestamp";
|
||||
|
||||
public static String COL_ID = "id";
|
||||
public static String COL_NAME = "name";
|
||||
public static String COL_UPDATED_AT = "updated_at";
|
||||
public static String COL_FIRST_NAME = "first_name";
|
||||
public static String COL_LAST_NAME = "last_name";
|
||||
public static String COL_LAST_NAME_WITH_SPACE = "last name";
|
||||
public static String COL_CURSOR = "cursor_field";
|
||||
public static String COL_TIMESTAMP = "timestamp";
|
||||
public static String COL_TIMESTAMP_TYPE = "TIMESTAMP";
|
||||
public static Number ID_VALUE_1 = 1;
|
||||
public static Number ID_VALUE_2 = 2;
|
||||
public static Number ID_VALUE_3 = 3;
|
||||
public static Number ID_VALUE_4 = 4;
|
||||
public static Number ID_VALUE_5 = 5;
|
||||
static protected String COL_ID = "id";
|
||||
static protected String COL_NAME = "name";
|
||||
static protected String COL_UPDATED_AT = "updated_at";
|
||||
static protected String COL_FIRST_NAME = "first_name";
|
||||
static protected String COL_LAST_NAME = "last_name";
|
||||
static protected String COL_LAST_NAME_WITH_SPACE = "last name";
|
||||
static protected String COL_CURSOR = "cursor_field";
|
||||
static protected String COL_TIMESTAMP = "timestamp";
|
||||
static protected String COL_TIMESTAMP_TYPE = "TIMESTAMP";
|
||||
static protected Number ID_VALUE_1 = 1;
|
||||
static protected Number ID_VALUE_2 = 2;
|
||||
static protected Number ID_VALUE_3 = 3;
|
||||
static protected Number ID_VALUE_4 = 4;
|
||||
static protected Number ID_VALUE_5 = 5;
|
||||
|
||||
public static String DROP_SCHEMA_QUERY = "DROP SCHEMA IF EXISTS %s CASCADE";
|
||||
public static String COLUMN_CLAUSE_WITH_PK = "id INTEGER, name VARCHAR(200) NOT NULL, updated_at DATE NOT NULL";
|
||||
public static String COLUMN_CLAUSE_WITHOUT_PK = "id INTEGER, name VARCHAR(200) NOT NULL, updated_at DATE NOT NULL";
|
||||
public static String COLUMN_CLAUSE_WITH_COMPOSITE_PK =
|
||||
static protected String DROP_SCHEMA_QUERY = "DROP SCHEMA IF EXISTS %s CASCADE";
|
||||
static protected String COLUMN_CLAUSE_WITH_PK = "id INTEGER, name VARCHAR(200) NOT NULL, updated_at DATE NOT NULL";
|
||||
static protected String COLUMN_CLAUSE_WITHOUT_PK = "id INTEGER, name VARCHAR(200) NOT NULL, updated_at DATE NOT NULL";
|
||||
static protected String COLUMN_CLAUSE_WITH_COMPOSITE_PK =
|
||||
"first_name VARCHAR(200) NOT NULL, last_name VARCHAR(200) NOT NULL, updated_at DATE NOT NULL";
|
||||
|
||||
public static String CREATE_TABLE_WITHOUT_CURSOR_TYPE_QUERY = "CREATE TABLE %s (%s bit NOT NULL);";
|
||||
public static String INSERT_TABLE_WITHOUT_CURSOR_TYPE_QUERY = "INSERT INTO %s VALUES(0);";
|
||||
public static String CREATE_TABLE_WITH_NULLABLE_CURSOR_TYPE_QUERY = "CREATE TABLE %s (%s VARCHAR(20));";
|
||||
public static String INSERT_TABLE_WITH_NULLABLE_CURSOR_TYPE_QUERY = "INSERT INTO %s VALUES('Hello world :)');";
|
||||
public static String INSERT_TABLE_NAME_AND_TIMESTAMP_QUERY = "INSERT INTO %s (name, timestamp) VALUES ('%s', '%s')";
|
||||
static protected String CREATE_TABLE_WITHOUT_CURSOR_TYPE_QUERY = "CREATE TABLE %s (%s bit NOT NULL);";
|
||||
static protected String INSERT_TABLE_WITHOUT_CURSOR_TYPE_QUERY = "INSERT INTO %s VALUES(0);";
|
||||
static protected String CREATE_TABLE_WITH_NULLABLE_CURSOR_TYPE_QUERY = "CREATE TABLE %s (%s VARCHAR(20));";
|
||||
static protected String INSERT_TABLE_WITH_NULLABLE_CURSOR_TYPE_QUERY = "INSERT INTO %s VALUES('Hello world :)');";
|
||||
static protected String INSERT_TABLE_NAME_AND_TIMESTAMP_QUERY = "INSERT INTO %s (name, timestamp) VALUES ('%s', '%s')";
|
||||
|
||||
public JsonNode config;
|
||||
public DataSource dataSource;
|
||||
public JdbcDatabase database;
|
||||
public JdbcSourceOperations sourceOperations = getSourceOperations();
|
||||
public Source source;
|
||||
public static String streamName;
|
||||
protected T testdb;
|
||||
|
||||
protected String streamName() {
|
||||
return TABLE_NAME;
|
||||
}
|
||||
|
||||
/**
|
||||
* A valid configuration to connect to a test database.
|
||||
*
|
||||
* @return config
|
||||
*/
|
||||
abstract protected JsonNode config();
|
||||
|
||||
/**
|
||||
* An instance of the source that should be tests.
|
||||
*
|
||||
* @return abstract jdbc source
|
||||
*/
|
||||
abstract protected S source();
|
||||
|
||||
/**
|
||||
* Creates a TestDatabase instance to be used in {@link #setup()}.
|
||||
*
|
||||
* @return TestDatabase instance to use for test case.
|
||||
*/
|
||||
abstract protected T createTestDatabase();
|
||||
|
||||
/**
|
||||
* These tests write records without specifying a namespace (schema name). They will be written into
|
||||
@@ -140,54 +144,7 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
*
|
||||
* @return name that will be used to namespace the record.
|
||||
*/
|
||||
public abstract boolean supportsSchemas();
|
||||
|
||||
/**
|
||||
* A valid configuration to connect to a test database.
|
||||
*
|
||||
* @return config
|
||||
*/
|
||||
public abstract JsonNode getConfig();
|
||||
|
||||
/**
|
||||
* Full qualified class name of the JDBC driver for the database.
|
||||
*
|
||||
* @return driver
|
||||
*/
|
||||
public abstract String getDriverClass();
|
||||
|
||||
/**
|
||||
* An instance of the source that should be tests.
|
||||
*
|
||||
* @return abstract jdbc source
|
||||
*/
|
||||
public abstract AbstractJdbcSource<?> getJdbcSource();
|
||||
|
||||
/**
|
||||
* In some cases the Source that is being tested may be an AbstractJdbcSource, but because it is
|
||||
* decorated, Java cannot recognize it as such. In these cases, as a workaround a user can choose to
|
||||
* override getJdbcSource and have it return null. Then they can override this method with the
|
||||
* decorated source AND override getToDatabaseConfigFunction with the appropriate
|
||||
* toDatabaseConfigFunction that is hidden behind the decorator.
|
||||
*
|
||||
* @return source
|
||||
*/
|
||||
public Source getSource() {
|
||||
return getJdbcSource();
|
||||
}
|
||||
|
||||
/**
|
||||
* See getSource() for when to override this method.
|
||||
*
|
||||
* @return a function that maps a source's config to a jdbc config.
|
||||
*/
|
||||
public Function<JsonNode, JsonNode> getToDatabaseConfigFunction() {
|
||||
return getJdbcSource()::toDatabaseConfig;
|
||||
}
|
||||
|
||||
protected JdbcSourceOperations getSourceOperations() {
|
||||
return getDefaultSourceOperations();
|
||||
}
|
||||
abstract protected boolean supportsSchemas();
|
||||
|
||||
protected String createTableQuery(final String tableName, final String columnClause, final String primaryKeyClause) {
|
||||
return String.format("CREATE TABLE %s(%s %s %s)",
|
||||
@@ -211,100 +168,46 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
return clause.toString();
|
||||
}
|
||||
|
||||
protected String getJdbcParameterDelimiter() {
|
||||
return "&";
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
public void setup() throws Exception {
|
||||
source = getSource();
|
||||
config = getConfig();
|
||||
final JsonNode jdbcConfig = getToDatabaseConfigFunction().apply(config);
|
||||
|
||||
streamName = TABLE_NAME;
|
||||
|
||||
dataSource = getDataSource(jdbcConfig);
|
||||
|
||||
database = new StreamingJdbcDatabase(dataSource,
|
||||
getDefaultSourceOperations(),
|
||||
AdaptiveStreamingQueryConfig::new);
|
||||
|
||||
testdb = createTestDatabase();
|
||||
if (supportsSchemas()) {
|
||||
createSchemas();
|
||||
}
|
||||
|
||||
if (getDriverClass().toLowerCase().contains("oracle")) {
|
||||
database.execute(connection -> connection.createStatement()
|
||||
.execute("ALTER SESSION SET NLS_DATE_FORMAT = 'YYYY-MM-DD'"));
|
||||
if (testdb.getDatabaseDriver().equals(DatabaseDriver.ORACLE)) {
|
||||
testdb.with("ALTER SESSION SET NLS_DATE_FORMAT = 'YYYY-MM-DD'");
|
||||
}
|
||||
|
||||
database.execute(connection -> {
|
||||
|
||||
connection.createStatement().execute(
|
||||
createTableQuery(getFullyQualifiedTableName(TABLE_NAME), COLUMN_CLAUSE_WITH_PK,
|
||||
primaryKeyClause(Collections.singletonList("id"))));
|
||||
connection.createStatement().execute(
|
||||
String.format("INSERT INTO %s(id, name, updated_at) VALUES (1,'picard', '2004-10-19')",
|
||||
getFullyQualifiedTableName(TABLE_NAME)));
|
||||
connection.createStatement().execute(
|
||||
String.format("INSERT INTO %s(id, name, updated_at) VALUES (2, 'crusher', '2005-10-19')",
|
||||
getFullyQualifiedTableName(TABLE_NAME)));
|
||||
connection.createStatement().execute(
|
||||
String.format("INSERT INTO %s(id, name, updated_at) VALUES (3, 'vash', '2006-10-19')",
|
||||
getFullyQualifiedTableName(TABLE_NAME)));
|
||||
|
||||
connection.createStatement().execute(
|
||||
createTableQuery(getFullyQualifiedTableName(TABLE_NAME_WITHOUT_PK),
|
||||
COLUMN_CLAUSE_WITHOUT_PK, ""));
|
||||
connection.createStatement().execute(
|
||||
String.format("INSERT INTO %s(id, name, updated_at) VALUES (1,'picard', '2004-10-19')",
|
||||
getFullyQualifiedTableName(TABLE_NAME_WITHOUT_PK)));
|
||||
connection.createStatement().execute(
|
||||
String.format("INSERT INTO %s(id, name, updated_at) VALUES (2, 'crusher', '2005-10-19')",
|
||||
getFullyQualifiedTableName(TABLE_NAME_WITHOUT_PK)));
|
||||
connection.createStatement().execute(
|
||||
String.format("INSERT INTO %s(id, name, updated_at) VALUES (3, 'vash', '2006-10-19')",
|
||||
getFullyQualifiedTableName(TABLE_NAME_WITHOUT_PK)));
|
||||
|
||||
connection.createStatement().execute(
|
||||
createTableQuery(getFullyQualifiedTableName(TABLE_NAME_COMPOSITE_PK),
|
||||
COLUMN_CLAUSE_WITH_COMPOSITE_PK,
|
||||
primaryKeyClause(List.of("first_name", "last_name"))));
|
||||
connection.createStatement().execute(
|
||||
String.format(
|
||||
"INSERT INTO %s(first_name, last_name, updated_at) VALUES ('first' ,'picard', '2004-10-19')",
|
||||
getFullyQualifiedTableName(TABLE_NAME_COMPOSITE_PK)));
|
||||
connection.createStatement().execute(
|
||||
String.format(
|
||||
"INSERT INTO %s(first_name, last_name, updated_at) VALUES ('second', 'crusher', '2005-10-19')",
|
||||
getFullyQualifiedTableName(TABLE_NAME_COMPOSITE_PK)));
|
||||
connection.createStatement().execute(
|
||||
String.format(
|
||||
"INSERT INTO %s(first_name, last_name, updated_at) VALUES ('third', 'vash', '2006-10-19')",
|
||||
getFullyQualifiedTableName(TABLE_NAME_COMPOSITE_PK)));
|
||||
|
||||
});
|
||||
testdb
|
||||
.with(createTableQuery(getFullyQualifiedTableName(TABLE_NAME), COLUMN_CLAUSE_WITH_PK, primaryKeyClause(Collections.singletonList("id"))))
|
||||
.with("INSERT INTO %s(id, name, updated_at) VALUES (1, 'picard', '2004-10-19')", getFullyQualifiedTableName(TABLE_NAME))
|
||||
.with("INSERT INTO %s(id, name, updated_at) VALUES (2, 'crusher', '2005-10-19')", getFullyQualifiedTableName(TABLE_NAME))
|
||||
.with("INSERT INTO %s(id, name, updated_at) VALUES (3, 'vash', '2006-10-19')", getFullyQualifiedTableName(TABLE_NAME))
|
||||
.with(createTableQuery(getFullyQualifiedTableName(TABLE_NAME_WITHOUT_PK), COLUMN_CLAUSE_WITHOUT_PK, ""))
|
||||
.with("INSERT INTO %s(id, name, updated_at) VALUES (1, 'picard', '2004-10-19')", getFullyQualifiedTableName(TABLE_NAME_WITHOUT_PK))
|
||||
.with("INSERT INTO %s(id, name, updated_at) VALUES (2, 'crusher', '2005-10-19')", getFullyQualifiedTableName(TABLE_NAME_WITHOUT_PK))
|
||||
.with("INSERT INTO %s(id, name, updated_at) VALUES (3, 'vash', '2006-10-19')", getFullyQualifiedTableName(TABLE_NAME_WITHOUT_PK))
|
||||
.with(createTableQuery(getFullyQualifiedTableName(TABLE_NAME_COMPOSITE_PK), COLUMN_CLAUSE_WITH_COMPOSITE_PK,
|
||||
primaryKeyClause(List.of("first_name", "last_name"))))
|
||||
.with("INSERT INTO %s(first_name, last_name, updated_at) VALUES ('first', 'picard', '2004-10-19')",
|
||||
getFullyQualifiedTableName(TABLE_NAME_COMPOSITE_PK))
|
||||
.with("INSERT INTO %s(first_name, last_name, updated_at) VALUES ('second', 'crusher', '2005-10-19')",
|
||||
getFullyQualifiedTableName(TABLE_NAME_COMPOSITE_PK))
|
||||
.with("INSERT INTO %s(first_name, last_name, updated_at) VALUES ('third', 'vash', '2006-10-19')",
|
||||
getFullyQualifiedTableName(TABLE_NAME_COMPOSITE_PK));
|
||||
}
|
||||
|
||||
protected void maybeSetShorterConnectionTimeout() {
|
||||
protected void maybeSetShorterConnectionTimeout(final JsonNode config) {
|
||||
// Optionally implement this to speed up test cases which will result in a connection timeout.
|
||||
}
|
||||
|
||||
protected DataSource getDataSource(final JsonNode jdbcConfig) {
|
||||
return DataSourceFactory.create(
|
||||
jdbcConfig.get(JdbcUtils.USERNAME_KEY).asText(),
|
||||
jdbcConfig.has(JdbcUtils.PASSWORD_KEY) ? jdbcConfig.get(JdbcUtils.PASSWORD_KEY).asText() : null,
|
||||
getDriverClass(),
|
||||
jdbcConfig.get(JdbcUtils.JDBC_URL_KEY).asText(),
|
||||
JdbcUtils.parseJdbcParameters(jdbcConfig, JdbcUtils.CONNECTION_PROPERTIES_KEY, getJdbcParameterDelimiter()));
|
||||
}
|
||||
|
||||
public void tearDown() throws SQLException {
|
||||
dropSchemas();
|
||||
@AfterEach
|
||||
public void tearDown() {
|
||||
testdb.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSpec() throws Exception {
|
||||
final ConnectorSpecification actual = source.spec();
|
||||
final ConnectorSpecification actual = source().spec();
|
||||
final String resourceString = MoreResources.readResource("spec.json");
|
||||
final ConnectorSpecification expected = Jsons.deserialize(resourceString, ConnectorSpecification.class);
|
||||
|
||||
@@ -313,22 +216,23 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
|
||||
@Test
|
||||
void testCheckSuccess() throws Exception {
|
||||
final AirbyteConnectionStatus actual = source.check(config);
|
||||
final AirbyteConnectionStatus actual = source().check(config());
|
||||
final AirbyteConnectionStatus expected = new AirbyteConnectionStatus().withStatus(Status.SUCCEEDED);
|
||||
assertEquals(expected, actual);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testCheckFailure() throws Exception {
|
||||
maybeSetShorterConnectionTimeout();
|
||||
final var config = config();
|
||||
maybeSetShorterConnectionTimeout(config);
|
||||
((ObjectNode) config).put(JdbcUtils.PASSWORD_KEY, "fake");
|
||||
final AirbyteConnectionStatus actual = source.check(config);
|
||||
final AirbyteConnectionStatus actual = source().check(config);
|
||||
assertEquals(Status.FAILED, actual.getStatus());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testDiscover() throws Exception {
|
||||
final AirbyteCatalog actual = filterOutOtherSchemas(source.discover(config));
|
||||
final AirbyteCatalog actual = filterOutOtherSchemas(source().discover(config()));
|
||||
final AirbyteCatalog expected = getCatalog(getDefaultNamespace());
|
||||
assertEquals(expected.getStreams().size(), actual.getStreams().size());
|
||||
actual.getStreams().forEach(actualStream -> {
|
||||
@@ -343,13 +247,9 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
|
||||
@Test
|
||||
protected void testDiscoverWithNonCursorFields() throws Exception {
|
||||
database.execute(connection -> {
|
||||
connection.createStatement()
|
||||
.execute(String.format(CREATE_TABLE_WITHOUT_CURSOR_TYPE_QUERY, getFullyQualifiedTableName(TABLE_NAME_WITHOUT_CURSOR_TYPE), COL_CURSOR));
|
||||
connection.createStatement().execute(String.format(INSERT_TABLE_WITHOUT_CURSOR_TYPE_QUERY,
|
||||
getFullyQualifiedTableName(TABLE_NAME_WITHOUT_CURSOR_TYPE)));
|
||||
});
|
||||
final AirbyteCatalog actual = filterOutOtherSchemas(source.discover(config));
|
||||
testdb.with(CREATE_TABLE_WITHOUT_CURSOR_TYPE_QUERY, getFullyQualifiedTableName(TABLE_NAME_WITHOUT_CURSOR_TYPE), COL_CURSOR)
|
||||
.with(INSERT_TABLE_WITHOUT_CURSOR_TYPE_QUERY, getFullyQualifiedTableName(TABLE_NAME_WITHOUT_CURSOR_TYPE));
|
||||
final AirbyteCatalog actual = filterOutOtherSchemas(source().discover(config()));
|
||||
final AirbyteStream stream =
|
||||
actual.getStreams().stream().filter(s -> s.getName().equalsIgnoreCase(TABLE_NAME_WITHOUT_CURSOR_TYPE)).findFirst().orElse(null);
|
||||
assertNotNull(stream);
|
||||
@@ -360,14 +260,9 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
|
||||
@Test
|
||||
protected void testDiscoverWithNullableCursorFields() throws Exception {
|
||||
database.execute(connection -> {
|
||||
connection.createStatement()
|
||||
.execute(String.format(CREATE_TABLE_WITH_NULLABLE_CURSOR_TYPE_QUERY, getFullyQualifiedTableName(TABLE_NAME_WITH_NULLABLE_CURSOR_TYPE),
|
||||
COL_CURSOR));
|
||||
connection.createStatement().execute(String.format(INSERT_TABLE_WITH_NULLABLE_CURSOR_TYPE_QUERY,
|
||||
getFullyQualifiedTableName(TABLE_NAME_WITH_NULLABLE_CURSOR_TYPE)));
|
||||
});
|
||||
final AirbyteCatalog actual = filterOutOtherSchemas(source.discover(config));
|
||||
testdb.with(CREATE_TABLE_WITH_NULLABLE_CURSOR_TYPE_QUERY, getFullyQualifiedTableName(TABLE_NAME_WITH_NULLABLE_CURSOR_TYPE), COL_CURSOR)
|
||||
.with(INSERT_TABLE_WITH_NULLABLE_CURSOR_TYPE_QUERY, getFullyQualifiedTableName(TABLE_NAME_WITH_NULLABLE_CURSOR_TYPE));
|
||||
final AirbyteCatalog actual = filterOutOtherSchemas(source().discover(config()));
|
||||
final AirbyteStream stream =
|
||||
actual.getStreams().stream().filter(s -> s.getName().equalsIgnoreCase(TABLE_NAME_WITH_NULLABLE_CURSOR_TYPE)).findFirst().orElse(null);
|
||||
assertNotNull(stream);
|
||||
@@ -394,28 +289,22 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
@Test
|
||||
void testDiscoverWithMultipleSchemas() throws Exception {
|
||||
// clickhouse and mysql do not have a concept of schemas, so this test does not make sense for them.
|
||||
String driverClass = getDriverClass().toLowerCase();
|
||||
if (driverClass.contains("mysql") || driverClass.contains("clickhouse") || driverClass.contains("teradata")) {
|
||||
return;
|
||||
switch (testdb.getDatabaseDriver()) {
|
||||
case MYSQL, CLICKHOUSE, TERADATA:
|
||||
return;
|
||||
}
|
||||
|
||||
// add table and data to a separate schema.
|
||||
database.execute(connection -> {
|
||||
connection.createStatement().execute(
|
||||
String.format("CREATE TABLE %s(id VARCHAR(200) NOT NULL, name VARCHAR(200) NOT NULL)",
|
||||
RelationalDbQueryUtils.getFullyQualifiedTableName(SCHEMA_NAME2, TABLE_NAME)));
|
||||
connection.createStatement()
|
||||
.execute(String.format("INSERT INTO %s(id, name) VALUES ('1','picard')",
|
||||
RelationalDbQueryUtils.getFullyQualifiedTableName(SCHEMA_NAME2, TABLE_NAME)));
|
||||
connection.createStatement()
|
||||
.execute(String.format("INSERT INTO %s(id, name) VALUES ('2', 'crusher')",
|
||||
RelationalDbQueryUtils.getFullyQualifiedTableName(SCHEMA_NAME2, TABLE_NAME)));
|
||||
connection.createStatement()
|
||||
.execute(String.format("INSERT INTO %s(id, name) VALUES ('3', 'vash')",
|
||||
RelationalDbQueryUtils.getFullyQualifiedTableName(SCHEMA_NAME2, TABLE_NAME)));
|
||||
});
|
||||
testdb.with("CREATE TABLE %s(id VARCHAR(200) NOT NULL, name VARCHAR(200) NOT NULL)",
|
||||
RelationalDbQueryUtils.getFullyQualifiedTableName(SCHEMA_NAME2, TABLE_NAME))
|
||||
.with("INSERT INTO %s(id, name) VALUES ('1','picard')",
|
||||
RelationalDbQueryUtils.getFullyQualifiedTableName(SCHEMA_NAME2, TABLE_NAME))
|
||||
.with("INSERT INTO %s(id, name) VALUES ('2', 'crusher')",
|
||||
RelationalDbQueryUtils.getFullyQualifiedTableName(SCHEMA_NAME2, TABLE_NAME))
|
||||
.with("INSERT INTO %s(id, name) VALUES ('3', 'vash')",
|
||||
RelationalDbQueryUtils.getFullyQualifiedTableName(SCHEMA_NAME2, TABLE_NAME));
|
||||
|
||||
final AirbyteCatalog actual = source.discover(config);
|
||||
final AirbyteCatalog actual = source().discover(config());
|
||||
|
||||
final AirbyteCatalog expected = getCatalog(getDefaultNamespace());
|
||||
final List<AirbyteStream> catalogStreams = new ArrayList<>();
|
||||
@@ -438,7 +327,7 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
void testReadSuccess() throws Exception {
|
||||
final List<AirbyteMessage> actualMessages =
|
||||
MoreIterators.toList(
|
||||
source.read(config, getConfiguredCatalogWithOneStream(getDefaultNamespace()), null));
|
||||
source().read(config(), getConfiguredCatalogWithOneStream(getDefaultNamespace()), null));
|
||||
|
||||
setEmittedAtToNull(actualMessages);
|
||||
final List<AirbyteMessage> expectedMessages = getTestMessages();
|
||||
@@ -449,9 +338,9 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
@Test
|
||||
void testReadOneColumn() throws Exception {
|
||||
final ConfiguredAirbyteCatalog catalog = CatalogHelpers
|
||||
.createConfiguredAirbyteCatalog(streamName, getDefaultNamespace(), Field.of(COL_ID, JsonSchemaType.NUMBER));
|
||||
.createConfiguredAirbyteCatalog(streamName(), getDefaultNamespace(), Field.of(COL_ID, JsonSchemaType.NUMBER));
|
||||
final List<AirbyteMessage> actualMessages = MoreIterators
|
||||
.toList(source.read(config, catalog, null));
|
||||
.toList(source().read(config(), catalog, null));
|
||||
|
||||
setEmittedAtToNull(actualMessages);
|
||||
|
||||
@@ -481,23 +370,12 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
final List<AirbyteMessage> expectedMessages = new ArrayList<>(getTestMessages());
|
||||
|
||||
for (int i = 2; i < 10; i++) {
|
||||
final int iFinal = i;
|
||||
final String streamName2 = streamName + i;
|
||||
database.execute(connection -> {
|
||||
connection.createStatement()
|
||||
.execute(
|
||||
createTableQuery(getFullyQualifiedTableName(TABLE_NAME + iFinal),
|
||||
"id INTEGER, name VARCHAR(200)", ""));
|
||||
connection.createStatement()
|
||||
.execute(String.format("INSERT INTO %s(id, name) VALUES (1,'picard')",
|
||||
getFullyQualifiedTableName(TABLE_NAME + iFinal)));
|
||||
connection.createStatement()
|
||||
.execute(String.format("INSERT INTO %s(id, name) VALUES (2, 'crusher')",
|
||||
getFullyQualifiedTableName(TABLE_NAME + iFinal)));
|
||||
connection.createStatement()
|
||||
.execute(String.format("INSERT INTO %s(id, name) VALUES (3, 'vash')",
|
||||
getFullyQualifiedTableName(TABLE_NAME + iFinal)));
|
||||
});
|
||||
final String streamName2 = streamName() + i;
|
||||
final String tableName = getFullyQualifiedTableName(TABLE_NAME + i);
|
||||
testdb.with(createTableQuery(tableName, "id INTEGER, name VARCHAR(200)", ""))
|
||||
.with("INSERT INTO %s(id, name) VALUES (1,'picard')", tableName)
|
||||
.with("INSERT INTO %s(id, name) VALUES (2, 'crusher')", tableName)
|
||||
.with("INSERT INTO %s(id, name) VALUES (3, 'vash')", tableName);
|
||||
catalog.getStreams().add(CatalogHelpers.createConfiguredAirbyteStream(
|
||||
streamName2,
|
||||
getDefaultNamespace(),
|
||||
@@ -508,7 +386,7 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
}
|
||||
|
||||
final List<AirbyteMessage> actualMessages = MoreIterators
|
||||
.toList(source.read(config, catalog, null));
|
||||
.toList(source().read(config(), catalog, null));
|
||||
|
||||
setEmittedAtToNull(actualMessages);
|
||||
|
||||
@@ -541,7 +419,7 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
getConfiguredCatalogWithOneStream(getDefaultNamespace()).getStreams().get(0),
|
||||
streamForTableWithSpaces));
|
||||
final List<AirbyteMessage> actualMessages = MoreIterators
|
||||
.toList(source.read(config, catalog, null));
|
||||
.toList(source().read(config(), catalog, null));
|
||||
|
||||
setEmittedAtToNull(actualMessages);
|
||||
|
||||
@@ -577,7 +455,7 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
.withStreams(List.of(spiedAbStream));
|
||||
doCallRealMethod().doThrow(new RuntimeException()).when(spiedAbStream).getStream();
|
||||
|
||||
assertThrows(RuntimeException.class, () -> source.read(config, catalog, null));
|
||||
assertThrows(RuntimeException.class, () -> source().read(config(), catalog, null));
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -665,6 +543,7 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
|
||||
@Test
|
||||
void testReadOneTableIncrementallyTwice() throws Exception {
|
||||
final var config = config();
|
||||
final String namespace = getDefaultNamespace();
|
||||
final ConfiguredAirbyteCatalog configuredCatalog = getConfiguredCatalogWithOneStream(namespace);
|
||||
configuredCatalog.getStreams().forEach(airbyteStream -> {
|
||||
@@ -674,7 +553,7 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
});
|
||||
|
||||
final List<AirbyteMessage> actualMessagesFirstSync = MoreIterators
|
||||
.toList(source.read(config, configuredCatalog, createEmptyState(streamName, namespace)));
|
||||
.toList(source().read(config, configuredCatalog, createEmptyState(streamName(), namespace)));
|
||||
|
||||
final Optional<AirbyteMessage> stateAfterFirstSyncOptional = actualMessagesFirstSync.stream()
|
||||
.filter(r -> r.getType() == Type.STATE).findFirst();
|
||||
@@ -683,7 +562,7 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
executeStatementReadIncrementallyTwice();
|
||||
|
||||
final List<AirbyteMessage> actualMessagesSecondSync = MoreIterators
|
||||
.toList(source.read(config, configuredCatalog, extractState(stateAfterFirstSyncOptional.get())));
|
||||
.toList(source().read(config, configuredCatalog, extractState(stateAfterFirstSyncOptional.get())));
|
||||
|
||||
assertEquals(2,
|
||||
(int) actualMessagesSecondSync.stream().filter(r -> r.getType() == Type.RECORD).count());
|
||||
@@ -696,33 +575,28 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
assertTrue(actualMessagesSecondSync.containsAll(expectedMessages));
|
||||
}
|
||||
|
||||
protected void executeStatementReadIncrementallyTwice() throws SQLException {
|
||||
database.execute(connection -> {
|
||||
connection.createStatement().execute(
|
||||
String.format("INSERT INTO %s(id, name, updated_at) VALUES (4,'riker', '2006-10-19')",
|
||||
getFullyQualifiedTableName(TABLE_NAME)));
|
||||
connection.createStatement().execute(
|
||||
String.format("INSERT INTO %s(id, name, updated_at) VALUES (5, 'data', '2006-10-19')",
|
||||
getFullyQualifiedTableName(TABLE_NAME)));
|
||||
});
|
||||
protected void executeStatementReadIncrementallyTwice() {
|
||||
testdb
|
||||
.with("INSERT INTO %s(id, name, updated_at) VALUES (4, 'riker', '2006-10-19')", getFullyQualifiedTableName(TABLE_NAME))
|
||||
.with("INSERT INTO %s(id, name, updated_at) VALUES (5, 'data', '2006-10-19')", getFullyQualifiedTableName(TABLE_NAME));
|
||||
}
|
||||
|
||||
protected List<AirbyteMessage> getExpectedAirbyteMessagesSecondSync(final String namespace) {
|
||||
final List<AirbyteMessage> expectedMessages = new ArrayList<>();
|
||||
expectedMessages.add(new AirbyteMessage().withType(Type.RECORD)
|
||||
.withRecord(new AirbyteRecordMessage().withStream(streamName).withNamespace(namespace)
|
||||
.withRecord(new AirbyteRecordMessage().withStream(streamName()).withNamespace(namespace)
|
||||
.withData(Jsons.jsonNode(Map
|
||||
.of(COL_ID, ID_VALUE_4,
|
||||
COL_NAME, "riker",
|
||||
COL_UPDATED_AT, "2006-10-19")))));
|
||||
expectedMessages.add(new AirbyteMessage().withType(Type.RECORD)
|
||||
.withRecord(new AirbyteRecordMessage().withStream(streamName).withNamespace(namespace)
|
||||
.withRecord(new AirbyteRecordMessage().withStream(streamName()).withNamespace(namespace)
|
||||
.withData(Jsons.jsonNode(Map
|
||||
.of(COL_ID, ID_VALUE_5,
|
||||
COL_NAME, "data",
|
||||
COL_UPDATED_AT, "2006-10-19")))));
|
||||
final DbStreamState state = new DbStreamState()
|
||||
.withStreamName(streamName)
|
||||
.withStreamName(streamName())
|
||||
.withStreamNamespace(namespace)
|
||||
.withCursorField(List.of(COL_ID))
|
||||
.withCursor("5")
|
||||
@@ -734,20 +608,12 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
@Test
|
||||
void testReadMultipleTablesIncrementally() throws Exception {
|
||||
final String tableName2 = TABLE_NAME + 2;
|
||||
final String streamName2 = streamName + 2;
|
||||
database.execute(ctx -> {
|
||||
ctx.createStatement().execute(
|
||||
createTableQuery(getFullyQualifiedTableName(tableName2), "id INTEGER, name VARCHAR(200)", ""));
|
||||
ctx.createStatement().execute(
|
||||
String.format("INSERT INTO %s(id, name) VALUES (1,'picard')",
|
||||
getFullyQualifiedTableName(tableName2)));
|
||||
ctx.createStatement().execute(
|
||||
String.format("INSERT INTO %s(id, name) VALUES (2, 'crusher')",
|
||||
getFullyQualifiedTableName(tableName2)));
|
||||
ctx.createStatement().execute(
|
||||
String.format("INSERT INTO %s(id, name) VALUES (3, 'vash')",
|
||||
getFullyQualifiedTableName(tableName2)));
|
||||
});
|
||||
final String streamName2 = streamName() + 2;
|
||||
final String fqTableName2 = getFullyQualifiedTableName(tableName2);
|
||||
testdb.with(createTableQuery(fqTableName2, "id INTEGER, name VARCHAR(200)", ""))
|
||||
.with("INSERT INTO %s(id, name) VALUES (1,'picard')", fqTableName2)
|
||||
.with("INSERT INTO %s(id, name) VALUES (2, 'crusher')", fqTableName2)
|
||||
.with("INSERT INTO %s(id, name) VALUES (3, 'vash')", fqTableName2);
|
||||
|
||||
final String namespace = getDefaultNamespace();
|
||||
final ConfiguredAirbyteCatalog configuredCatalog = getConfiguredCatalogWithOneStream(
|
||||
@@ -764,7 +630,7 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
});
|
||||
|
||||
final List<AirbyteMessage> actualMessagesFirstSync = MoreIterators
|
||||
.toList(source.read(config, configuredCatalog, createEmptyState(streamName, namespace)));
|
||||
.toList(source().read(config(), configuredCatalog, createEmptyState(streamName(), namespace)));
|
||||
|
||||
// get last state message.
|
||||
final Optional<AirbyteMessage> stateAfterFirstSyncOptional = actualMessagesFirstSync.stream()
|
||||
@@ -779,7 +645,7 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
// Represents the state after the first stream has been updated
|
||||
final List<DbStreamState> expectedStateStreams1 = List.of(
|
||||
new DbStreamState()
|
||||
.withStreamName(streamName)
|
||||
.withStreamName(streamName())
|
||||
.withStreamNamespace(namespace)
|
||||
.withCursorField(List.of(COL_ID))
|
||||
.withCursor("3")
|
||||
@@ -792,7 +658,7 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
// Represents the state after both streams have been updated
|
||||
final List<DbStreamState> expectedStateStreams2 = List.of(
|
||||
new DbStreamState()
|
||||
.withStreamName(streamName)
|
||||
.withStreamName(streamName())
|
||||
.withStreamNamespace(namespace)
|
||||
.withCursorField(List.of(COL_ID))
|
||||
.withCursor("3")
|
||||
@@ -843,17 +709,14 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
// See https://github.com/airbytehq/airbyte/issues/14732 for rationale and details.
|
||||
@Test
|
||||
public void testIncrementalWithConcurrentInsertion() throws Exception {
|
||||
final String driverName = getDriverClass().toLowerCase();
|
||||
final String namespace = getDefaultNamespace();
|
||||
final String fullyQualifiedTableName = getFullyQualifiedTableName(TABLE_NAME_AND_TIMESTAMP);
|
||||
final String columnDefinition = String.format("name VARCHAR(200) NOT NULL, %s %s NOT NULL", COL_TIMESTAMP, COL_TIMESTAMP_TYPE);
|
||||
|
||||
// 1st sync
|
||||
database.execute(ctx -> {
|
||||
ctx.createStatement().execute(createTableQuery(fullyQualifiedTableName, columnDefinition, ""));
|
||||
ctx.createStatement().execute(String.format(INSERT_TABLE_NAME_AND_TIMESTAMP_QUERY, fullyQualifiedTableName, "a", "2021-01-01 00:00:00"));
|
||||
ctx.createStatement().execute(String.format(INSERT_TABLE_NAME_AND_TIMESTAMP_QUERY, fullyQualifiedTableName, "b", "2021-01-01 00:00:00"));
|
||||
});
|
||||
testdb.with(createTableQuery(fullyQualifiedTableName, columnDefinition, ""))
|
||||
.with(INSERT_TABLE_NAME_AND_TIMESTAMP_QUERY, fullyQualifiedTableName, "a", "2021-01-01 00:00:00")
|
||||
.with(INSERT_TABLE_NAME_AND_TIMESTAMP_QUERY, fullyQualifiedTableName, "b", "2021-01-01 00:00:00");
|
||||
|
||||
final ConfiguredAirbyteCatalog configuredCatalog = CatalogHelpers.toDefaultConfiguredCatalog(
|
||||
new AirbyteCatalog().withStreams(List.of(
|
||||
@@ -870,7 +733,7 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
});
|
||||
|
||||
final List<AirbyteMessage> firstSyncActualMessages = MoreIterators.toList(
|
||||
source.read(config, configuredCatalog, createEmptyState(TABLE_NAME_AND_TIMESTAMP, namespace)));
|
||||
source().read(config(), configuredCatalog, createEmptyState(TABLE_NAME_AND_TIMESTAMP, namespace)));
|
||||
|
||||
// cursor after 1st sync: 2021-01-01 00:00:00, count 2
|
||||
final Optional<AirbyteMessage> firstSyncStateOptional = firstSyncActualMessages.stream().filter(r -> r.getType() == Type.STATE).findFirst();
|
||||
@@ -886,19 +749,17 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
.map(r -> r.getRecord().getData().get(COL_NAME).asText())
|
||||
.toList();
|
||||
// teradata doesn't make insertion order guarantee when equal ordering value
|
||||
if (driverName.contains("teradata")) {
|
||||
if (testdb.getDatabaseDriver().equals(DatabaseDriver.TERADATA)) {
|
||||
assertThat(List.of("a", "b"), Matchers.containsInAnyOrder(firstSyncNames.toArray()));
|
||||
} else {
|
||||
assertEquals(List.of("a", "b"), firstSyncNames);
|
||||
}
|
||||
|
||||
// 2nd sync
|
||||
database.execute(ctx -> {
|
||||
ctx.createStatement().execute(String.format(INSERT_TABLE_NAME_AND_TIMESTAMP_QUERY, fullyQualifiedTableName, "c", "2021-01-02 00:00:00"));
|
||||
});
|
||||
testdb.with(INSERT_TABLE_NAME_AND_TIMESTAMP_QUERY, fullyQualifiedTableName, "c", "2021-01-02 00:00:00");
|
||||
|
||||
final List<AirbyteMessage> secondSyncActualMessages = MoreIterators.toList(
|
||||
source.read(config, configuredCatalog, createState(TABLE_NAME_AND_TIMESTAMP, namespace, firstSyncState)));
|
||||
source().read(config(), configuredCatalog, createState(TABLE_NAME_AND_TIMESTAMP, namespace, firstSyncState)));
|
||||
|
||||
// cursor after 2nd sync: 2021-01-02 00:00:00, count 1
|
||||
final Optional<AirbyteMessage> secondSyncStateOptional = secondSyncActualMessages.stream().filter(r -> r.getType() == Type.STATE).findFirst();
|
||||
@@ -916,14 +777,12 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
assertEquals(List.of("c"), secondSyncNames);
|
||||
|
||||
// 3rd sync has records with duplicated cursors
|
||||
database.execute(ctx -> {
|
||||
ctx.createStatement().execute(String.format(INSERT_TABLE_NAME_AND_TIMESTAMP_QUERY, fullyQualifiedTableName, "d", "2021-01-02 00:00:00"));
|
||||
ctx.createStatement().execute(String.format(INSERT_TABLE_NAME_AND_TIMESTAMP_QUERY, fullyQualifiedTableName, "e", "2021-01-02 00:00:00"));
|
||||
ctx.createStatement().execute(String.format(INSERT_TABLE_NAME_AND_TIMESTAMP_QUERY, fullyQualifiedTableName, "f", "2021-01-03 00:00:00"));
|
||||
});
|
||||
testdb.with(INSERT_TABLE_NAME_AND_TIMESTAMP_QUERY, fullyQualifiedTableName, "d", "2021-01-02 00:00:00")
|
||||
.with(INSERT_TABLE_NAME_AND_TIMESTAMP_QUERY, fullyQualifiedTableName, "e", "2021-01-02 00:00:00")
|
||||
.with(INSERT_TABLE_NAME_AND_TIMESTAMP_QUERY, fullyQualifiedTableName, "f", "2021-01-03 00:00:00");
|
||||
|
||||
final List<AirbyteMessage> thirdSyncActualMessages = MoreIterators.toList(
|
||||
source.read(config, configuredCatalog, createState(TABLE_NAME_AND_TIMESTAMP, namespace, secondSyncState)));
|
||||
source().read(config(), configuredCatalog, createState(TABLE_NAME_AND_TIMESTAMP, namespace, secondSyncState)));
|
||||
|
||||
// Cursor after 3rd sync is: 2021-01-03 00:00:00, count 1.
|
||||
final Optional<AirbyteMessage> thirdSyncStateOptional = thirdSyncActualMessages.stream().filter(r -> r.getType() == Type.STATE).findFirst();
|
||||
@@ -942,12 +801,11 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
.toList();
|
||||
|
||||
// teradata doesn't make insertion order guarantee when equal ordering value
|
||||
if (driverName.contains("teradata")) {
|
||||
if (testdb.getDatabaseDriver().equals(DatabaseDriver.TERADATA)) {
|
||||
assertThat(List.of("c", "d", "e", "f"), Matchers.containsInAnyOrder(thirdSyncExpectedNames.toArray()));
|
||||
} else {
|
||||
assertEquals(List.of("c", "d", "e", "f"), thirdSyncExpectedNames);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
protected JsonNode getStateData(final AirbyteMessage airbyteMessage, final String streamName) {
|
||||
@@ -989,7 +847,7 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
final DbStreamState dbStreamState = buildStreamState(airbyteStream, initialCursorField, initialCursorValue);
|
||||
|
||||
final List<AirbyteMessage> actualMessages = MoreIterators
|
||||
.toList(source.read(config, configuredCatalog, Jsons.jsonNode(createState(List.of(dbStreamState)))));
|
||||
.toList(source().read(config(), configuredCatalog, Jsons.jsonNode(createState(List.of(dbStreamState)))));
|
||||
|
||||
setEmittedAtToNull(actualMessages);
|
||||
|
||||
@@ -1019,7 +877,7 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
final ConfiguredAirbyteCatalog catalog = CatalogHelpers.toDefaultConfiguredCatalog(getCatalog(defaultNamespace));
|
||||
// Filter to only keep the main stream name as configured stream
|
||||
catalog.withStreams(
|
||||
catalog.getStreams().stream().filter(s -> s.getStream().getName().equals(streamName))
|
||||
catalog.getStreams().stream().filter(s -> s.getStream().getName().equals(streamName()))
|
||||
.collect(Collectors.toList()));
|
||||
return catalog;
|
||||
}
|
||||
@@ -1056,20 +914,20 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
protected List<AirbyteMessage> getTestMessages() {
|
||||
return List.of(
|
||||
new AirbyteMessage().withType(Type.RECORD)
|
||||
.withRecord(new AirbyteRecordMessage().withStream(streamName).withNamespace(getDefaultNamespace())
|
||||
.withRecord(new AirbyteRecordMessage().withStream(streamName()).withNamespace(getDefaultNamespace())
|
||||
.withData(Jsons.jsonNode(Map
|
||||
.of(COL_ID, ID_VALUE_1,
|
||||
COL_NAME, "picard",
|
||||
COL_UPDATED_AT, "2004-10-19")))),
|
||||
new AirbyteMessage().withType(Type.RECORD)
|
||||
.withRecord(new AirbyteRecordMessage().withStream(streamName).withNamespace(getDefaultNamespace())
|
||||
.withRecord(new AirbyteRecordMessage().withStream(streamName()).withNamespace(getDefaultNamespace())
|
||||
.withData(Jsons.jsonNode(Map
|
||||
.of(COL_ID, ID_VALUE_2,
|
||||
COL_NAME, "crusher",
|
||||
COL_UPDATED_AT,
|
||||
"2005-10-19")))),
|
||||
new AirbyteMessage().withType(Type.RECORD)
|
||||
.withRecord(new AirbyteRecordMessage().withStream(streamName).withNamespace(getDefaultNamespace())
|
||||
.withRecord(new AirbyteRecordMessage().withStream(streamName()).withNamespace(getDefaultNamespace())
|
||||
.withData(Jsons.jsonNode(Map
|
||||
.of(COL_ID, ID_VALUE_3,
|
||||
COL_NAME, "vash",
|
||||
@@ -1108,7 +966,7 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
final String tableNameWithSpaces = TABLE_NAME_WITH_SPACES + "2";
|
||||
final String streamName2 = tableNameWithSpaces;
|
||||
|
||||
database.execute(connection -> {
|
||||
try (final var connection = testdb.getDataSource().getConnection()) {
|
||||
final String identifierQuoteString = connection.getMetaData().getIdentifierQuoteString();
|
||||
connection.createStatement()
|
||||
.execute(
|
||||
@@ -1132,7 +990,7 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
getFullyQualifiedTableName(
|
||||
enquoteIdentifier(tableNameWithSpaces, identifierQuoteString)),
|
||||
enquoteIdentifier(COL_LAST_NAME_WITH_SPACE, identifierQuoteString)));
|
||||
});
|
||||
}
|
||||
|
||||
return CatalogHelpers.createConfiguredAirbyteStream(
|
||||
streamName2,
|
||||
@@ -1145,32 +1003,27 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
return RelationalDbQueryUtils.getFullyQualifiedTableName(getDefaultSchemaName(), tableName);
|
||||
}
|
||||
|
||||
public void createSchemas() throws SQLException {
|
||||
protected void createSchemas() {
|
||||
if (supportsSchemas()) {
|
||||
for (final String schemaName : TEST_SCHEMAS) {
|
||||
final String createSchemaQuery = String.format("CREATE SCHEMA %s;", schemaName);
|
||||
database.execute(connection -> connection.createStatement().execute(createSchemaQuery));
|
||||
testdb.with("CREATE SCHEMA %s;", schemaName);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void dropSchemas() throws SQLException {
|
||||
protected void dropSchemas() {
|
||||
if (supportsSchemas()) {
|
||||
for (final String schemaName : TEST_SCHEMAS) {
|
||||
final String dropSchemaQuery = String
|
||||
.format(DROP_SCHEMA_QUERY, schemaName);
|
||||
database.execute(connection -> connection.createStatement().execute(dropSchemaQuery));
|
||||
testdb.with(DROP_SCHEMA_QUERY, schemaName);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private JsonNode convertIdBasedOnDatabase(final int idValue) {
|
||||
final var driverClass = getDriverClass().toLowerCase();
|
||||
if (driverClass.contains("oracle") || driverClass.contains("snowflake")) {
|
||||
return Jsons.jsonNode(BigDecimal.valueOf(idValue));
|
||||
} else {
|
||||
return Jsons.jsonNode(idValue);
|
||||
}
|
||||
return switch (testdb.getDatabaseDriver()) {
|
||||
case ORACLE, SNOWFLAKE -> Jsons.jsonNode(BigDecimal.valueOf(idValue));
|
||||
default -> Jsons.jsonNode(idValue);
|
||||
};
|
||||
}
|
||||
|
||||
private String getDefaultSchemaName() {
|
||||
@@ -1178,13 +1031,11 @@ public abstract class JdbcSourceAcceptanceTest {
|
||||
}
|
||||
|
||||
protected String getDefaultNamespace() {
|
||||
// mysql does not support schemas. it namespaces using database names instead.
|
||||
if (getDriverClass().toLowerCase().contains("mysql") || getDriverClass().toLowerCase().contains("clickhouse") ||
|
||||
getDriverClass().toLowerCase().contains("teradata")) {
|
||||
return config.get(JdbcUtils.DATABASE_KEY).asText();
|
||||
} else {
|
||||
return SCHEMA_NAME;
|
||||
}
|
||||
return switch (testdb.getDatabaseDriver()) {
|
||||
// mysql does not support schemas, it namespaces using database names instead.
|
||||
case MYSQL, CLICKHOUSE, TERADATA -> testdb.getDatabaseName();
|
||||
default -> SCHEMA_NAME;
|
||||
};
|
||||
}
|
||||
|
||||
protected static void setEmittedAtToNull(final Iterable<AirbyteMessage> messages) {
|
||||
|
||||
@@ -0,0 +1,118 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.cdk.testutils;
|
||||
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.testcontainers.containers.JdbcDatabaseContainer;
|
||||
import org.testcontainers.containers.output.Slf4jLogConsumer;
|
||||
import org.testcontainers.utility.DockerImageName;
|
||||
|
||||
/**
|
||||
* ContainerFactory is the companion interface to {@link TestDatabase} for providing it with
|
||||
* suitable testcontainer instances.
|
||||
*/
|
||||
public interface ContainerFactory<C extends JdbcDatabaseContainer<?>> {
|
||||
|
||||
/**
|
||||
* Creates a new, unshared testcontainer instance. This usually wraps the default constructor for
|
||||
* the testcontainer type.
|
||||
*/
|
||||
C createNewContainer(DockerImageName imageName);
|
||||
|
||||
/**
|
||||
* Returns the class object of the testcontainer.
|
||||
*/
|
||||
Class<?> getContainerClass();
|
||||
|
||||
/**
|
||||
* Returns a shared instance of the testcontainer.
|
||||
*/
|
||||
default C shared(String imageName, String... methods) {
|
||||
final String mapKey = Stream.concat(
|
||||
Stream.of(imageName, this.getClass().getCanonicalName()),
|
||||
Stream.of(methods))
|
||||
.collect(Collectors.joining("+"));
|
||||
return Singleton.getOrCreate(mapKey, this);
|
||||
}
|
||||
|
||||
/**
|
||||
* This class is exclusively used by {@link #shared(String, String...)}. It wraps a specific shared
|
||||
* testcontainer instance, which is created exactly once.
|
||||
*/
|
||||
class Singleton<C extends JdbcDatabaseContainer<?>> {
|
||||
|
||||
static private final Logger LOGGER = LoggerFactory.getLogger(Singleton.class);
|
||||
static private final ConcurrentHashMap<String, Singleton<?>> LAZY = new ConcurrentHashMap<>();
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
static private <C extends JdbcDatabaseContainer<?>> C getOrCreate(String mapKey, ContainerFactory<C> factory) {
|
||||
final Singleton<?> singleton = LAZY.computeIfAbsent(mapKey, Singleton<C>::new);
|
||||
return ((Singleton<C>) singleton).getOrCreate(factory);
|
||||
}
|
||||
|
||||
final private String imageName;
|
||||
final private List<String> methodNames;
|
||||
|
||||
private C sharedContainer;
|
||||
private RuntimeException containerCreationError;
|
||||
|
||||
private Singleton(String imageNamePlusMethods) {
|
||||
final String[] parts = imageNamePlusMethods.split("\\+");
|
||||
this.imageName = parts[0];
|
||||
this.methodNames = Arrays.stream(parts).skip(2).toList();
|
||||
}
|
||||
|
||||
private synchronized C getOrCreate(ContainerFactory<C> factory) {
|
||||
if (sharedContainer == null && containerCreationError == null) {
|
||||
try {
|
||||
create(imageName, factory, methodNames);
|
||||
} catch (RuntimeException e) {
|
||||
sharedContainer = null;
|
||||
containerCreationError = e;
|
||||
}
|
||||
}
|
||||
if (containerCreationError != null) {
|
||||
throw new RuntimeException(
|
||||
"Error during container creation for imageName=" + imageName
|
||||
+ ", factory=" + factory.getClass().getName()
|
||||
+ ", methods=" + methodNames,
|
||||
containerCreationError);
|
||||
}
|
||||
return sharedContainer;
|
||||
}
|
||||
|
||||
private void create(String imageName, ContainerFactory<C> factory, List<String> methodNames) {
|
||||
LOGGER.info("Creating new shared container based on {} with {}.", imageName, methodNames);
|
||||
try {
|
||||
final var parsed = DockerImageName.parse(imageName);
|
||||
final var methods = new ArrayList<Method>();
|
||||
for (String methodName : methodNames) {
|
||||
methods.add(factory.getClass().getMethod(methodName, factory.getContainerClass()));
|
||||
}
|
||||
sharedContainer = factory.createNewContainer(parsed);
|
||||
sharedContainer.withLogConsumer(new Slf4jLogConsumer(LOGGER));
|
||||
for (Method method : methods) {
|
||||
LOGGER.info("Calling {} in {} on new shared container based on {}.",
|
||||
method.getName(), factory.getClass().getName(), imageName);
|
||||
method.invoke(factory, sharedContainer);
|
||||
}
|
||||
sharedContainer.start();
|
||||
} catch (IllegalAccessException | InvocationTargetException | NoSuchMethodException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
@@ -1,304 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.cdk.testutils;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.PostgresUtils;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.util.HostPortResolver;
|
||||
import io.airbyte.commons.string.Strings;
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.jooq.DSLContext;
|
||||
import org.jooq.SQLDialect;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.testcontainers.containers.Network;
|
||||
import org.testcontainers.containers.PostgreSQLContainer;
|
||||
import org.testcontainers.utility.DockerImageName;
|
||||
import org.testcontainers.utility.MountableFile;
|
||||
|
||||
/**
|
||||
* {@link PostgresTestDatabase} is a convenience object which allows for efficient use of
|
||||
* {@link PostgreSQLContainer} instances in tests. Each test container is shared throughout the
|
||||
* whole JVM. Isolation is performed by creating a new database and a new user for each
|
||||
* {@link PostgresTestDatabase} instance. These are dropped when the instance is closed.
|
||||
*/
|
||||
public class PostgresTestDatabase implements AutoCloseable {
|
||||
|
||||
static private final Logger LOGGER = LoggerFactory.getLogger(PostgresTestDatabase.class);
|
||||
|
||||
/**
|
||||
* Create a new {@link PostgresTestDatabase} instance.
|
||||
*
|
||||
* @param imageName base image to use for the underlying {@link PostgreSQLContainer}.
|
||||
* @param methods {@link ContainerFactory} methods that need to be called.
|
||||
* @return a new {@link PostgresTestDatabase} instance which may reuse a shared
|
||||
* {@link PostgreSQLContainer}.
|
||||
*/
|
||||
static public PostgresTestDatabase make(String imageName, String... methods) {
|
||||
final String imageNamePlusMethods = Stream.concat(
|
||||
Stream.of(imageName),
|
||||
Stream.of(methods))
|
||||
.collect(Collectors.joining("+"));
|
||||
final ContainerFactory factory = ContainerFactory.LAZY.computeIfAbsent(imageNamePlusMethods, ContainerFactory::new);
|
||||
return new PostgresTestDatabase(factory.getOrCreateSharedContainer());
|
||||
}
|
||||
|
||||
private PostgresTestDatabase(PostgreSQLContainer<?> sharedContainer) {
|
||||
this.container = sharedContainer;
|
||||
this.suffix = Strings.addRandomSuffix("", "_", 10);
|
||||
this.dbName = "db" + suffix;
|
||||
this.userName = "test_user" + suffix;
|
||||
this.password = "test_password" + suffix;
|
||||
execSQL(
|
||||
String.format("CREATE DATABASE %s", dbName),
|
||||
String.format("CREATE USER %s PASSWORD '%s'", userName, password),
|
||||
String.format("GRANT ALL PRIVILEGES ON DATABASE %s TO %s", dbName, userName),
|
||||
String.format("ALTER USER %s WITH SUPERUSER", userName));
|
||||
|
||||
this.jdbcUrl = String.format(
|
||||
DatabaseDriver.POSTGRESQL.getUrlFormatString(),
|
||||
sharedContainer.getHost(),
|
||||
sharedContainer.getFirstMappedPort(),
|
||||
dbName);
|
||||
this.dslContext = DSLContextFactory.create(
|
||||
userName,
|
||||
password,
|
||||
DatabaseDriver.POSTGRESQL.getDriverClassName(),
|
||||
jdbcUrl,
|
||||
SQLDialect.POSTGRES);
|
||||
this.database = new Database(dslContext);
|
||||
}
|
||||
|
||||
public final PostgreSQLContainer<?> container;
|
||||
public final String suffix, dbName, userName, password, jdbcUrl;
|
||||
public final DSLContext dslContext;
|
||||
public final Database database;
|
||||
|
||||
/**
|
||||
* Convenience method for building identifiers which are unique to this instance.
|
||||
*/
|
||||
public String withSuffix(String str) {
|
||||
return str + suffix;
|
||||
}
|
||||
|
||||
/**
|
||||
* Convenience method for initializing a config builder for use in integration tests.
|
||||
*/
|
||||
public ImmutableMap.Builder<Object, Object> makeConfigBuilder() {
|
||||
return ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, HostPortResolver.resolveHost(container))
|
||||
.put(JdbcUtils.PORT_KEY, HostPortResolver.resolvePort(container))
|
||||
.put(JdbcUtils.DATABASE_KEY, dbName)
|
||||
.put(JdbcUtils.USERNAME_KEY, userName)
|
||||
.put(JdbcUtils.PASSWORD_KEY, password);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the {@link PostgresUtils.Certificate} for this instance; requires
|
||||
* {@link ContainerFactory#withCert} call.
|
||||
*/
|
||||
public PostgresUtils.Certificate getCertificate() {
|
||||
final String caCert, clientKey, clientCert;
|
||||
try {
|
||||
caCert = container.execInContainer("su", "-c", "cat ca.crt").getStdout().trim();
|
||||
clientKey = container.execInContainer("su", "-c", "cat client.key").getStdout().trim();
|
||||
clientCert = container.execInContainer("su", "-c", "cat client.crt").getStdout().trim();
|
||||
} catch (IOException e) {
|
||||
throw new UncheckedIOException(e);
|
||||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
return new PostgresUtils.Certificate(caCert, clientCert, clientKey);
|
||||
}
|
||||
|
||||
private void execSQL(String... stmts) {
|
||||
final List<String> cmd = Stream.concat(
|
||||
Stream.of("psql", "-a", "-d", container.getDatabaseName(), "-U", container.getUsername()),
|
||||
Stream.of(stmts).flatMap(stmt -> Stream.of("-c", stmt)))
|
||||
.toList();
|
||||
try {
|
||||
LOGGER.debug("executing {}", Strings.join(cmd, " "));
|
||||
final var exec = container.execInContainer(cmd.toArray(new String[0]));
|
||||
LOGGER.debug("exit code: {}\nstdout:\n{}\nstderr:\n{}", exec.getExitCode(), exec.getStdout(), exec.getStderr());
|
||||
} catch (IOException e) {
|
||||
throw new UncheckedIOException(e);
|
||||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Drop the database owned by this instance.
|
||||
*/
|
||||
public void dropDatabase() {
|
||||
execSQL(String.format("DROP DATABASE %s", dbName));
|
||||
}
|
||||
|
||||
/**
|
||||
* Close resources held by this instance. This deliberately avoids dropping the database, which is
|
||||
* really expensive in Postgres. This is because a DROP DATABASE in Postgres triggers a CHECKPOINT.
|
||||
* Call {@link #dropDatabase} to explicitly drop the database.
|
||||
*/
|
||||
@Override
|
||||
public void close() {
|
||||
dslContext.close();
|
||||
execSQL(String.format("DROP USER %s", userName));
|
||||
}
|
||||
|
||||
static private class ContainerFactory {
|
||||
|
||||
static private final Logger LOGGER = LoggerFactory.getLogger(ContainerFactory.class);
|
||||
static private final ConcurrentHashMap<String, ContainerFactory> LAZY = new ConcurrentHashMap<>();
|
||||
|
||||
final private String imageName;
|
||||
final private List<Method> methods;
|
||||
private PostgreSQLContainer<?> sharedContainer;
|
||||
private RuntimeException containerCreationError;
|
||||
|
||||
private ContainerFactory(String imageNamePlusMethods) {
|
||||
final String[] parts = imageNamePlusMethods.split("\\+");
|
||||
this.imageName = parts[0];
|
||||
this.methods = Arrays.stream(parts).skip(1).map(methodName -> {
|
||||
try {
|
||||
return ContainerFactory.class.getMethod(methodName);
|
||||
} catch (NoSuchMethodException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}).toList();
|
||||
}
|
||||
|
||||
private synchronized PostgreSQLContainer<?> getOrCreateSharedContainer() {
|
||||
if (sharedContainer == null) {
|
||||
if (containerCreationError != null) {
|
||||
throw new RuntimeException(
|
||||
"Error during container creation for imageName=" + imageName + ", methods=" + methods.stream().map(Method::getName).toList(),
|
||||
containerCreationError);
|
||||
}
|
||||
LOGGER.info("Creating new shared container based on {} with {}.", imageName, methods.stream().map(Method::getName).toList());
|
||||
try {
|
||||
final var parsed = DockerImageName.parse(imageName).asCompatibleSubstituteFor("postgres");
|
||||
sharedContainer = new PostgreSQLContainer<>(parsed);
|
||||
for (Method method : methods) {
|
||||
LOGGER.info("Calling {} on new shared container based on {}.", method.getName(),
|
||||
imageName);
|
||||
method.invoke(this);
|
||||
}
|
||||
sharedContainer.start();
|
||||
} catch (IllegalAccessException | InvocationTargetException e) {
|
||||
containerCreationError = new RuntimeException(e);
|
||||
this.sharedContainer = null;
|
||||
throw containerCreationError;
|
||||
} catch (RuntimeException e) {
|
||||
this.sharedContainer = null;
|
||||
containerCreationError = e;
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
return sharedContainer;
|
||||
}
|
||||
|
||||
/**
|
||||
* Apply the postgresql.conf file that we've packaged as a resource.
|
||||
*/
|
||||
public void withConf() {
|
||||
sharedContainer
|
||||
.withCopyFileToContainer(
|
||||
MountableFile.forClasspathResource("postgresql.conf"),
|
||||
"/etc/postgresql/postgresql.conf")
|
||||
.withCommand("postgres -c config_file=/etc/postgresql/postgresql.conf");
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new network and bind it to the container.
|
||||
*/
|
||||
public void withNetwork() {
|
||||
sharedContainer.withNetwork(Network.newNetwork());
|
||||
}
|
||||
|
||||
/**
|
||||
* Configure postgres with wal_level=logical.
|
||||
*/
|
||||
public void withWalLevelLogical() {
|
||||
sharedContainer.withCommand("postgres -c wal_level=logical");
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate SSL certificates and tell postgres to enable SSL and use them.
|
||||
*/
|
||||
public void withCert() {
|
||||
sharedContainer.start();
|
||||
String[] commands = {
|
||||
"psql -U test -c \"CREATE USER postgres WITH PASSWORD 'postgres';\"",
|
||||
"psql -U test -c \"GRANT CONNECT ON DATABASE \"test\" TO postgres;\"",
|
||||
"psql -U test -c \"ALTER USER postgres WITH SUPERUSER;\"",
|
||||
"openssl ecparam -name prime256v1 -genkey -noout -out ca.key",
|
||||
"openssl req -new -x509 -sha256 -key ca.key -out ca.crt -subj \"/CN=127.0.0.1\"",
|
||||
"openssl ecparam -name prime256v1 -genkey -noout -out server.key",
|
||||
"openssl req -new -sha256 -key server.key -out server.csr -subj \"/CN=localhost\"",
|
||||
"openssl x509 -req -in server.csr -CA ca.crt -CAkey ca.key -CAcreateserial -out server.crt -days 365 -sha256",
|
||||
"cp server.key /etc/ssl/private/",
|
||||
"cp server.crt /etc/ssl/private/",
|
||||
"cp ca.crt /etc/ssl/private/",
|
||||
"chmod og-rwx /etc/ssl/private/server.* /etc/ssl/private/ca.*",
|
||||
"chown postgres:postgres /etc/ssl/private/server.crt /etc/ssl/private/server.key /etc/ssl/private/ca.crt",
|
||||
"echo \"ssl = on\" >> /var/lib/postgresql/data/postgresql.conf",
|
||||
"echo \"ssl_cert_file = '/etc/ssl/private/server.crt'\" >> /var/lib/postgresql/data/postgresql.conf",
|
||||
"echo \"ssl_key_file = '/etc/ssl/private/server.key'\" >> /var/lib/postgresql/data/postgresql.conf",
|
||||
"echo \"ssl_ca_file = '/etc/ssl/private/ca.crt'\" >> /var/lib/postgresql/data/postgresql.conf",
|
||||
"mkdir root/.postgresql",
|
||||
"echo \"hostssl all all 127.0.0.1/32 cert clientcert=verify-full\" >> /var/lib/postgresql/data/pg_hba.conf",
|
||||
"openssl ecparam -name prime256v1 -genkey -noout -out client.key",
|
||||
"openssl req -new -sha256 -key client.key -out client.csr -subj \"/CN=postgres\"",
|
||||
"openssl x509 -req -in client.csr -CA ca.crt -CAkey ca.key -CAcreateserial -out client.crt -days 365 -sha256",
|
||||
"cp client.crt ~/.postgresql/postgresql.crt",
|
||||
"cp client.key ~/.postgresql/postgresql.key",
|
||||
"chmod 0600 ~/.postgresql/postgresql.crt ~/.postgresql/postgresql.key",
|
||||
"cp ca.crt root/.postgresql/ca.crt",
|
||||
"chown postgres:postgres ~/.postgresql/ca.crt",
|
||||
"psql -U test -c \"SELECT pg_reload_conf();\"",
|
||||
};
|
||||
for (String cmd : commands) {
|
||||
try {
|
||||
sharedContainer.execInContainer("su", "-c", cmd);
|
||||
} catch (IOException e) {
|
||||
throw new UncheckedIOException(e);
|
||||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Tell postgres to enable SSL.
|
||||
*/
|
||||
public void withSSL() {
|
||||
sharedContainer.withCommand("postgres " +
|
||||
"-c ssl=on " +
|
||||
"-c ssl_cert_file=/var/lib/postgresql/server.crt " +
|
||||
"-c ssl_key_file=/var/lib/postgresql/server.key");
|
||||
}
|
||||
|
||||
/**
|
||||
* Configure postgres with client_encoding=sql_ascii.
|
||||
*/
|
||||
public void withASCII() {
|
||||
sharedContainer.withCommand("postgres -c client_encoding=sql_ascii");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,293 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.cdk.testutils;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.airbyte.cdk.db.ContextQueryFunction;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DataSourceFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.util.HostPortResolver;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.commons.string.Strings;
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.sql.SQLException;
|
||||
import java.time.Duration;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Stream;
|
||||
import javax.sql.DataSource;
|
||||
import org.jooq.DSLContext;
|
||||
import org.jooq.SQLDialect;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.testcontainers.containers.JdbcDatabaseContainer;
|
||||
|
||||
/**
|
||||
* TestDatabase provides a convenient pattern for interacting with databases when testing SQL
|
||||
* database sources. The basic idea is to share the same database testcontainer instance for all
|
||||
* tests and to use SQL constructs such as DATABASE and USER to isolate each test case's state.
|
||||
*
|
||||
* @param <C> the type of the backing testcontainer.
|
||||
* @param <T> itself
|
||||
* @param <B> the type of the object returned by {@link #configBuilder()}
|
||||
*/
|
||||
abstract public class TestDatabase<C extends JdbcDatabaseContainer<?>, T extends TestDatabase<C, T, B>, B extends TestDatabase.ConfigBuilder<T, B>>
|
||||
implements AutoCloseable {
|
||||
|
||||
static private final Logger LOGGER = LoggerFactory.getLogger(TestDatabase.class);
|
||||
|
||||
final private C container;
|
||||
final private String suffix;
|
||||
final private ArrayList<String> cleanupSQL = new ArrayList<>();
|
||||
final private Map<String, String> connectionProperties = new HashMap<>();
|
||||
|
||||
private DataSource dataSource;
|
||||
private DSLContext dslContext;
|
||||
|
||||
protected TestDatabase(C container) {
|
||||
this.container = container;
|
||||
this.suffix = Strings.addRandomSuffix("", "_", 10);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
protected T self() {
|
||||
return (T) this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a key-value pair to the JDBC URL's query parameters.
|
||||
*/
|
||||
public T withConnectionProperty(String key, String value) {
|
||||
if (isInitialized()) {
|
||||
throw new RuntimeException("TestDatabase instance is already initialized");
|
||||
}
|
||||
connectionProperties.put(key, value);
|
||||
return self();
|
||||
}
|
||||
|
||||
/**
|
||||
* Enqueues a SQL statement to be executed when this object is closed.
|
||||
*/
|
||||
public T onClose(String fmtSql, Object... fmtArgs) {
|
||||
cleanupSQL.add(String.format(fmtSql, fmtArgs));
|
||||
return self();
|
||||
}
|
||||
|
||||
/**
|
||||
* Executes a SQL statement after calling String.format on the arguments.
|
||||
*/
|
||||
public T with(String fmtSql, Object... fmtArgs) {
|
||||
execSQL(Stream.of(String.format(fmtSql, fmtArgs)));
|
||||
return self();
|
||||
}
|
||||
|
||||
/**
|
||||
* Executes SQL statements as root to provide the necessary isolation for the lifetime of this
|
||||
* object. This typically entails at least a CREATE DATABASE and a CREATE USER. Also Initializes the
|
||||
* {@link DataSource} and {@link DSLContext} owned by this object.
|
||||
*/
|
||||
final public T initialized() {
|
||||
inContainerBootstrapCmd().forEach(this::execInContainer);
|
||||
this.dataSource = DataSourceFactory.create(
|
||||
getUserName(),
|
||||
getPassword(),
|
||||
getDatabaseDriver().getDriverClassName(),
|
||||
getJdbcUrl(),
|
||||
connectionProperties);
|
||||
this.dslContext = DSLContextFactory.create(dataSource, getSqlDialect());
|
||||
return self();
|
||||
}
|
||||
|
||||
final public boolean isInitialized() {
|
||||
return dslContext != null;
|
||||
}
|
||||
|
||||
abstract protected Stream<Stream<String>> inContainerBootstrapCmd();
|
||||
|
||||
abstract protected Stream<String> inContainerUndoBootstrapCmd();
|
||||
|
||||
abstract public DatabaseDriver getDatabaseDriver();
|
||||
|
||||
abstract public SQLDialect getSqlDialect();
|
||||
|
||||
final public C getContainer() {
|
||||
return container;
|
||||
}
|
||||
|
||||
public String withNamespace(String name) {
|
||||
return name + suffix;
|
||||
}
|
||||
|
||||
public String getDatabaseName() {
|
||||
return withNamespace("db");
|
||||
}
|
||||
|
||||
public String getUserName() {
|
||||
return withNamespace("user");
|
||||
}
|
||||
|
||||
public String getPassword() {
|
||||
return "password";
|
||||
}
|
||||
|
||||
public DataSource getDataSource() {
|
||||
if (!isInitialized()) {
|
||||
throw new RuntimeException("TestDatabase instance is not yet initialized");
|
||||
}
|
||||
return dataSource;
|
||||
}
|
||||
|
||||
final public DSLContext getDslContext() {
|
||||
if (!isInitialized()) {
|
||||
throw new RuntimeException("TestDatabase instance is not yet initialized");
|
||||
}
|
||||
return dslContext;
|
||||
}
|
||||
|
||||
public String getJdbcUrl() {
|
||||
return String.format(
|
||||
getDatabaseDriver().getUrlFormatString(),
|
||||
getContainer().getHost(),
|
||||
getContainer().getFirstMappedPort(),
|
||||
getDatabaseName());
|
||||
}
|
||||
|
||||
public Database getDatabase() {
|
||||
return new Database(getDslContext());
|
||||
}
|
||||
|
||||
protected void execSQL(Stream<String> sql) {
|
||||
try {
|
||||
getDatabase().query(ctx -> {
|
||||
sql.forEach(ctx::execute);
|
||||
return null;
|
||||
});
|
||||
} catch (SQLException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
protected void execInContainer(Stream<String> cmds) {
|
||||
final List<String> cmd = cmds.toList();
|
||||
if (cmd.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
try {
|
||||
LOGGER.debug("executing {}", Strings.join(cmd, " "));
|
||||
final var exec = getContainer().execInContainer(cmd.toArray(new String[0]));
|
||||
if (exec.getExitCode() == 0) {
|
||||
LOGGER.debug("execution success\nstdout:\n{}\nstderr:\n{}", exec.getStdout(), exec.getStderr());
|
||||
} else {
|
||||
LOGGER.error("execution failure, code {}\nstdout:\n{}\nstderr:\n{}", exec.getExitCode(), exec.getStdout(), exec.getStderr());
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new UncheckedIOException(e);
|
||||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public <X> X query(final ContextQueryFunction<X> transform) throws SQLException {
|
||||
return getDatabase().query(transform);
|
||||
}
|
||||
|
||||
public <X> X transaction(final ContextQueryFunction<X> transform) throws SQLException {
|
||||
return getDatabase().transaction(transform);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a builder for the connector config object.
|
||||
*/
|
||||
public B configBuilder() {
|
||||
return new ConfigBuilder<T, B>(self()).self();
|
||||
}
|
||||
|
||||
public B testConfigBuilder() {
|
||||
return configBuilder()
|
||||
.withHostAndPort()
|
||||
.withCredentials()
|
||||
.withDatabase();
|
||||
}
|
||||
|
||||
public B integrationTestConfigBuilder() {
|
||||
return configBuilder()
|
||||
.withResolvedHostAndPort()
|
||||
.withCredentials()
|
||||
.withDatabase();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
execSQL(this.cleanupSQL.stream());
|
||||
dslContext.close();
|
||||
execInContainer(inContainerUndoBootstrapCmd());
|
||||
}
|
||||
|
||||
static public class ConfigBuilder<T extends TestDatabase<?, ?, ?>, B extends ConfigBuilder<T, B>> {
|
||||
|
||||
static public final Duration DEFAULT_CDC_REPLICATION_INITIAL_WAIT = Duration.ofSeconds(5);
|
||||
|
||||
protected final ImmutableMap.Builder<Object, Object> builder = ImmutableMap.builder();
|
||||
protected final T testDatabase;
|
||||
|
||||
protected ConfigBuilder(T testDatabase) {
|
||||
this.testDatabase = testDatabase;
|
||||
}
|
||||
|
||||
public JsonNode build() {
|
||||
return Jsons.jsonNode(builder.build());
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
final protected B self() {
|
||||
return (B) this;
|
||||
}
|
||||
|
||||
public B with(Object key, Object value) {
|
||||
builder.put(key, value);
|
||||
return self();
|
||||
}
|
||||
|
||||
public B withDatabase() {
|
||||
return this
|
||||
.with(JdbcUtils.DATABASE_KEY, testDatabase.getDatabaseName());
|
||||
}
|
||||
|
||||
public B withCredentials() {
|
||||
return this
|
||||
.with(JdbcUtils.USERNAME_KEY, testDatabase.getUserName())
|
||||
.with(JdbcUtils.PASSWORD_KEY, testDatabase.getPassword());
|
||||
}
|
||||
|
||||
public B withResolvedHostAndPort() {
|
||||
return this
|
||||
.with(JdbcUtils.HOST_KEY, HostPortResolver.resolveHost(testDatabase.getContainer()))
|
||||
.with(JdbcUtils.PORT_KEY, HostPortResolver.resolvePort(testDatabase.getContainer()));
|
||||
}
|
||||
|
||||
public B withHostAndPort() {
|
||||
return this
|
||||
.with(JdbcUtils.HOST_KEY, testDatabase.getContainer().getHost())
|
||||
.with(JdbcUtils.PORT_KEY, testDatabase.getContainer().getFirstMappedPort());
|
||||
}
|
||||
|
||||
public B withoutSsl() {
|
||||
return with(JdbcUtils.SSL_KEY, false);
|
||||
}
|
||||
|
||||
public B withSsl(Map<Object, Object> sslMode) {
|
||||
return with(JdbcUtils.SSL_KEY, true).with(JdbcUtils.SSL_MODE_KEY, sslMode);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
@@ -4,13 +4,11 @@ plugins {
|
||||
}
|
||||
|
||||
airbyteJavaConnector {
|
||||
cdkVersionRequired = '0.4.1'
|
||||
cdkVersionRequired = '0.5.0'
|
||||
features = ['db-sources']
|
||||
useLocalCdk = false
|
||||
}
|
||||
|
||||
airbyteJavaConnector.addCdkDependencies()
|
||||
|
||||
configurations.all {
|
||||
resolutionStrategy {
|
||||
force libs.jooq
|
||||
@@ -26,6 +24,7 @@ dependencies {
|
||||
implementation project(':airbyte-integrations:connectors:source-mssql')
|
||||
implementation libs.jooq
|
||||
|
||||
testImplementation testFixtures(project(':airbyte-integrations:connectors:source-mssql'))
|
||||
testImplementation 'org.apache.commons:commons-lang3:3.11'
|
||||
testImplementation libs.testcontainers.mssqlserver
|
||||
testImplementation 'org.hamcrest:hamcrest-all:1.3'
|
||||
|
||||
@@ -0,0 +1 @@
|
||||
testExecutionConcurrency=-1
|
||||
@@ -11,7 +11,7 @@ data:
|
||||
connectorSubtype: database
|
||||
connectorType: source
|
||||
definitionId: b5ea17b1-f170-46dc-bc31-cc744ca984c1
|
||||
dockerImageTag: 3.0.0
|
||||
dockerImageTag: 3.0.1
|
||||
dockerRepository: airbyte/source-mssql-strict-encrypt
|
||||
githubIssueLabel: source-mssql
|
||||
icon: mssql.svg
|
||||
|
||||
@@ -19,7 +19,7 @@ public class MssqlSourceStrictEncrypt extends SpecModifyingSource implements Sou
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(MssqlSourceStrictEncrypt.class);
|
||||
|
||||
public MssqlSourceStrictEncrypt() {
|
||||
super(MssqlSource.sshWrappedSource());
|
||||
super(MssqlSource.sshWrappedSource(new MssqlSource()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -5,95 +5,49 @@
|
||||
package io.airbyte.integrations.source.mssql_strict_encrypt;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
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.cdk.integrations.util.HostPortResolver;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.commons.resources.MoreResources;
|
||||
import io.airbyte.integrations.source.mssql.MsSQLContainerFactory;
|
||||
import io.airbyte.integrations.source.mssql.MsSQLTestDatabase;
|
||||
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.ConnectorSpecification;
|
||||
import java.sql.SQLException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import org.apache.commons.lang3.RandomStringUtils;
|
||||
import org.jooq.DSLContext;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.testcontainers.containers.MSSQLServerContainer;
|
||||
|
||||
public class MssqlStrictEncryptSourceAcceptanceTest extends SourceAcceptanceTest {
|
||||
|
||||
protected static final String SCHEMA_NAME = "dbo";
|
||||
protected static final String STREAM_NAME = "id_and_name";
|
||||
protected static MSSQLServerContainer<?> db;
|
||||
protected JsonNode config;
|
||||
|
||||
@AfterAll
|
||||
public static void closeContainer() {
|
||||
if (db != null) {
|
||||
db.close();
|
||||
db.stop();
|
||||
}
|
||||
private MsSQLTestDatabase testdb;
|
||||
|
||||
@Override
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) {
|
||||
final var container = new MsSQLContainerFactory().shared("mcr.microsoft.com/mssql/server:2022-RTM-CU2-ubuntu-20.04");
|
||||
testdb = new MsSQLTestDatabase(container);
|
||||
testdb = testdb
|
||||
.withConnectionProperty("encrypt", "true")
|
||||
.withConnectionProperty("trustServerCertificate", "true")
|
||||
.withConnectionProperty("databaseName", testdb.getDatabaseName())
|
||||
.initialized()
|
||||
.with("CREATE TABLE id_and_name(id INTEGER, name VARCHAR(200), born DATETIMEOFFSET(7));")
|
||||
.with("INSERT INTO id_and_name (id, name, born) VALUES " +
|
||||
"(1,'picard', '2124-03-04T01:01:01Z'), " +
|
||||
"(2, 'crusher', '2124-03-04T01:01:01Z'), " +
|
||||
"(3, 'vash', '2124-03-04T01:01:01Z');");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) throws SQLException {
|
||||
if (db == null) {
|
||||
db = new MSSQLServerContainer<>("mcr.microsoft.com/mssql/server:2022-RTM-CU2-ubuntu-20.04").acceptLicense();
|
||||
db.start();
|
||||
}
|
||||
|
||||
final JsonNode configWithoutDbName = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, HostPortResolver.resolveHost(db))
|
||||
.put(JdbcUtils.PORT_KEY, HostPortResolver.resolvePort(db))
|
||||
.put(JdbcUtils.USERNAME_KEY, db.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, db.getPassword())
|
||||
.build());
|
||||
final String dbName = "db_" + RandomStringUtils.randomAlphabetic(10).toLowerCase();
|
||||
|
||||
try (final DSLContext dslContext = DSLContextFactory.create(
|
||||
configWithoutDbName.get(JdbcUtils.USERNAME_KEY).asText(),
|
||||
configWithoutDbName.get(JdbcUtils.PASSWORD_KEY).asText(),
|
||||
DatabaseDriver.MSSQLSERVER.getDriverClassName(),
|
||||
String.format("jdbc:sqlserver://%s:%d;encrypt=true;trustServerCertificate=true;",
|
||||
db.getHost(),
|
||||
db.getFirstMappedPort()),
|
||||
null)) {
|
||||
final Database database = getDatabase(dslContext);
|
||||
database.query(ctx -> {
|
||||
ctx.fetch(String.format("CREATE DATABASE %s;", dbName));
|
||||
ctx.fetch(String.format("USE %s;", dbName));
|
||||
ctx.fetch("CREATE TABLE id_and_name(id INTEGER, name VARCHAR(200), born DATETIMEOFFSET(7));");
|
||||
ctx.fetch(
|
||||
"INSERT INTO id_and_name (id, name, born) VALUES " +
|
||||
"(1,'picard', '2124-03-04T01:01:01Z'), " +
|
||||
"(2, 'crusher', '2124-03-04T01:01:01Z'), " +
|
||||
"(3, 'vash', '2124-03-04T01:01:01Z');");
|
||||
return null;
|
||||
});
|
||||
}
|
||||
|
||||
config = Jsons.clone(configWithoutDbName);
|
||||
((ObjectNode) config).put(JdbcUtils.DATABASE_KEY, dbName);
|
||||
((ObjectNode) config).put("ssl_method", Jsons.jsonNode(Map.of("ssl_method", "encrypted_trust_server_certificate")));
|
||||
protected void tearDown(final TestDestinationEnv testEnv) {
|
||||
testdb.close();
|
||||
}
|
||||
|
||||
private static Database getDatabase(final DSLContext dslContext) {
|
||||
return new Database(dslContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void tearDown(final TestDestinationEnv testEnv) throws Exception {}
|
||||
|
||||
@Override
|
||||
protected String getImageName() {
|
||||
return "airbyte/source-mssql-strict-encrypt:dev";
|
||||
@@ -106,7 +60,9 @@ public class MssqlStrictEncryptSourceAcceptanceTest extends SourceAcceptanceTest
|
||||
|
||||
@Override
|
||||
protected JsonNode getConfig() {
|
||||
return config;
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.withSsl(Map.of("ssl_method", "encrypted_trust_server_certificate"))
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -8,92 +8,57 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.airbyte.cdk.db.factory.DataSourceFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
import io.airbyte.cdk.db.jdbc.DefaultJdbcDatabase;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.base.Source;
|
||||
import io.airbyte.cdk.integrations.base.ssh.SshHelpers;
|
||||
import io.airbyte.cdk.integrations.source.jdbc.AbstractJdbcSource;
|
||||
import io.airbyte.cdk.integrations.source.jdbc.test.JdbcSourceAcceptanceTest;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.commons.resources.MoreResources;
|
||||
import io.airbyte.commons.string.Strings;
|
||||
import io.airbyte.integrations.source.mssql.MssqlSource;
|
||||
import io.airbyte.integrations.source.mssql.MsSQLContainerFactory;
|
||||
import io.airbyte.integrations.source.mssql.MsSQLTestDatabase;
|
||||
import io.airbyte.protocol.models.Field;
|
||||
import io.airbyte.protocol.models.JsonSchemaType;
|
||||
import io.airbyte.protocol.models.v0.AirbyteCatalog;
|
||||
import io.airbyte.protocol.models.v0.CatalogHelpers;
|
||||
import io.airbyte.protocol.models.v0.ConnectorSpecification;
|
||||
import io.airbyte.protocol.models.v0.SyncMode;
|
||||
import java.sql.JDBCType;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
import javax.sql.DataSource;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.testcontainers.containers.MSSQLServerContainer;
|
||||
|
||||
public class MssqlStrictEncryptJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest {
|
||||
public class MssqlStrictEncryptJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest<MssqlSourceStrictEncrypt, MsSQLTestDatabase> {
|
||||
|
||||
private static MSSQLServerContainer<?> dbContainer;
|
||||
private static DataSource dataSource;
|
||||
private JsonNode config;
|
||||
|
||||
@BeforeAll
|
||||
static void init() {
|
||||
static {
|
||||
// In mssql, timestamp is generated automatically, so we need to use
|
||||
// the datetime type instead so that we can set the value manually.
|
||||
COL_TIMESTAMP_TYPE = "DATETIME";
|
||||
|
||||
if (dbContainer == null) {
|
||||
dbContainer = new MSSQLServerContainer<>("mcr.microsoft.com/mssql/server:2022-RTM-CU2-ubuntu-20.04").acceptLicense();
|
||||
dbContainer.start();
|
||||
}
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
public void setup() throws Exception {
|
||||
final JsonNode configWithoutDbName = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, dbContainer.getHost())
|
||||
.put(JdbcUtils.PORT_KEY, dbContainer.getFirstMappedPort())
|
||||
.put(JdbcUtils.USERNAME_KEY, dbContainer.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, dbContainer.getPassword())
|
||||
.build());
|
||||
|
||||
dataSource = DataSourceFactory.create(
|
||||
configWithoutDbName.get(JdbcUtils.USERNAME_KEY).asText(),
|
||||
configWithoutDbName.get(JdbcUtils.PASSWORD_KEY).asText(),
|
||||
DatabaseDriver.MSSQLSERVER.getDriverClassName(),
|
||||
String.format("jdbc:sqlserver://%s:%d;encrypt=true;trustServerCertificate=true;",
|
||||
dbContainer.getHost(),
|
||||
dbContainer.getFirstMappedPort()));
|
||||
|
||||
try {
|
||||
database = new DefaultJdbcDatabase(dataSource);
|
||||
|
||||
final String dbName = Strings.addRandomSuffix("db", "_", 10).toLowerCase();
|
||||
|
||||
database.execute(ctx -> ctx.createStatement().execute(String.format("CREATE DATABASE %s;", dbName)));
|
||||
|
||||
config = Jsons.clone(configWithoutDbName);
|
||||
((ObjectNode) config).put(JdbcUtils.DATABASE_KEY, dbName);
|
||||
((ObjectNode) config).put("ssl_method", Jsons.jsonNode(Map.of("ssl_method", "encrypted_trust_server_certificate")));
|
||||
|
||||
super.setup();
|
||||
} finally {
|
||||
DataSourceFactory.close(dataSource);
|
||||
}
|
||||
@Override
|
||||
protected void maybeSetShorterConnectionTimeout(final JsonNode config) {
|
||||
((ObjectNode) config).put(JdbcUtils.JDBC_URL_PARAMS_KEY, "loginTimeout=1");
|
||||
}
|
||||
|
||||
@AfterAll
|
||||
public static void cleanUp() throws Exception {
|
||||
dbContainer.close();
|
||||
protected JsonNode config() {
|
||||
return testdb.testConfigBuilder()
|
||||
.withSsl(Map.of("ssl_method", "encrypted_trust_server_certificate"))
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected MssqlSourceStrictEncrypt source() {
|
||||
return new MssqlSourceStrictEncrypt();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected MsSQLTestDatabase createTestDatabase() {
|
||||
final var container = new MsSQLContainerFactory().shared("mcr.microsoft.com/mssql/server:2022-RTM-CU2-ubuntu-20.04");
|
||||
final var testdb = new MsSQLTestDatabase(container);
|
||||
return testdb
|
||||
.withConnectionProperty("encrypt", "true")
|
||||
.withConnectionProperty("trustServerCertificate", "true")
|
||||
.withConnectionProperty("databaseName", testdb.getDatabaseName())
|
||||
.initialized();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -101,34 +66,9 @@ public class MssqlStrictEncryptJdbcSourceAcceptanceTest extends JdbcSourceAccept
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public JsonNode getConfig() {
|
||||
return config;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<JsonNode, JsonNode> getToDatabaseConfigFunction() {
|
||||
return new MssqlSource()::toDatabaseConfig;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDriverClass() {
|
||||
return MssqlSource.DRIVER_CLASS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractJdbcSource<JDBCType> getJdbcSource() {
|
||||
return new MssqlSource();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source getSource() {
|
||||
return new MssqlSourceStrictEncrypt();
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSpec() throws Exception {
|
||||
final ConnectorSpecification actual = source.spec();
|
||||
final ConnectorSpecification actual = source().spec();
|
||||
final ConnectorSpecification expected =
|
||||
SshHelpers.injectSshIntoSpec(Jsons.deserialize(MoreResources.readResource("expected_spec.json"), ConnectorSpecification.class));
|
||||
|
||||
|
||||
@@ -4,7 +4,7 @@ plugins {
|
||||
}
|
||||
|
||||
airbyteJavaConnector {
|
||||
cdkVersionRequired = '0.4.1'
|
||||
cdkVersionRequired = '0.5.0'
|
||||
features = ['db-sources']
|
||||
useLocalCdk = false
|
||||
}
|
||||
@@ -15,7 +15,7 @@ configurations.all {
|
||||
}
|
||||
}
|
||||
|
||||
airbyteJavaConnector.addCdkDependencies()
|
||||
|
||||
|
||||
application {
|
||||
mainClass = 'io.airbyte.integrations.source.mssql.MssqlSource'
|
||||
@@ -25,12 +25,14 @@ application {
|
||||
dependencies {
|
||||
implementation libs.postgresql
|
||||
|
||||
|
||||
implementation libs.debezium.sqlserver
|
||||
implementation 'com.microsoft.sqlserver:mssql-jdbc:10.2.1.jre8'
|
||||
implementation 'org.codehaus.plexus:plexus-utils:3.4.2'
|
||||
|
||||
testImplementation 'org.apache.commons:commons-lang3:3.11'
|
||||
testImplementation libs.testcontainers.mssqlserver
|
||||
testImplementation 'org.hamcrest:hamcrest-all:1.3'
|
||||
testImplementation 'org.awaitility:awaitility:4.2.0'
|
||||
|
||||
testImplementation libs.testcontainers.mssqlserver
|
||||
testFixturesImplementation libs.testcontainers.mssqlserver
|
||||
}
|
||||
|
||||
@@ -0,0 +1 @@
|
||||
testExecutionConcurrency=-1
|
||||
@@ -9,7 +9,7 @@ data:
|
||||
connectorSubtype: database
|
||||
connectorType: source
|
||||
definitionId: b5ea17b1-f170-46dc-bc31-cc744ca984c1
|
||||
dockerImageTag: 3.0.0
|
||||
dockerImageTag: 3.0.1
|
||||
dockerRepository: airbyte/source-mssql
|
||||
documentationUrl: https://docs.airbyte.com/integrations/sources/mssql
|
||||
githubIssueLabel: source-mssql
|
||||
|
||||
@@ -13,6 +13,7 @@ import io.airbyte.protocol.models.v0.AirbyteStream;
|
||||
import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog;
|
||||
import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream;
|
||||
import io.airbyte.protocol.models.v0.SyncMode;
|
||||
import java.time.Duration;
|
||||
import java.util.Properties;
|
||||
import java.util.stream.Collectors;
|
||||
import org.codehaus.plexus.util.StringUtils;
|
||||
@@ -30,6 +31,11 @@ public class MssqlCdcHelper {
|
||||
private static final String CDC_SNAPSHOT_ISOLATION_FIELD = "snapshot_isolation";
|
||||
private static final String CDC_DATA_TO_SYNC_FIELD = "data_to_sync";
|
||||
|
||||
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 enum ReplicationMethod {
|
||||
STANDARD,
|
||||
CDC
|
||||
@@ -160,6 +166,14 @@ public class MssqlCdcHelper {
|
||||
props.setProperty("schema.include.list", getSchema(catalog));
|
||||
props.setProperty("database.names", config.get(JdbcUtils.DATABASE_KEY).asText());
|
||||
|
||||
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()));
|
||||
// TODO: enable heartbeats in MS SQL Server.
|
||||
props.setProperty("heartbeat.interval.ms", "0");
|
||||
|
||||
if (config.has("ssl_method")) {
|
||||
final JsonNode sslConfig = config.get("ssl_method");
|
||||
final String sslMethod = sslConfig.get("ssl_method").asText();
|
||||
|
||||
@@ -30,7 +30,7 @@ import io.airbyte.cdk.integrations.base.Source;
|
||||
import io.airbyte.cdk.integrations.base.ssh.SshWrappedSource;
|
||||
import io.airbyte.cdk.integrations.debezium.AirbyteDebeziumHandler;
|
||||
import io.airbyte.cdk.integrations.debezium.internals.DebeziumPropertiesManager;
|
||||
import io.airbyte.cdk.integrations.debezium.internals.FirstRecordWaitTimeUtil;
|
||||
import io.airbyte.cdk.integrations.debezium.internals.RecordWaitTimeUtil;
|
||||
import io.airbyte.cdk.integrations.debezium.internals.mssql.MssqlCdcTargetPosition;
|
||||
import io.airbyte.cdk.integrations.source.jdbc.AbstractJdbcSource;
|
||||
import io.airbyte.cdk.integrations.source.relationaldb.TableInfo;
|
||||
@@ -88,8 +88,8 @@ public class MssqlSource extends AbstractJdbcSource<JDBCType> implements Source
|
||||
public static final String CDC_DEFAULT_CURSOR = "_ab_cdc_cursor";
|
||||
private List<String> schemas;
|
||||
|
||||
public static Source sshWrappedSource() {
|
||||
return new SshWrappedSource(new MssqlSource(), JdbcUtils.HOST_LIST_KEY, JdbcUtils.PORT_LIST_KEY);
|
||||
public static Source sshWrappedSource(MssqlSource source) {
|
||||
return new SshWrappedSource(source, JdbcUtils.HOST_LIST_KEY, JdbcUtils.PORT_LIST_KEY);
|
||||
}
|
||||
|
||||
public MssqlSource() {
|
||||
@@ -451,11 +451,16 @@ public class MssqlSource extends AbstractJdbcSource<JDBCType> implements Source
|
||||
final JsonNode sourceConfig = database.getSourceConfig();
|
||||
if (MssqlCdcHelper.isCdc(sourceConfig) && isAnyStreamIncrementalSyncMode(catalog)) {
|
||||
LOGGER.info("using CDC: {}", true);
|
||||
final Duration firstRecordWaitTime = FirstRecordWaitTimeUtil.getFirstRecordWaitTime(sourceConfig);
|
||||
final AirbyteDebeziumHandler<Lsn> handler =
|
||||
new AirbyteDebeziumHandler<>(sourceConfig,
|
||||
MssqlCdcTargetPosition.getTargetPosition(database, sourceConfig.get(JdbcUtils.DATABASE_KEY).asText()), true, firstRecordWaitTime,
|
||||
OptionalInt.empty());
|
||||
final Duration firstRecordWaitTime = RecordWaitTimeUtil.getFirstRecordWaitTime(sourceConfig);
|
||||
final Duration subsequentRecordWaitTime = RecordWaitTimeUtil.getSubsequentRecordWaitTime(sourceConfig);
|
||||
final var targetPosition = MssqlCdcTargetPosition.getTargetPosition(database, sourceConfig.get(JdbcUtils.DATABASE_KEY).asText());
|
||||
final AirbyteDebeziumHandler<Lsn> handler = new AirbyteDebeziumHandler<>(
|
||||
sourceConfig,
|
||||
targetPosition,
|
||||
true,
|
||||
firstRecordWaitTime,
|
||||
subsequentRecordWaitTime,
|
||||
OptionalInt.empty());
|
||||
|
||||
final MssqlCdcConnectorMetadataInjector mssqlCdcConnectorMetadataInjector = MssqlCdcConnectorMetadataInjector.getInstance(emittedAt);
|
||||
|
||||
@@ -565,7 +570,7 @@ public class MssqlSource extends AbstractJdbcSource<JDBCType> implements Source
|
||||
}
|
||||
|
||||
public static void main(final String[] args) throws Exception {
|
||||
final Source source = MssqlSource.sshWrappedSource();
|
||||
final Source source = MssqlSource.sshWrappedSource(new MssqlSource());
|
||||
LOGGER.info("starting source: {}", MssqlSource.class);
|
||||
new IntegrationRunner(source).run(args);
|
||||
LOGGER.info("completed source: {}", MssqlSource.class);
|
||||
|
||||
@@ -4,18 +4,14 @@
|
||||
|
||||
package io.airbyte.integrations.source.mssql;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
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.protocol.models.JsonSchemaType;
|
||||
import org.jooq.DSLContext;
|
||||
import org.testcontainers.containers.MSSQLServerContainer;
|
||||
|
||||
public abstract class AbstractMssqlSourceDatatypeTest extends AbstractSourceDatabaseTypeTest {
|
||||
|
||||
protected static MSSQLServerContainer<?> container;
|
||||
protected JsonNode config;
|
||||
protected DSLContext dslContext;
|
||||
protected MsSQLTestDatabase testdb;
|
||||
|
||||
@Override
|
||||
protected String getNameSpace() {
|
||||
@@ -28,14 +24,11 @@ public abstract class AbstractMssqlSourceDatatypeTest extends AbstractSourceData
|
||||
}
|
||||
|
||||
@Override
|
||||
protected JsonNode getConfig() {
|
||||
return config;
|
||||
protected void tearDown(final TestDestinationEnv testEnv) {
|
||||
testdb.close();
|
||||
}
|
||||
|
||||
protected static final String DB_NAME = "comprehensive";
|
||||
|
||||
protected static final String CREATE_TABLE_SQL =
|
||||
"USE " + DB_NAME + "\nCREATE TABLE %1$s(%2$s INTEGER PRIMARY KEY, %3$s %4$s)";
|
||||
protected static final String CREATE_TABLE_SQL = "CREATE TABLE %1$s(%2$s INTEGER PRIMARY KEY, %3$s %4$s)";
|
||||
|
||||
@Override
|
||||
protected void initTests() {
|
||||
|
||||
@@ -5,12 +5,7 @@
|
||||
package io.airbyte.integrations.source.mssql;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.base.ssh.SshBastionContainer;
|
||||
import io.airbyte.cdk.integrations.base.ssh.SshHelpers;
|
||||
import io.airbyte.cdk.integrations.base.ssh.SshTunnel;
|
||||
@@ -25,91 +20,50 @@ 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.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Objects;
|
||||
import org.apache.commons.lang3.RandomStringUtils;
|
||||
import org.jooq.DSLContext;
|
||||
import org.testcontainers.containers.JdbcDatabaseContainer;
|
||||
import org.testcontainers.containers.MSSQLServerContainer;
|
||||
import org.testcontainers.containers.Network;
|
||||
|
||||
public abstract class AbstractSshMssqlSourceAcceptanceTest extends SourceAcceptanceTest {
|
||||
|
||||
private static final String STREAM_NAME = "dbo.id_and_name";
|
||||
private static final String STREAM_NAME2 = "dbo.starships";
|
||||
private static final Network network = Network.newNetwork();
|
||||
private static JsonNode config;
|
||||
private String dbName;
|
||||
private MSSQLServerContainer<?> db;
|
||||
private final SshBastionContainer bastion = new SshBastionContainer();
|
||||
|
||||
public abstract SshTunnel.TunnelMethod getTunnelMethod();
|
||||
|
||||
protected MsSQLTestDatabase testdb;
|
||||
protected SshBastionContainer bastion;
|
||||
|
||||
@Override
|
||||
protected JsonNode getConfig() {
|
||||
try {
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.with("tunnel_method", bastion.getTunnelMethod(getTunnelMethod(), false))
|
||||
.build();
|
||||
} catch (IOException e) {
|
||||
throw new UncheckedIOException(e);
|
||||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) throws Exception {
|
||||
startTestContainers();
|
||||
config = bastion.getTunnelConfig(getTunnelMethod(), getMSSQLDbConfigBuilder(db), false);
|
||||
populateDatabaseTestData();
|
||||
}
|
||||
|
||||
public ImmutableMap.Builder<Object, Object> getMSSQLDbConfigBuilder(final JdbcDatabaseContainer<?> db) {
|
||||
dbName = "db_" + RandomStringUtils.randomAlphabetic(10).toLowerCase();
|
||||
return ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, Objects.requireNonNull(db.getContainerInfo().getNetworkSettings()
|
||||
.getNetworks()
|
||||
.get(((Network.NetworkImpl) network).getName())
|
||||
.getIpAddress()))
|
||||
.put(JdbcUtils.USERNAME_KEY, db.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, db.getPassword())
|
||||
.put(JdbcUtils.PORT_KEY, db.getExposedPorts().get(0))
|
||||
.put(JdbcUtils.DATABASE_KEY, dbName);
|
||||
}
|
||||
|
||||
private Database getDatabaseFromConfig(final JsonNode config) {
|
||||
final DSLContext dslContext = DSLContextFactory.create(
|
||||
config.get(JdbcUtils.USERNAME_KEY).asText(),
|
||||
config.get(JdbcUtils.PASSWORD_KEY).asText(),
|
||||
DatabaseDriver.MSSQLSERVER.getDriverClassName(),
|
||||
String.format("jdbc:sqlserver://%s:%d;",
|
||||
db.getHost(),
|
||||
db.getFirstMappedPort()),
|
||||
null);
|
||||
return new Database(dslContext);
|
||||
}
|
||||
|
||||
private void startTestContainers() {
|
||||
bastion.initAndStartBastion(network);
|
||||
initAndStartJdbcContainer();
|
||||
}
|
||||
|
||||
private void initAndStartJdbcContainer() {
|
||||
db = new MSSQLServerContainer<>("mcr.microsoft.com/mssql/server:2017-latest")
|
||||
.withNetwork(network)
|
||||
.acceptLicense();
|
||||
db.start();
|
||||
}
|
||||
|
||||
private void populateDatabaseTestData() throws Exception {
|
||||
SshTunnel.sshWrap(
|
||||
getConfig(),
|
||||
JdbcUtils.HOST_LIST_KEY,
|
||||
JdbcUtils.PORT_LIST_KEY,
|
||||
mangledConfig -> {
|
||||
getDatabaseFromConfig(mangledConfig).query(ctx -> {
|
||||
ctx.fetch(String.format("CREATE DATABASE %s;", dbName));
|
||||
ctx.fetch(String.format("ALTER DATABASE %s SET AUTO_CLOSE OFF WITH NO_WAIT;", dbName));
|
||||
ctx.fetch(String.format("USE %s;", dbName));
|
||||
ctx.fetch("CREATE TABLE id_and_name(id INTEGER, name VARCHAR(200), born DATETIMEOFFSET(7));");
|
||||
ctx.fetch(
|
||||
"INSERT INTO id_and_name (id, name, born) VALUES (1,'picard', '2124-03-04T01:01:01Z'), (2, 'crusher', '2124-03-04T01:01:01Z'), (3, 'vash', '2124-03-04T01:01:01Z');");
|
||||
return null;
|
||||
});
|
||||
});
|
||||
testdb = MsSQLTestDatabase.in("mcr.microsoft.com/mssql/server:2017-latest", "withNetwork");
|
||||
testdb = testdb
|
||||
.with("ALTER DATABASE %s SET AUTO_CLOSE OFF WITH NO_WAIT;", testdb.getDatabaseName())
|
||||
.with("CREATE TABLE id_and_name(id INTEGER, name VARCHAR(200), born DATETIMEOFFSET(7));")
|
||||
.with("INSERT INTO id_and_name (id, name, born) VALUES " +
|
||||
"(1, 'picard', '2124-03-04T01:01:01Z'), " +
|
||||
"(2, 'crusher', '2124-03-04T01:01:01Z'), " +
|
||||
"(3, 'vash', '2124-03-04T01:01:01Z');");
|
||||
bastion.initAndStartBastion(testdb.getContainer().getNetwork());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void tearDown(final TestDestinationEnv testEnv) {
|
||||
bastion.stopAndCloseContainers(db);
|
||||
bastion.close();
|
||||
testdb.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -122,11 +76,6 @@ public abstract class AbstractSshMssqlSourceAcceptanceTest extends SourceAccepta
|
||||
return SshHelpers.getSpecAndInjectSsh();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected JsonNode getConfig() {
|
||||
return config;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ConfiguredAirbyteCatalog getConfiguredCatalog() {
|
||||
return new ConfiguredAirbyteCatalog().withStreams(Lists.newArrayList(
|
||||
|
||||
@@ -5,18 +5,10 @@
|
||||
package io.airbyte.integrations.source.mssql;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DataSourceFactory;
|
||||
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.cdk.integrations.util.HostPortResolver;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.commons.string.Strings;
|
||||
import io.airbyte.protocol.models.Field;
|
||||
import io.airbyte.protocol.models.JsonSchemaType;
|
||||
import io.airbyte.protocol.models.v0.CatalogHelpers;
|
||||
@@ -26,32 +18,15 @@ 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.Map;
|
||||
import org.jooq.DSLContext;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.testcontainers.containers.MSSQLServerContainer;
|
||||
|
||||
public class CdcMssqlSourceAcceptanceTest extends SourceAcceptanceTest {
|
||||
|
||||
private static final String SCHEMA_NAME = "dbo";
|
||||
private static final String STREAM_NAME = "id_and_name";
|
||||
private static final String STREAM_NAME2 = "starships";
|
||||
private static final String TEST_USER_PASSWORD = "testerjester[1]";
|
||||
private static final String CDC_ROLE_NAME = "cdc_selector";
|
||||
public static MSSQLServerContainer<?> container;
|
||||
private String dbName;
|
||||
private String testUserName;
|
||||
private JsonNode config;
|
||||
private Database database;
|
||||
private DSLContext dslContext;
|
||||
|
||||
@AfterAll
|
||||
public static void closeContainer() {
|
||||
if (container != null) {
|
||||
container.close();
|
||||
container.stop();
|
||||
}
|
||||
}
|
||||
private MsSQLTestDatabase testdb;
|
||||
|
||||
@Override
|
||||
protected String getImageName() {
|
||||
@@ -65,7 +40,10 @@ public class CdcMssqlSourceAcceptanceTest extends SourceAcceptanceTest {
|
||||
|
||||
@Override
|
||||
protected JsonNode getConfig() {
|
||||
return config;
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.withCdcReplication()
|
||||
.withoutSsl()
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -103,123 +81,40 @@ public class CdcMssqlSourceAcceptanceTest extends SourceAcceptanceTest {
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) throws InterruptedException {
|
||||
if (container == null) {
|
||||
container = new MSSQLServerContainer<>("mcr.microsoft.com/mssql/server:2019-latest").acceptLicense();
|
||||
container.addEnv("MSSQL_AGENT_ENABLED", "True"); // need this running for cdc to work
|
||||
container.start();
|
||||
}
|
||||
|
||||
dbName = Strings.addRandomSuffix("db", "_", 10).toLowerCase();
|
||||
testUserName = Strings.addRandomSuffix("test", "_", 5).toLowerCase();
|
||||
|
||||
final JsonNode replicationConfig = Jsons.jsonNode(Map.of(
|
||||
"method", "CDC",
|
||||
"data_to_sync", "Existing and New",
|
||||
"initial_waiting_seconds", 5,
|
||||
"snapshot_isolation", "Snapshot"));
|
||||
|
||||
config = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, HostPortResolver.resolveHost(container))
|
||||
.put(JdbcUtils.PORT_KEY, HostPortResolver.resolvePort(container))
|
||||
.put(JdbcUtils.DATABASE_KEY, dbName)
|
||||
.put(JdbcUtils.USERNAME_KEY, testUserName)
|
||||
.put(JdbcUtils.PASSWORD_KEY, TEST_USER_PASSWORD)
|
||||
.put("replication_method", replicationConfig)
|
||||
.put("ssl_method", Jsons.jsonNode(Map.of("ssl_method", "unencrypted")))
|
||||
.build());
|
||||
|
||||
dslContext = DSLContextFactory.create(DataSourceFactory.create(
|
||||
container.getUsername(),
|
||||
container.getPassword(),
|
||||
container.getDriverClassName(),
|
||||
String.format("jdbc:sqlserver://%s:%d;",
|
||||
container.getHost(),
|
||||
container.getFirstMappedPort()),
|
||||
Map.of("encrypt", "false")), null);
|
||||
database = new Database(dslContext);
|
||||
|
||||
executeQuery("CREATE DATABASE " + dbName + ";");
|
||||
executeQuery("ALTER DATABASE " + dbName + "\n\tSET ALLOW_SNAPSHOT_ISOLATION ON");
|
||||
executeQuery("USE " + dbName + "\n" + "EXEC sys.sp_cdc_enable_db");
|
||||
|
||||
setupTestUser();
|
||||
revokeAllPermissions();
|
||||
createAndPopulateTables();
|
||||
grantCorrectPermissions();
|
||||
}
|
||||
|
||||
private void setupTestUser() {
|
||||
executeQuery("USE " + dbName);
|
||||
executeQuery("CREATE LOGIN " + testUserName + " WITH PASSWORD = '" + TEST_USER_PASSWORD + "';");
|
||||
executeQuery("CREATE USER " + testUserName + " FOR LOGIN " + testUserName + ";");
|
||||
}
|
||||
|
||||
private void revokeAllPermissions() {
|
||||
executeQuery("REVOKE ALL FROM " + testUserName + " CASCADE;");
|
||||
executeQuery("EXEC sp_msforeachtable \"REVOKE ALL ON '?' TO " + testUserName + ";\"");
|
||||
}
|
||||
|
||||
private void createAndPopulateTables() throws InterruptedException {
|
||||
executeQuery(String.format("CREATE TABLE %s.%s(id INTEGER PRIMARY KEY, name VARCHAR(200));",
|
||||
SCHEMA_NAME, STREAM_NAME));
|
||||
executeQuery(String.format("INSERT INTO %s.%s (id, name) VALUES (1,'picard'), (2, 'crusher'), (3, 'vash');",
|
||||
SCHEMA_NAME, STREAM_NAME));
|
||||
executeQuery(String.format("CREATE TABLE %s.%s(id INTEGER PRIMARY KEY, name VARCHAR(200));",
|
||||
SCHEMA_NAME, STREAM_NAME2));
|
||||
executeQuery(String.format("INSERT INTO %s.%s (id, name) VALUES (1,'enterprise-d'), (2, 'defiant'), (3, 'yamato');",
|
||||
SCHEMA_NAME, STREAM_NAME2));
|
||||
|
||||
// sometimes seeing an error that we can't enable cdc on a table while sql server agent is still
|
||||
// spinning up
|
||||
// solving with a simple while retry loop
|
||||
boolean failingToStart = true;
|
||||
int retryNum = 0;
|
||||
final int maxRetries = 10;
|
||||
while (failingToStart) {
|
||||
try {
|
||||
// enabling CDC on each table
|
||||
final String[] tables = {STREAM_NAME, STREAM_NAME2};
|
||||
for (final String table : tables) {
|
||||
executeQuery(String.format(
|
||||
"EXEC sys.sp_cdc_enable_table\n"
|
||||
+ "\t@source_schema = N'%s',\n"
|
||||
+ "\t@source_name = N'%s', \n"
|
||||
+ "\t@role_name = N'%s',\n"
|
||||
+ "\t@supports_net_changes = 0",
|
||||
SCHEMA_NAME, table, CDC_ROLE_NAME));
|
||||
}
|
||||
failingToStart = false;
|
||||
} catch (final Exception e) {
|
||||
if (retryNum >= maxRetries) {
|
||||
throw e;
|
||||
} else {
|
||||
retryNum++;
|
||||
Thread.sleep(10000); // 10 seconds
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void grantCorrectPermissions() {
|
||||
executeQuery(String.format("EXEC sp_addrolemember N'%s', N'%s';", "db_datareader", testUserName));
|
||||
executeQuery(String.format("USE %s;\n" + "GRANT SELECT ON SCHEMA :: [%s] TO %s", dbName, "cdc", testUserName));
|
||||
executeQuery(String.format("EXEC sp_addrolemember N'%s', N'%s';", CDC_ROLE_NAME, testUserName));
|
||||
}
|
||||
|
||||
private void executeQuery(final String query) {
|
||||
try {
|
||||
database.query(
|
||||
ctx -> ctx
|
||||
.execute(query));
|
||||
} catch (final Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) {
|
||||
testdb = MsSQLTestDatabase.in("mcr.microsoft.com/mssql/server:2022-latest", "withAgent");
|
||||
final var enableCdcSqlFmt = """
|
||||
EXEC sys.sp_cdc_enable_table
|
||||
\t@source_schema = N'%s',
|
||||
\t@source_name = N'%s',
|
||||
\t@role_name = N'%s',
|
||||
\t@supports_net_changes = 0""";
|
||||
testdb
|
||||
.withSnapshotIsolation()
|
||||
.withCdc()
|
||||
.withWaitUntilAgentRunning()
|
||||
// create tables
|
||||
.with("CREATE TABLE %s.%s(id INTEGER PRIMARY KEY, name VARCHAR(200));", SCHEMA_NAME, STREAM_NAME)
|
||||
.with("CREATE TABLE %s.%s(id INTEGER PRIMARY KEY, name VARCHAR(200));", SCHEMA_NAME, STREAM_NAME2)
|
||||
// populate tables
|
||||
.with("INSERT INTO %s.%s (id, name) VALUES (1,'picard'), (2, 'crusher'), (3, 'vash');", SCHEMA_NAME, STREAM_NAME)
|
||||
.with("INSERT INTO %s.%s (id, name) VALUES (1,'enterprise-d'), (2, 'defiant'), (3, 'yamato');", SCHEMA_NAME, STREAM_NAME2)
|
||||
// enable cdc on tables for designated role
|
||||
.with(enableCdcSqlFmt, SCHEMA_NAME, STREAM_NAME, CDC_ROLE_NAME)
|
||||
.with(enableCdcSqlFmt, SCHEMA_NAME, STREAM_NAME2, CDC_ROLE_NAME)
|
||||
.withWaitUntilMaxLsnAvailable()
|
||||
// revoke user permissions
|
||||
.with("REVOKE ALL FROM %s CASCADE;", testdb.getUserName())
|
||||
.with("EXEC sp_msforeachtable \"REVOKE ALL ON '?' TO %s;\"", testdb.getUserName())
|
||||
// grant user permissions
|
||||
.with("EXEC sp_addrolemember N'%s', N'%s';", "db_datareader", testdb.getUserName())
|
||||
.with("GRANT SELECT ON SCHEMA :: [cdc] TO %s", testdb.getUserName())
|
||||
.with("EXEC sp_addrolemember N'%s', N'%s';", CDC_ROLE_NAME, testdb.getUserName());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void tearDown(final TestDestinationEnv testEnv) {
|
||||
dslContext.close();
|
||||
testdb.close();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -5,73 +5,25 @@
|
||||
package io.airbyte.integrations.source.mssql;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DataSourceFactory;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
|
||||
import io.airbyte.cdk.integrations.util.HostPortResolver;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import java.util.Map;
|
||||
import org.testcontainers.containers.MSSQLServerContainer;
|
||||
|
||||
public class CdcMssqlSourceDatatypeTest extends AbstractMssqlSourceDatatypeTest {
|
||||
|
||||
@Override
|
||||
protected void tearDown(final TestDestinationEnv testEnv) {
|
||||
dslContext.close();
|
||||
container.close();
|
||||
protected JsonNode getConfig() {
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.withCdcReplication()
|
||||
.withoutSsl()
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Database setupDatabase() throws Exception {
|
||||
container = new MSSQLServerContainer<>("mcr.microsoft.com/mssql/server:2019-latest").acceptLicense();
|
||||
container.addEnv("MSSQL_AGENT_ENABLED", "True"); // need this running for cdc to work
|
||||
container.start();
|
||||
|
||||
final JsonNode replicationConfig = Jsons.jsonNode(Map.of(
|
||||
"method", "CDC",
|
||||
"data_to_sync", "Existing and New",
|
||||
"initial_waiting_seconds", 5,
|
||||
"snapshot_isolation", "Snapshot"));
|
||||
|
||||
config = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, HostPortResolver.resolveHost(container))
|
||||
.put(JdbcUtils.PORT_KEY, HostPortResolver.resolvePort(container))
|
||||
.put(JdbcUtils.DATABASE_KEY, DB_NAME)
|
||||
.put(JdbcUtils.USERNAME_KEY, container.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, container.getPassword())
|
||||
.put("replication_method", replicationConfig)
|
||||
.put("ssl_method", Jsons.jsonNode(Map.of("ssl_method", "unencrypted")))
|
||||
.build());
|
||||
|
||||
dslContext = DSLContextFactory.create(DataSourceFactory.create(
|
||||
container.getUsername(),
|
||||
container.getPassword(),
|
||||
container.getDriverClassName(),
|
||||
String.format("jdbc:sqlserver://%s:%d;",
|
||||
container.getHost(),
|
||||
container.getFirstMappedPort()),
|
||||
Map.of("encrypt", "false")), null);
|
||||
final Database database = new Database(dslContext);
|
||||
|
||||
executeQuery("CREATE DATABASE " + DB_NAME + ";");
|
||||
executeQuery("ALTER DATABASE " + DB_NAME + "\n\tSET ALLOW_SNAPSHOT_ISOLATION ON");
|
||||
executeQuery("USE " + DB_NAME + "\n" + "EXEC sys.sp_cdc_enable_db");
|
||||
|
||||
return database;
|
||||
}
|
||||
|
||||
private void executeQuery(final String query) {
|
||||
try {
|
||||
final Database database = new Database(dslContext);
|
||||
database.query(
|
||||
ctx -> ctx
|
||||
.execute(query));
|
||||
} catch (final Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
protected Database setupDatabase() {
|
||||
testdb = MsSQLTestDatabase.in("mcr.microsoft.com/mssql/server:2022-latest", "withAgent")
|
||||
.withSnapshotIsolation()
|
||||
.withCdc();
|
||||
return testdb.getDatabase();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -81,39 +33,39 @@ public class CdcMssqlSourceDatatypeTest extends AbstractMssqlSourceDatatypeTest
|
||||
}
|
||||
|
||||
private void enableCdcOnAllTables() {
|
||||
executeQuery("USE " + DB_NAME + "\n"
|
||||
+ "DECLARE @TableName VARCHAR(100)\n"
|
||||
+ "DECLARE @TableSchema VARCHAR(100)\n"
|
||||
+ "DECLARE CDC_Cursor CURSOR FOR\n"
|
||||
+ " SELECT * FROM ( \n"
|
||||
+ " SELECT Name,SCHEMA_NAME(schema_id) AS TableSchema\n"
|
||||
+ " FROM sys.objects\n"
|
||||
+ " WHERE type = 'u'\n"
|
||||
+ " AND is_ms_shipped <> 1\n"
|
||||
+ " ) CDC\n"
|
||||
+ "OPEN CDC_Cursor\n"
|
||||
+ "FETCH NEXT FROM CDC_Cursor INTO @TableName,@TableSchema\n"
|
||||
+ "WHILE @@FETCH_STATUS = 0\n"
|
||||
+ " BEGIN\n"
|
||||
+ " DECLARE @SQL NVARCHAR(1000)\n"
|
||||
+ " DECLARE @CDC_Status TINYINT\n"
|
||||
+ " SET @CDC_Status=(SELECT COUNT(*)\n"
|
||||
+ " FROM cdc.change_tables\n"
|
||||
+ " WHERE Source_object_id = OBJECT_ID(@TableSchema+'.'+@TableName))\n"
|
||||
+ " --IF CDC is not enabled on Table, Enable CDC\n"
|
||||
+ " IF @CDC_Status <> 1\n"
|
||||
+ " BEGIN\n"
|
||||
+ " SET @SQL='EXEC sys.sp_cdc_enable_table\n"
|
||||
+ " @source_schema = '''+@TableSchema+''',\n"
|
||||
+ " @source_name = ''' + @TableName\n"
|
||||
+ " + ''',\n"
|
||||
+ " @role_name = null;'\n"
|
||||
+ " EXEC sp_executesql @SQL\n"
|
||||
+ " END\n"
|
||||
+ " FETCH NEXT FROM CDC_Cursor INTO @TableName,@TableSchema\n"
|
||||
+ "END\n"
|
||||
+ "CLOSE CDC_Cursor\n"
|
||||
+ "DEALLOCATE CDC_Cursor");
|
||||
testdb.with("""
|
||||
DECLARE @TableName VARCHAR(100)
|
||||
DECLARE @TableSchema VARCHAR(100)
|
||||
DECLARE CDC_Cursor CURSOR FOR
|
||||
SELECT * FROM (
|
||||
SELECT Name,SCHEMA_NAME(schema_id) AS TableSchema
|
||||
FROM sys.objects
|
||||
WHERE type = 'u'
|
||||
AND is_ms_shipped <> 1
|
||||
) CDC
|
||||
OPEN CDC_Cursor
|
||||
FETCH NEXT FROM CDC_Cursor INTO @TableName,@TableSchema
|
||||
WHILE @@FETCH_STATUS = 0
|
||||
BEGIN
|
||||
DECLARE @SQL NVARCHAR(1000)
|
||||
DECLARE @CDC_Status TINYINT
|
||||
SET @CDC_Status=(SELECT COUNT(*)
|
||||
FROM cdc.change_tables
|
||||
WHERE Source_object_id = OBJECT_ID(@TableSchema+'.'+@TableName))
|
||||
--IF CDC is not enabled on Table, Enable CDC
|
||||
IF @CDC_Status <> 1
|
||||
BEGIN
|
||||
SET @SQL='EXEC sys.sp_cdc_enable_table
|
||||
@source_schema = '''+@TableSchema+''',
|
||||
@source_name = ''' + @TableName
|
||||
+ ''',
|
||||
@role_name = null;'
|
||||
EXEC sp_executesql @SQL
|
||||
END
|
||||
FETCH NEXT FROM CDC_Cursor INTO @TableName,@TableSchema
|
||||
END
|
||||
CLOSE CDC_Cursor
|
||||
DEALLOCATE CDC_Cursor""");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -5,19 +5,10 @@
|
||||
package io.airbyte.integrations.source.mssql;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DataSourceFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
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.cdk.integrations.util.HostPortResolver;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.commons.string.Strings;
|
||||
import io.airbyte.protocol.models.Field;
|
||||
import io.airbyte.protocol.models.JsonSchemaType;
|
||||
import io.airbyte.protocol.models.v0.CatalogHelpers;
|
||||
@@ -25,61 +16,28 @@ import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog;
|
||||
import io.airbyte.protocol.models.v0.ConnectorSpecification;
|
||||
import java.sql.SQLException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import org.jooq.DSLContext;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.testcontainers.containers.MSSQLServerContainer;
|
||||
|
||||
public class MssqlSourceAcceptanceTest extends SourceAcceptanceTest {
|
||||
|
||||
protected static final String SCHEMA_NAME = "dbo";
|
||||
protected static final String STREAM_NAME = "id_and_name";
|
||||
protected static MSSQLServerContainer<?> db;
|
||||
protected JsonNode config;
|
||||
|
||||
@AfterAll
|
||||
public static void closeContainer() {
|
||||
if (db != null) {
|
||||
db.close();
|
||||
db.stop();
|
||||
}
|
||||
}
|
||||
protected MsSQLTestDatabase testdb;
|
||||
|
||||
@Override
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) throws SQLException {
|
||||
if (db == null) {
|
||||
db = new MSSQLServerContainer<>("mcr.microsoft.com/mssql/server:2022-RTM-CU2-ubuntu-20.04").acceptLicense();
|
||||
db.start();
|
||||
}
|
||||
final JsonNode configWithoutDbName = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, HostPortResolver.resolveHost(db))
|
||||
.put(JdbcUtils.PORT_KEY, HostPortResolver.resolvePort(db))
|
||||
.put(JdbcUtils.USERNAME_KEY, db.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, db.getPassword())
|
||||
.build());
|
||||
final String dbName = Strings.addRandomSuffix("db", "_", 10).toLowerCase();
|
||||
|
||||
try (final DSLContext dslContext = getDslContext(configWithoutDbName)) {
|
||||
final Database database = getDatabase(dslContext);
|
||||
database.query(ctx -> {
|
||||
ctx.fetch(String.format("CREATE DATABASE %s;", dbName));
|
||||
ctx.fetch(String.format("USE %s;", dbName));
|
||||
ctx.fetch("CREATE TABLE id_and_name(id INTEGER, name VARCHAR(200), born DATETIMEOFFSET(7));");
|
||||
ctx.fetch(
|
||||
"INSERT INTO id_and_name (id, name, born) VALUES " +
|
||||
"(1,'picard', '2124-03-04T01:01:01Z'), " +
|
||||
"(2, 'crusher', '2124-03-04T01:01:01Z'), (3, 'vash', '2124-03-04T01:01:01Z');");
|
||||
return null;
|
||||
});
|
||||
}
|
||||
|
||||
config = Jsons.clone(configWithoutDbName);
|
||||
((ObjectNode) config).put(JdbcUtils.DATABASE_KEY, dbName);
|
||||
((ObjectNode) config).put("ssl_method", Jsons.jsonNode(Map.of("ssl_method", "unencrypted")));
|
||||
testdb = MsSQLTestDatabase.in("mcr.microsoft.com/mssql/server:2022-RTM-CU2-ubuntu-20.04")
|
||||
.with("CREATE TABLE id_and_name(id INTEGER, name VARCHAR(200), born DATETIMEOFFSET(7));")
|
||||
.with("INSERT INTO id_and_name (id, name, born) VALUES " +
|
||||
"(1, 'picard', '2124-03-04T01:01:01Z'), " +
|
||||
"(2, 'crusher', '2124-03-04T01:01:01Z'), " +
|
||||
"(3, 'vash', '2124-03-04T01:01:01Z');");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void tearDown(final TestDestinationEnv testEnv) throws Exception {}
|
||||
protected void tearDown(final TestDestinationEnv testEnv) {
|
||||
testdb.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getImageName() {
|
||||
@@ -93,7 +51,9 @@ public class MssqlSourceAcceptanceTest extends SourceAcceptanceTest {
|
||||
|
||||
@Override
|
||||
protected JsonNode getConfig() {
|
||||
return config;
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.withoutSsl()
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -111,19 +71,4 @@ public class MssqlSourceAcceptanceTest extends SourceAcceptanceTest {
|
||||
return Jsons.jsonNode(new HashMap<>());
|
||||
}
|
||||
|
||||
private static DSLContext getDslContext(final JsonNode config) {
|
||||
return DSLContextFactory.create(DataSourceFactory.create(
|
||||
config.get(JdbcUtils.USERNAME_KEY).asText(),
|
||||
config.get(JdbcUtils.PASSWORD_KEY).asText(),
|
||||
DatabaseDriver.MSSQLSERVER.getDriverClassName(),
|
||||
String.format("jdbc:sqlserver://%s:%d;",
|
||||
db.getHost(),
|
||||
db.getFirstMappedPort()),
|
||||
Map.of("encrypt", "false")), null);
|
||||
}
|
||||
|
||||
private static Database getDatabase(final DSLContext dslContext) {
|
||||
return new Database(dslContext);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -5,70 +5,21 @@
|
||||
package io.airbyte.integrations.source.mssql;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DataSourceFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
|
||||
import io.airbyte.cdk.integrations.util.HostPortResolver;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import java.util.Map;
|
||||
import org.jooq.DSLContext;
|
||||
import org.testcontainers.containers.MSSQLServerContainer;
|
||||
|
||||
public class MssqlSourceDatatypeTest extends AbstractMssqlSourceDatatypeTest {
|
||||
|
||||
@Override
|
||||
protected Database setupDatabase() throws Exception {
|
||||
container = new MSSQLServerContainer<>("mcr.microsoft.com/mssql/server:2019-latest")
|
||||
.acceptLicense();
|
||||
container.start();
|
||||
|
||||
final JsonNode configWithoutDbName = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, HostPortResolver.resolveHost(container))
|
||||
.put(JdbcUtils.PORT_KEY, HostPortResolver.resolvePort(container))
|
||||
.put(JdbcUtils.USERNAME_KEY, container.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, container.getPassword())
|
||||
.build());
|
||||
|
||||
dslContext = getDslContext(configWithoutDbName);
|
||||
final Database database = getDatabase(dslContext);
|
||||
database.query(ctx -> {
|
||||
ctx.fetch(String.format("CREATE DATABASE %s;", DB_NAME));
|
||||
ctx.fetch(String.format("USE %s;", DB_NAME));
|
||||
return null;
|
||||
});
|
||||
|
||||
config = Jsons.clone(configWithoutDbName);
|
||||
((ObjectNode) config).put(JdbcUtils.DATABASE_KEY, DB_NAME);
|
||||
((ObjectNode) config).put("ssl_method", Jsons.jsonNode(Map.of("ssl_method", "unencrypted")));
|
||||
|
||||
return database;
|
||||
}
|
||||
|
||||
private static DSLContext getDslContext(final JsonNode config) {
|
||||
return DSLContextFactory.create(DataSourceFactory.create(
|
||||
config.get(JdbcUtils.USERNAME_KEY).asText(),
|
||||
config.get(JdbcUtils.PASSWORD_KEY).asText(),
|
||||
DatabaseDriver.MSSQLSERVER.getDriverClassName(),
|
||||
String.format("jdbc:sqlserver://%s:%d;",
|
||||
container.getHost(),
|
||||
container.getFirstMappedPort()),
|
||||
Map.of("encrypt", "false")), null);
|
||||
}
|
||||
|
||||
private static Database getDatabase(final DSLContext dslContext) {
|
||||
return new Database(dslContext);
|
||||
protected Database setupDatabase() {
|
||||
testdb = MsSQLTestDatabase.in("mcr.microsoft.com/mssql/server:2022-RTM-CU2-ubuntu-20.04");
|
||||
return testdb.getDatabase();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void tearDown(final TestDestinationEnv testEnv) {
|
||||
dslContext.close();
|
||||
container.stop();
|
||||
container.close();
|
||||
protected JsonNode getConfig() {
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.withoutSsl()
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -5,80 +5,32 @@
|
||||
package io.airbyte.integrations.source.mssql;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
|
||||
import io.airbyte.cdk.integrations.util.HostPortResolver;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import java.sql.SQLException;
|
||||
import java.util.Map;
|
||||
import org.apache.commons.lang3.RandomStringUtils;
|
||||
import org.jooq.DSLContext;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.testcontainers.containers.MSSQLServerContainer;
|
||||
|
||||
public class SslEnabledMssqlSourceAcceptanceTest extends MssqlSourceAcceptanceTest {
|
||||
|
||||
@AfterAll
|
||||
public static void closeContainer() {
|
||||
if (db != null) {
|
||||
db.close();
|
||||
db.stop();
|
||||
}
|
||||
@Override
|
||||
protected JsonNode getConfig() {
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.withSsl(Map.of("ssl_method", "encrypted_trust_server_certificate"))
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) throws SQLException {
|
||||
if (db == null) {
|
||||
db = new MSSQLServerContainer<>("mcr.microsoft.com/mssql/server:2022-RTM-CU2-ubuntu-20.04").acceptLicense();
|
||||
db.start();
|
||||
}
|
||||
|
||||
final JsonNode configWithoutDbName = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, HostPortResolver.resolveHost(db))
|
||||
.put(JdbcUtils.PORT_KEY, HostPortResolver.resolvePort(db))
|
||||
.put(JdbcUtils.USERNAME_KEY, db.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, db.getPassword())
|
||||
.build());
|
||||
final String dbName = "db_" + RandomStringUtils.randomAlphabetic(10).toLowerCase();
|
||||
|
||||
try (final DSLContext dslContext = getDslContext(configWithoutDbName)) {
|
||||
final Database database = getDatabase(dslContext);
|
||||
database.query(ctx -> {
|
||||
ctx.fetch(String.format("CREATE DATABASE %s;", dbName));
|
||||
ctx.fetch(String.format("USE %s;", dbName));
|
||||
ctx.fetch("CREATE TABLE id_and_name(id INTEGER, name VARCHAR(200), born DATETIMEOFFSET(7));");
|
||||
ctx.fetch(
|
||||
"INSERT INTO id_and_name (id, name, born) VALUES " +
|
||||
"(1,'picard', '2124-03-04T01:01:01Z'), " +
|
||||
"(2, 'crusher', '2124-03-04T01:01:01Z'), " +
|
||||
"(3, 'vash', '2124-03-04T01:01:01Z');");
|
||||
return null;
|
||||
});
|
||||
}
|
||||
|
||||
config = Jsons.clone(configWithoutDbName);
|
||||
((ObjectNode) config).put(JdbcUtils.DATABASE_KEY, dbName);
|
||||
((ObjectNode) config).put("ssl_method", Jsons.jsonNode(Map.of("ssl_method", "encrypted_trust_server_certificate")));
|
||||
}
|
||||
|
||||
private DSLContext getDslContext(final JsonNode baseConfig) {
|
||||
return DSLContextFactory.create(
|
||||
baseConfig.get(JdbcUtils.USERNAME_KEY).asText(),
|
||||
baseConfig.get(JdbcUtils.PASSWORD_KEY).asText(),
|
||||
DatabaseDriver.MSSQLSERVER.getDriverClassName(),
|
||||
String.format("jdbc:sqlserver://%s:%d;encrypt=true;trustServerCertificate=true;",
|
||||
db.getHost(),
|
||||
db.getFirstMappedPort()),
|
||||
null);
|
||||
}
|
||||
|
||||
private static Database getDatabase(final DSLContext dslContext) {
|
||||
return new Database(dslContext);
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) {
|
||||
final var container = new MsSQLContainerFactory().shared("mcr.microsoft.com/mssql/server:2022-RTM-CU2-ubuntu-20.04");
|
||||
testdb = new MsSQLTestDatabase(container);
|
||||
testdb = testdb
|
||||
.withConnectionProperty("encrypt", "true")
|
||||
.withConnectionProperty("trustServerCertificate", "true")
|
||||
.withConnectionProperty("databaseName", testdb.getDatabaseName())
|
||||
.initialized()
|
||||
.with("CREATE TABLE id_and_name(id INTEGER, name VARCHAR(200), born DATETIMEOFFSET(7));")
|
||||
.with("INSERT INTO id_and_name (id, name, born) VALUES " +
|
||||
"(1, 'picard', '2124-03-04T01:01:01Z'), " +
|
||||
"(2, 'crusher', '2124-03-04T01:01:01Z'), " +
|
||||
"(3, 'vash', '2124-03-04T01:01:01Z');");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -9,9 +9,9 @@ import static io.airbyte.cdk.integrations.debezium.internals.DebeziumEventUtils.
|
||||
import static io.airbyte.integrations.source.mssql.MssqlSource.CDC_DEFAULT_CURSOR;
|
||||
import static io.airbyte.integrations.source.mssql.MssqlSource.CDC_EVENT_SERIAL_NO;
|
||||
import static io.airbyte.integrations.source.mssql.MssqlSource.CDC_LSN;
|
||||
import static io.airbyte.integrations.source.mssql.MssqlSource.DRIVER_CLASS;
|
||||
import static io.airbyte.integrations.source.mssql.MssqlSource.MSSQL_CDC_OFFSET;
|
||||
import static io.airbyte.integrations.source.mssql.MssqlSource.MSSQL_DB_HISTORY;
|
||||
import static org.awaitility.Awaitility.await;
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
@@ -24,162 +24,147 @@ import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DataSourceFactory;
|
||||
import io.airbyte.cdk.db.jdbc.DefaultJdbcDatabase;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcDatabase;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.db.jdbc.StreamingJdbcDatabase;
|
||||
import io.airbyte.cdk.db.jdbc.streaming.AdaptiveStreamingQueryConfig;
|
||||
import io.airbyte.cdk.integrations.base.Source;
|
||||
import io.airbyte.cdk.integrations.debezium.CdcSourceTest;
|
||||
import io.airbyte.cdk.integrations.debezium.internals.mssql.MssqlCdcTargetPosition;
|
||||
import io.airbyte.commons.features.EnvVariableFeatureFlags;
|
||||
import io.airbyte.commons.features.FeatureFlagsWrapper;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.commons.string.Strings;
|
||||
import io.airbyte.protocol.models.v0.AirbyteConnectionStatus;
|
||||
import io.airbyte.protocol.models.v0.AirbyteStateMessage;
|
||||
import io.airbyte.protocol.models.v0.AirbyteStream;
|
||||
import io.airbyte.protocol.models.v0.SyncMode;
|
||||
import io.debezium.connector.sqlserver.Lsn;
|
||||
import java.sql.SQLException;
|
||||
import java.time.Duration;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import javax.sql.DataSource;
|
||||
import org.jooq.DSLContext;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.testcontainers.containers.MSSQLServerContainer;
|
||||
import org.testcontainers.utility.DockerImageName;
|
||||
|
||||
public class CdcMssqlSourceTest extends CdcSourceTest {
|
||||
public class CdcMssqlSourceTest extends CdcSourceTest<MssqlSource, MsSQLTestDatabase> {
|
||||
|
||||
private static final String CDC_ROLE_NAME = "cdc_selector";
|
||||
private static final String TEST_USER_PASSWORD = "testerjester[1]";
|
||||
public static MSSQLServerContainer<?> container;
|
||||
static private final String CDC_ROLE_NAME = "cdc_selector";
|
||||
|
||||
static private final String TEST_USER_NAME_PREFIX = "cdc_test_user";
|
||||
|
||||
// Deliberately do not share this test container, as we're going to mutate the global SQL Server
|
||||
// state.
|
||||
static private final MSSQLServerContainer<?> UNSHARED_CONTAINER = new MsSQLContainerFactory()
|
||||
.createNewContainer(DockerImageName.parse("mcr.microsoft.com/mssql/server:2022-latest"));
|
||||
|
||||
private String testUserName;
|
||||
private String dbName;
|
||||
private String dbNamewithDot;
|
||||
private Database database;
|
||||
private JdbcDatabase testJdbcDatabase;
|
||||
private MssqlSource source;
|
||||
private JsonNode config;
|
||||
private DSLContext dslContext;
|
||||
private DataSource dataSource;
|
||||
private DataSource testDataSource;
|
||||
|
||||
@BeforeEach
|
||||
public void setup() throws SQLException {
|
||||
init();
|
||||
setupTestUser();
|
||||
revokeAllPermissions();
|
||||
super.setup();
|
||||
grantCorrectPermissions();
|
||||
}
|
||||
|
||||
@BeforeAll
|
||||
public static void createContainer() {
|
||||
if (container == null) {
|
||||
container = new MSSQLServerContainer<>("mcr.microsoft.com/mssql/server:2019-latest").acceptLicense();
|
||||
container.addEnv("MSSQL_AGENT_ENABLED", "True"); // need this running for cdc to work
|
||||
container.start();
|
||||
}
|
||||
static public void beforeAll() {
|
||||
new MsSQLContainerFactory().withAgent(UNSHARED_CONTAINER);
|
||||
UNSHARED_CONTAINER.start();
|
||||
}
|
||||
|
||||
@AfterAll
|
||||
public static void closeContainer() {
|
||||
if (container != null) {
|
||||
container.close();
|
||||
container.stop();
|
||||
}
|
||||
static void afterAll() {
|
||||
UNSHARED_CONTAINER.close();
|
||||
}
|
||||
|
||||
private void init() {
|
||||
dbName = Strings.addRandomSuffix("db", "_", 10).toLowerCase();
|
||||
testUserName = Strings.addRandomSuffix("test", "_", 5).toLowerCase();
|
||||
dbNamewithDot = Strings.addRandomSuffix("db", ".", 10).toLowerCase();
|
||||
source = new MssqlSource();
|
||||
|
||||
final JsonNode replicationConfig = Jsons.jsonNode(Map.of(
|
||||
"method", "CDC",
|
||||
"data_to_sync", "Existing and New",
|
||||
"initial_waiting_seconds", INITIAL_WAITING_SECONDS,
|
||||
"snapshot_isolation", "Snapshot"));
|
||||
config = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, container.getHost())
|
||||
.put(JdbcUtils.PORT_KEY, container.getFirstMappedPort())
|
||||
.put(JdbcUtils.DATABASE_KEY, dbName)
|
||||
.put(JdbcUtils.SCHEMAS_KEY, List.of(MODELS_SCHEMA, MODELS_SCHEMA + "_random"))
|
||||
.put(JdbcUtils.USERNAME_KEY, testUserName)
|
||||
.put(JdbcUtils.PASSWORD_KEY, TEST_USER_PASSWORD)
|
||||
.put("replication_method", replicationConfig)
|
||||
.put("ssl_method", Jsons.jsonNode(Map.of("ssl_method", "unencrypted")))
|
||||
.build());
|
||||
|
||||
dataSource = DataSourceFactory.create(
|
||||
container.getUsername(),
|
||||
container.getPassword(),
|
||||
DRIVER_CLASS,
|
||||
String.format("jdbc:sqlserver://%s:%d",
|
||||
container.getHost(),
|
||||
container.getFirstMappedPort()),
|
||||
Map.of("encrypt", "false"));
|
||||
|
||||
testDataSource = DataSourceFactory.create(
|
||||
testUserName,
|
||||
TEST_USER_PASSWORD,
|
||||
DRIVER_CLASS,
|
||||
String.format("jdbc:sqlserver://%s:%d",
|
||||
container.getHost(),
|
||||
container.getFirstMappedPort()),
|
||||
Map.of("encrypt", "false"));
|
||||
|
||||
dslContext = DSLContextFactory.create(dataSource, null);
|
||||
|
||||
database = new Database(dslContext);
|
||||
|
||||
testJdbcDatabase = new DefaultJdbcDatabase(testDataSource);
|
||||
|
||||
executeQuery("CREATE DATABASE " + dbName + ";");
|
||||
executeQuery("CREATE DATABASE [" + dbNamewithDot + "];");
|
||||
switchSnapshotIsolation(true, dbName);
|
||||
}
|
||||
|
||||
private void switchSnapshotIsolation(final Boolean on, final String db) {
|
||||
final String onOrOff = on ? "ON" : "OFF";
|
||||
executeQuery("ALTER DATABASE " + db + "\n\tSET ALLOW_SNAPSHOT_ISOLATION " + onOrOff);
|
||||
}
|
||||
|
||||
private void setupTestUser() {
|
||||
executeQuery("USE " + dbName);
|
||||
executeQuery("CREATE LOGIN " + testUserName + " WITH PASSWORD = '" + TEST_USER_PASSWORD + "';");
|
||||
executeQuery("CREATE USER " + testUserName + " FOR LOGIN " + testUserName + ";");
|
||||
}
|
||||
|
||||
private void revokeAllPermissions() {
|
||||
executeQuery("REVOKE ALL FROM " + testUserName + " CASCADE;");
|
||||
executeQuery("EXEC sp_msforeachtable \"REVOKE ALL ON '?' TO " + testUserName + ";\"");
|
||||
}
|
||||
|
||||
private void alterPermissionsOnSchema(final Boolean grant, final String schema) {
|
||||
final String grantOrRemove = grant ? "GRANT" : "REVOKE";
|
||||
executeQuery(String.format("USE %s;\n" + "%s SELECT ON SCHEMA :: [%s] TO %s", dbName, grantOrRemove, schema, testUserName));
|
||||
}
|
||||
|
||||
private void grantCorrectPermissions() {
|
||||
alterPermissionsOnSchema(true, MODELS_SCHEMA);
|
||||
alterPermissionsOnSchema(true, MODELS_SCHEMA + "_random");
|
||||
alterPermissionsOnSchema(true, "cdc");
|
||||
executeQuery(String.format("EXEC sp_addrolemember N'%s', N'%s';", CDC_ROLE_NAME, testUserName));
|
||||
private String testUserName() {
|
||||
return testdb.withNamespace(TEST_USER_NAME_PREFIX);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String createSchemaQuery(final String schemaName) {
|
||||
return "CREATE SCHEMA " + schemaName;
|
||||
protected MsSQLTestDatabase createTestDatabase() {
|
||||
final var testdb = new MsSQLTestDatabase(UNSHARED_CONTAINER);
|
||||
return testdb
|
||||
.withConnectionProperty("encrypt", "false")
|
||||
.withConnectionProperty("databaseName", testdb.getDatabaseName())
|
||||
.initialized()
|
||||
.withSnapshotIsolation()
|
||||
.withCdc()
|
||||
.withWaitUntilAgentRunning();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected MssqlSource source() {
|
||||
final var source = new MssqlSource();
|
||||
source.setFeatureFlags(FeatureFlagsWrapper.overridingUseStreamCapableState(new EnvVariableFeatureFlags(), true));
|
||||
return source;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected JsonNode config() {
|
||||
return testdb.configBuilder()
|
||||
.withHostAndPort()
|
||||
.withDatabase()
|
||||
.with(JdbcUtils.USERNAME_KEY, testUserName())
|
||||
.with(JdbcUtils.PASSWORD_KEY, testdb.getPassword())
|
||||
.withSchemas(modelsSchema(), randomSchema())
|
||||
.withCdcReplication()
|
||||
.withoutSsl()
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
@BeforeEach
|
||||
protected void setup() {
|
||||
super.setup();
|
||||
|
||||
// Enables cdc on MODELS_SCHEMA.MODELS_STREAM_NAME, giving CDC_ROLE_NAME select access.
|
||||
final var enableCdcSqlFmt = """
|
||||
EXEC sys.sp_cdc_enable_table
|
||||
\t@source_schema = N'%s',
|
||||
\t@source_name = N'%s',
|
||||
\t@role_name = N'%s',
|
||||
\t@supports_net_changes = 0""";
|
||||
testdb
|
||||
.with(enableCdcSqlFmt, modelsSchema(), MODELS_STREAM_NAME, CDC_ROLE_NAME)
|
||||
.with(enableCdcSqlFmt, randomSchema(), RANDOM_TABLE_NAME, CDC_ROLE_NAME);
|
||||
|
||||
// Create a test user to be used by the source, with proper permissions.
|
||||
testdb
|
||||
.with("CREATE LOGIN %s WITH PASSWORD = '%s', DEFAULT_DATABASE = %s", testUserName(), testdb.getPassword(), testdb.getDatabaseName())
|
||||
.with("CREATE USER %s FOR LOGIN %s WITH DEFAULT_SCHEMA = [dbo]", testUserName(), testUserName())
|
||||
.with("REVOKE ALL FROM %s CASCADE;", testUserName())
|
||||
.with("EXEC sp_msforeachtable \"REVOKE ALL ON '?' TO %s;\"", testUserName())
|
||||
.with("GRANT SELECT ON SCHEMA :: [%s] TO %s", modelsSchema(), testUserName())
|
||||
.with("GRANT SELECT ON SCHEMA :: [%s] TO %s", randomSchema(), testUserName())
|
||||
.with("GRANT SELECT ON SCHEMA :: [cdc] TO %s", testUserName())
|
||||
.with("USE [master]")
|
||||
.with("GRANT VIEW SERVER STATE TO %s", testUserName())
|
||||
.with("USE [%s]", testdb.getDatabaseName())
|
||||
.with("EXEC sp_addrolemember N'%s', N'%s';", CDC_ROLE_NAME, testUserName());
|
||||
|
||||
testDataSource = DataSourceFactory.create(
|
||||
testUserName(),
|
||||
testdb.getPassword(),
|
||||
testdb.getDatabaseDriver().getDriverClassName(),
|
||||
testdb.getJdbcUrl(),
|
||||
Map.of("encrypt", "false"));
|
||||
}
|
||||
|
||||
@Override
|
||||
@AfterEach
|
||||
protected void tearDown() {
|
||||
try {
|
||||
DataSourceFactory.close(testDataSource);
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
super.tearDown();
|
||||
|
||||
}
|
||||
|
||||
private JdbcDatabase testDatabase() {
|
||||
return new DefaultJdbcDatabase(testDataSource);
|
||||
}
|
||||
|
||||
// TODO : Delete this Override when MSSQL supports individual table snapshot
|
||||
@@ -189,53 +174,7 @@ public class CdcMssqlSourceTest extends CdcSourceTest {
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String randomTableSchema() {
|
||||
return MODELS_SCHEMA + "_random";
|
||||
}
|
||||
|
||||
private void switchCdcOnDatabase(final Boolean enable, final String db) {
|
||||
final String storedProc = enable ? "sys.sp_cdc_enable_db" : "sys.sp_cdc_disable_db";
|
||||
executeQuery("USE [" + db + "]\n" + "EXEC " + storedProc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void createTable(final String schemaName, final String tableName, final String columnClause) {
|
||||
switchCdcOnDatabase(true, dbName);
|
||||
super.createTable(schemaName, tableName, columnClause);
|
||||
|
||||
// sometimes seeing an error that we can't enable cdc on a table while sql server agent is still
|
||||
// spinning up
|
||||
// solving with a simple while retry loop
|
||||
boolean failingToStart = true;
|
||||
int retryNum = 0;
|
||||
final int maxRetries = 10;
|
||||
while (failingToStart) {
|
||||
try {
|
||||
executeQuery(String.format(
|
||||
"EXEC sys.sp_cdc_enable_table\n"
|
||||
+ "\t@source_schema = N'%s',\n"
|
||||
+ "\t@source_name = N'%s', \n"
|
||||
+ "\t@role_name = N'%s',\n"
|
||||
+ "\t@supports_net_changes = 0",
|
||||
schemaName, tableName, CDC_ROLE_NAME)); // enables cdc on MODELS_SCHEMA.MODELS_STREAM_NAME, giving CDC_ROLE_NAME select access
|
||||
failingToStart = false;
|
||||
} catch (final Exception e) {
|
||||
if (retryNum >= maxRetries) {
|
||||
throw e;
|
||||
} else {
|
||||
retryNum++;
|
||||
try {
|
||||
Thread.sleep(10000); // 10 seconds
|
||||
} catch (final InterruptedException ex) {
|
||||
throw new RuntimeException(ex);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String columnClause(final Map<String, String> columnsWithDataType, final Optional<String> primaryKey) {
|
||||
protected String columnClause(final Map<String, String> columnsWithDataType, final Optional<String> primaryKey) {
|
||||
final StringBuilder columnClause = new StringBuilder();
|
||||
int i = 0;
|
||||
for (final Map.Entry<String, String> column : columnsWithDataType.entrySet()) {
|
||||
@@ -254,59 +193,42 @@ public class CdcMssqlSourceTest extends CdcSourceTest {
|
||||
return columnClause.toString();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() {
|
||||
try {
|
||||
dslContext.close();
|
||||
DataSourceFactory.close(dataSource);
|
||||
DataSourceFactory.close(testDataSource);
|
||||
} catch (final Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testAssertCdcEnabledInDb() {
|
||||
// since we enable cdc in setup, assert that we successfully pass this first
|
||||
assertDoesNotThrow(() -> source.assertCdcEnabledInDb(config, testJdbcDatabase));
|
||||
assertDoesNotThrow(() -> source().assertCdcEnabledInDb(config(), testDatabase()));
|
||||
// then disable cdc and assert the check fails
|
||||
switchCdcOnDatabase(false, dbName);
|
||||
assertThrows(RuntimeException.class, () -> source.assertCdcEnabledInDb(config, testJdbcDatabase));
|
||||
testdb.withoutCdc();
|
||||
assertThrows(RuntimeException.class, () -> source().assertCdcEnabledInDb(config(), testDatabase()));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testAssertCdcSchemaQueryable() {
|
||||
// correct access granted by setup so assert check passes
|
||||
assertDoesNotThrow(() -> source.assertCdcSchemaQueryable(config, testJdbcDatabase));
|
||||
assertDoesNotThrow(() -> source().assertCdcSchemaQueryable(config(), testDatabase()));
|
||||
// now revoke perms and assert that check fails
|
||||
alterPermissionsOnSchema(false, "cdc");
|
||||
assertThrows(com.microsoft.sqlserver.jdbc.SQLServerException.class, () -> source.assertCdcSchemaQueryable(config, testJdbcDatabase));
|
||||
}
|
||||
|
||||
private void switchSqlServerAgentAndWait(final Boolean start) throws InterruptedException {
|
||||
final String startOrStop = start ? "START" : "STOP";
|
||||
executeQuery(String.format("EXEC xp_servicecontrol N'%s',N'SQLServerAGENT';", startOrStop));
|
||||
Thread.sleep(15 * 1000); // 15 seconds to wait for change of agent state
|
||||
testdb.with("REVOKE SELECT ON SCHEMA :: [cdc] TO %s", testUserName());
|
||||
assertThrows(com.microsoft.sqlserver.jdbc.SQLServerException.class,
|
||||
() -> source().assertCdcSchemaQueryable(config(), testDatabase()));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testAssertSqlServerAgentRunning() throws InterruptedException {
|
||||
executeQuery(String.format("USE master;\n" + "GRANT VIEW SERVER STATE TO %s", testUserName));
|
||||
void testAssertSqlServerAgentRunning() {
|
||||
testdb.withAgentStopped().withWaitUntilAgentStopped();
|
||||
// assert expected failure if sql server agent stopped
|
||||
switchSqlServerAgentAndWait(false);
|
||||
assertThrows(RuntimeException.class, () -> source.assertSqlServerAgentRunning(testJdbcDatabase));
|
||||
assertThrows(RuntimeException.class, () -> source().assertSqlServerAgentRunning(testDatabase()));
|
||||
// assert success if sql server agent running
|
||||
switchSqlServerAgentAndWait(true);
|
||||
assertDoesNotThrow(() -> source.assertSqlServerAgentRunning(testJdbcDatabase));
|
||||
testdb.withAgentStarted().withWaitUntilAgentRunning();
|
||||
assertDoesNotThrow(() -> source().assertSqlServerAgentRunning(testDatabase()));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testAssertSnapshotIsolationAllowed() {
|
||||
// snapshot isolation enabled by setup so assert check passes
|
||||
assertDoesNotThrow(() -> source.assertSnapshotIsolationAllowed(config, testJdbcDatabase));
|
||||
assertDoesNotThrow(() -> source().assertSnapshotIsolationAllowed(config(), testDatabase()));
|
||||
// now disable snapshot isolation and assert that check fails
|
||||
switchSnapshotIsolation(false, dbName);
|
||||
assertThrows(RuntimeException.class, () -> source.assertSnapshotIsolationAllowed(config, testJdbcDatabase));
|
||||
testdb.withoutSnapshotIsolation();
|
||||
assertThrows(RuntimeException.class, () -> source().assertSnapshotIsolationAllowed(config(), testDatabase()));
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -317,10 +239,11 @@ public class CdcMssqlSourceTest extends CdcSourceTest {
|
||||
// set snapshot_isolation level to "Read Committed" to disable snapshot
|
||||
.put("snapshot_isolation", "Read Committed")
|
||||
.build());
|
||||
final var config = config();
|
||||
Jsons.replaceNestedValue(config, List.of("replication_method"), replicationConfig);
|
||||
assertDoesNotThrow(() -> source.assertSnapshotIsolationAllowed(config, testJdbcDatabase));
|
||||
switchSnapshotIsolation(false, dbName);
|
||||
assertDoesNotThrow(() -> source.assertSnapshotIsolationAllowed(config, testJdbcDatabase));
|
||||
assertDoesNotThrow(() -> source().assertSnapshotIsolationAllowed(config, testDatabase()));
|
||||
testdb.withoutSnapshotIsolation();
|
||||
assertDoesNotThrow(() -> source().assertSnapshotIsolationAllowed(config, testDatabase()));
|
||||
}
|
||||
|
||||
// Ensure the CDC check operations are included when CDC is enabled
|
||||
@@ -328,47 +251,52 @@ public class CdcMssqlSourceTest extends CdcSourceTest {
|
||||
@Test
|
||||
void testCdcCheckOperations() throws Exception {
|
||||
// assertCdcEnabledInDb
|
||||
switchCdcOnDatabase(false, dbName);
|
||||
AirbyteConnectionStatus status = getSource().check(getConfig());
|
||||
testdb.withoutCdc();
|
||||
AirbyteConnectionStatus status = source().check(config());
|
||||
assertEquals(status.getStatus(), AirbyteConnectionStatus.Status.FAILED);
|
||||
switchCdcOnDatabase(true, dbName);
|
||||
testdb.withCdc();
|
||||
// assertCdcSchemaQueryable
|
||||
alterPermissionsOnSchema(false, "cdc");
|
||||
status = getSource().check(getConfig());
|
||||
testdb.with("REVOKE SELECT ON SCHEMA :: [cdc] TO %s", testUserName());
|
||||
status = source().check(config());
|
||||
assertEquals(status.getStatus(), AirbyteConnectionStatus.Status.FAILED);
|
||||
alterPermissionsOnSchema(true, "cdc");
|
||||
testdb.with("GRANT SELECT ON SCHEMA :: [cdc] TO %s", testUserName());
|
||||
|
||||
// assertSqlServerAgentRunning
|
||||
executeQuery(String.format("USE master;\n" + "GRANT VIEW SERVER STATE TO %s", testUserName));
|
||||
switchSqlServerAgentAndWait(false);
|
||||
status = getSource().check(getConfig());
|
||||
|
||||
testdb.withAgentStopped().withWaitUntilAgentStopped();
|
||||
status = source().check(config());
|
||||
assertEquals(status.getStatus(), AirbyteConnectionStatus.Status.FAILED);
|
||||
switchSqlServerAgentAndWait(true);
|
||||
testdb.withAgentStarted().withWaitUntilAgentRunning();
|
||||
// assertSnapshotIsolationAllowed
|
||||
switchSnapshotIsolation(false, dbName);
|
||||
status = getSource().check(getConfig());
|
||||
testdb.withoutSnapshotIsolation();
|
||||
status = source().check(config());
|
||||
assertEquals(status.getStatus(), AirbyteConnectionStatus.Status.FAILED);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testCdcCheckOperationsWithDot() throws Exception {
|
||||
// assertCdcEnabledInDb and validate escape with special character
|
||||
switchCdcOnDatabase(true, dbNamewithDot);
|
||||
final AirbyteConnectionStatus status = getSource().check(getConfig());
|
||||
final String dbNameWithDot = testdb.getDatabaseName().replace("_", ".");
|
||||
testdb.with("CREATE DATABASE [%s];", dbNameWithDot)
|
||||
.with("USE [%s]", dbNameWithDot)
|
||||
.with("EXEC sys.sp_cdc_enable_db;");
|
||||
final AirbyteConnectionStatus status = source().check(config());
|
||||
assertEquals(status.getStatus(), AirbyteConnectionStatus.Status.SUCCEEDED);
|
||||
}
|
||||
|
||||
// todo: check LSN returned is actually the max LSN
|
||||
// todo: check we fail as expected under certain conditions
|
||||
@Test
|
||||
void testGetTargetPosition() throws InterruptedException {
|
||||
Thread.sleep(10 * 1000); // Sleeping because sometimes the db is not yet completely ready and the lsn is not found
|
||||
void testGetTargetPosition() {
|
||||
// check that getTargetPosition returns higher Lsn after inserting new row
|
||||
final Lsn firstLsn = MssqlCdcTargetPosition.getTargetPosition(testJdbcDatabase, dbName).targetLsn;
|
||||
executeQuery(String.format("USE %s; INSERT INTO %s.%s (%s, %s, %s) VALUES (%s, %s, '%s');",
|
||||
dbName, MODELS_SCHEMA, MODELS_STREAM_NAME, COL_ID, COL_MAKE_ID, COL_MODEL, 910019, 1, "another car"));
|
||||
Thread.sleep(15 * 1000); // 15 seconds to wait for Agent capture job to log cdc change
|
||||
final Lsn secondLsn = MssqlCdcTargetPosition.getTargetPosition(testJdbcDatabase, dbName).targetLsn;
|
||||
assertTrue(secondLsn.compareTo(firstLsn) > 0);
|
||||
testdb.withWaitUntilMaxLsnAvailable();
|
||||
final Lsn firstLsn = MssqlCdcTargetPosition.getTargetPosition(testDatabase(), testdb.getDatabaseName()).targetLsn;
|
||||
testdb.with("INSERT INTO %s.%s (%s, %s, %s) VALUES (%s, %s, '%s');",
|
||||
modelsSchema(), MODELS_STREAM_NAME, COL_ID, COL_MAKE_ID, COL_MODEL, 910019, 1, "another car");
|
||||
// Wait for Agent capture job to log CDC change.
|
||||
await().atMost(Duration.ofSeconds(45)).until(() -> {
|
||||
final Lsn secondLsn = MssqlCdcTargetPosition.getTargetPosition(testDatabase(), testdb.getDatabaseName()).targetLsn;
|
||||
return secondLsn.compareTo(firstLsn) > 0;
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -382,24 +310,12 @@ public class CdcMssqlSourceTest extends CdcSourceTest {
|
||||
|
||||
@Override
|
||||
protected MssqlCdcTargetPosition cdcLatestTargetPosition() {
|
||||
try {
|
||||
// Sleeping because sometimes the db is not yet completely ready and the lsn is not found
|
||||
Thread.sleep(5000);
|
||||
} catch (final InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
testdb.withWaitUntilMaxLsnAvailable();
|
||||
final JdbcDatabase jdbcDatabase = new StreamingJdbcDatabase(
|
||||
DataSourceFactory.create(config.get(JdbcUtils.USERNAME_KEY).asText(),
|
||||
config.get(JdbcUtils.PASSWORD_KEY).asText(),
|
||||
DRIVER_CLASS,
|
||||
String.format("jdbc:sqlserver://%s:%s;databaseName=%s;",
|
||||
config.get(JdbcUtils.HOST_KEY).asText(),
|
||||
config.get(JdbcUtils.PORT_KEY).asInt(),
|
||||
dbName),
|
||||
Map.of("encrypt", "false")),
|
||||
testDataSource,
|
||||
new MssqlSourceOperations(),
|
||||
AdaptiveStreamingQueryConfig::new);
|
||||
return MssqlCdcTargetPosition.getTargetPosition(jdbcDatabase, dbName);
|
||||
return MssqlCdcTargetPosition.getTargetPosition(jdbcDatabase, testdb.getDatabaseName());
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -451,21 +367,6 @@ public class CdcMssqlSourceTest extends CdcSourceTest {
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Source getSource() {
|
||||
return new MssqlSource();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected JsonNode getConfig() {
|
||||
return config;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Database getDatabase() {
|
||||
return database;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void assertExpectedStateMessages(final List<AirbyteStateMessage> stateMessages) {
|
||||
assertEquals(1, stateMessages.size());
|
||||
|
||||
@@ -9,102 +9,45 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.airbyte.cdk.db.factory.DataSourceFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
import io.airbyte.cdk.db.jdbc.DefaultJdbcDatabase;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcDatabase;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.source.jdbc.AbstractJdbcSource;
|
||||
import io.airbyte.cdk.integrations.source.jdbc.test.JdbcSourceAcceptanceTest;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.commons.string.Strings;
|
||||
import io.airbyte.protocol.models.Field;
|
||||
import io.airbyte.protocol.models.JsonSchemaType;
|
||||
import io.airbyte.protocol.models.v0.AirbyteCatalog;
|
||||
import io.airbyte.protocol.models.v0.AirbyteConnectionStatus;
|
||||
import io.airbyte.protocol.models.v0.CatalogHelpers;
|
||||
import io.airbyte.protocol.models.v0.SyncMode;
|
||||
import java.sql.JDBCType;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import javax.sql.DataSource;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.testcontainers.containers.MSSQLServerContainer;
|
||||
|
||||
public class MssqlJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest {
|
||||
public class MssqlJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest<MssqlSource, MsSQLTestDatabase> {
|
||||
|
||||
protected static final String USERNAME_WITHOUT_PERMISSION = "new_user";
|
||||
protected static final String PASSWORD_WITHOUT_PERMISSION = "password_3435!";
|
||||
private static MSSQLServerContainer<?> dbContainer;
|
||||
private JsonNode config;
|
||||
|
||||
@BeforeAll
|
||||
static void init() {
|
||||
static {
|
||||
// In mssql, timestamp is generated automatically, so we need to use
|
||||
// the datetime type instead so that we can set the value manually.
|
||||
COL_TIMESTAMP_TYPE = "DATETIME2";
|
||||
|
||||
dbContainer = new MSSQLServerContainer<>("mcr.microsoft.com/mssql/server:2019-latest").acceptLicense();
|
||||
dbContainer.start();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DataSource getDataSource(final JsonNode jdbcConfig) {
|
||||
final Map<String, String> connectionProperties = JdbcUtils.parseJdbcParameters(jdbcConfig, JdbcUtils.CONNECTION_PROPERTIES_KEY,
|
||||
getJdbcParameterDelimiter());
|
||||
connectionProperties.put("encrypt", "false");
|
||||
return DataSourceFactory.create(
|
||||
jdbcConfig.get(JdbcUtils.USERNAME_KEY).asText(),
|
||||
jdbcConfig.has(JdbcUtils.PASSWORD_KEY) ? jdbcConfig.get(JdbcUtils.PASSWORD_KEY).asText() : null,
|
||||
getDriverClass(),
|
||||
jdbcConfig.get(JdbcUtils.JDBC_URL_KEY).asText(),
|
||||
connectionProperties);
|
||||
protected JsonNode config() {
|
||||
return testdb.testConfigBuilder()
|
||||
.withoutSsl()
|
||||
.build();
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
public void setup() throws Exception {
|
||||
final JsonNode configWithoutDbName = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, dbContainer.getHost())
|
||||
.put(JdbcUtils.PORT_KEY, dbContainer.getFirstMappedPort())
|
||||
.put(JdbcUtils.USERNAME_KEY, dbContainer.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, dbContainer.getPassword())
|
||||
.build());
|
||||
|
||||
final DataSource dataSource = DataSourceFactory.create(
|
||||
configWithoutDbName.get(JdbcUtils.USERNAME_KEY).asText(),
|
||||
configWithoutDbName.get(JdbcUtils.PASSWORD_KEY).asText(),
|
||||
DatabaseDriver.MSSQLSERVER.getDriverClassName(),
|
||||
String.format("jdbc:sqlserver://%s:%d;",
|
||||
configWithoutDbName.get(JdbcUtils.HOST_KEY).asText(),
|
||||
configWithoutDbName.get(JdbcUtils.PORT_KEY).asInt()),
|
||||
Map.of("encrypt", "false"));
|
||||
|
||||
try {
|
||||
final JdbcDatabase database = new DefaultJdbcDatabase(dataSource);
|
||||
|
||||
final String dbName = Strings.addRandomSuffix("db", "_", 10).toLowerCase();
|
||||
|
||||
database.execute(ctx -> ctx.createStatement().execute(String.format("CREATE DATABASE %s;", dbName)));
|
||||
|
||||
config = Jsons.clone(configWithoutDbName);
|
||||
((ObjectNode) config).put(JdbcUtils.DATABASE_KEY, dbName);
|
||||
((ObjectNode) config).put("ssl_method", Jsons.jsonNode(Map.of("ssl_method", "unencrypted")));
|
||||
|
||||
super.setup();
|
||||
} finally {
|
||||
DataSourceFactory.close(dataSource);
|
||||
}
|
||||
@Override
|
||||
protected MssqlSource source() {
|
||||
return new MssqlSource();
|
||||
}
|
||||
|
||||
@AfterAll
|
||||
public static void cleanUp() throws Exception {
|
||||
dbContainer.close();
|
||||
@Override
|
||||
protected MsSQLTestDatabase createTestDatabase() {
|
||||
return MsSQLTestDatabase.in("mcr.microsoft.com/mssql/server:2022-latest");
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -113,69 +56,70 @@ public class MssqlJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest {
|
||||
}
|
||||
|
||||
@Override
|
||||
public JsonNode getConfig() {
|
||||
return Jsons.clone(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractJdbcSource<JDBCType> getJdbcSource() {
|
||||
return new MssqlSource();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDriverClass() {
|
||||
return MssqlSource.DRIVER_CLASS;
|
||||
protected void maybeSetShorterConnectionTimeout(final JsonNode config) {
|
||||
((ObjectNode) config).put(JdbcUtils.JDBC_URL_PARAMS_KEY, "loginTimeout=1");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testCheckIncorrectPasswordFailure() throws Exception {
|
||||
final var config = config();
|
||||
maybeSetShorterConnectionTimeout(config);
|
||||
((ObjectNode) config).put(JdbcUtils.PASSWORD_KEY, "fake");
|
||||
final AirbyteConnectionStatus status = source.check(config);
|
||||
final AirbyteConnectionStatus status = source().check(config);
|
||||
Assertions.assertEquals(AirbyteConnectionStatus.Status.FAILED, status.getStatus());
|
||||
assertTrue(status.getMessage().contains("State code: S0001; Error code: 18456;"));
|
||||
assertTrue(status.getMessage().contains("State code: S0001; Error code: 18456;"), status.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckIncorrectUsernameFailure() throws Exception {
|
||||
final var config = config();
|
||||
maybeSetShorterConnectionTimeout(config);
|
||||
((ObjectNode) config).put(JdbcUtils.USERNAME_KEY, "fake");
|
||||
final AirbyteConnectionStatus status = source.check(config);
|
||||
final AirbyteConnectionStatus status = source().check(config);
|
||||
Assertions.assertEquals(AirbyteConnectionStatus.Status.FAILED, status.getStatus());
|
||||
assertTrue(status.getMessage().contains("State code: S0001; Error code: 18456;"));
|
||||
assertTrue(status.getMessage().contains("State code: S0001; Error code: 18456;"), status.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckIncorrectHostFailure() throws Exception {
|
||||
final var config = config();
|
||||
maybeSetShorterConnectionTimeout(config);
|
||||
((ObjectNode) config).put(JdbcUtils.HOST_KEY, "localhost2");
|
||||
final AirbyteConnectionStatus status = source.check(config);
|
||||
final AirbyteConnectionStatus status = source().check(config);
|
||||
Assertions.assertEquals(AirbyteConnectionStatus.Status.FAILED, status.getStatus());
|
||||
assertTrue(status.getMessage().contains("State code: 08S01;"));
|
||||
assertTrue(status.getMessage().contains("State code: 08S01;"), status.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckIncorrectPortFailure() throws Exception {
|
||||
final var config = config();
|
||||
maybeSetShorterConnectionTimeout(config);
|
||||
((ObjectNode) config).put(JdbcUtils.PORT_KEY, "0000");
|
||||
final AirbyteConnectionStatus status = source.check(config);
|
||||
final AirbyteConnectionStatus status = source().check(config);
|
||||
Assertions.assertEquals(AirbyteConnectionStatus.Status.FAILED, status.getStatus());
|
||||
assertTrue(status.getMessage().contains("State code: 08S01;"));
|
||||
assertTrue(status.getMessage().contains("State code: 08S01;"), status.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckIncorrectDataBaseFailure() throws Exception {
|
||||
final var config = config();
|
||||
maybeSetShorterConnectionTimeout(config);
|
||||
((ObjectNode) config).put(JdbcUtils.DATABASE_KEY, "wrongdatabase");
|
||||
final AirbyteConnectionStatus status = source.check(config);
|
||||
final AirbyteConnectionStatus status = source().check(config);
|
||||
Assertions.assertEquals(AirbyteConnectionStatus.Status.FAILED, status.getStatus());
|
||||
assertTrue(status.getMessage().contains("State code: S0001; Error code: 4060;"));
|
||||
assertTrue(status.getMessage().contains("State code: S0001; Error code: 4060;"), status.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUserHasNoPermissionToDataBase() throws Exception {
|
||||
database.execute(ctx -> ctx.createStatement()
|
||||
.execute(String.format("CREATE LOGIN %s WITH PASSWORD = '%s'; ", USERNAME_WITHOUT_PERMISSION, PASSWORD_WITHOUT_PERMISSION)));
|
||||
final var config = config();
|
||||
maybeSetShorterConnectionTimeout(config);
|
||||
testdb.with("CREATE LOGIN %s WITH PASSWORD = '%s'; ", USERNAME_WITHOUT_PERMISSION, PASSWORD_WITHOUT_PERMISSION);
|
||||
((ObjectNode) config).put(JdbcUtils.USERNAME_KEY, USERNAME_WITHOUT_PERMISSION);
|
||||
((ObjectNode) config).put(JdbcUtils.PASSWORD_KEY, PASSWORD_WITHOUT_PERMISSION);
|
||||
final AirbyteConnectionStatus status = source.check(config);
|
||||
final AirbyteConnectionStatus status = source().check(config);
|
||||
assertEquals(AirbyteConnectionStatus.Status.FAILED, status.getStatus());
|
||||
assertTrue(status.getMessage().contains("State code: S0001; Error code: 4060;"));
|
||||
assertTrue(status.getMessage().contains("State code: S0001; Error code: 4060;"), status.getMessage());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -9,17 +9,8 @@ import static org.assertj.core.api.AssertionsForClassTypes.catchThrowable;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DataSourceFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.commons.exceptions.ConfigErrorException;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.commons.string.Strings;
|
||||
import io.airbyte.commons.util.MoreIterators;
|
||||
import io.airbyte.protocol.models.Field;
|
||||
import io.airbyte.protocol.models.JsonSchemaType;
|
||||
@@ -29,67 +20,44 @@ import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog;
|
||||
import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream;
|
||||
import io.airbyte.protocol.models.v0.DestinationSyncMode;
|
||||
import io.airbyte.protocol.models.v0.SyncMode;
|
||||
import java.sql.SQLException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import org.jooq.DSLContext;
|
||||
import org.junit.jupiter.api.*;
|
||||
import org.testcontainers.containers.MSSQLServerContainer;
|
||||
|
||||
class MssqlSourceTest {
|
||||
|
||||
private static final String DB_NAME = "dbo";
|
||||
private static final String STREAM_NAME = "id_and_name";
|
||||
private static final AirbyteCatalog CATALOG = new AirbyteCatalog().withStreams(Lists.newArrayList(CatalogHelpers.createAirbyteStream(
|
||||
STREAM_NAME,
|
||||
DB_NAME,
|
||||
"dbo",
|
||||
Field.of("id", JsonSchemaType.INTEGER),
|
||||
Field.of("name", JsonSchemaType.STRING),
|
||||
Field.of("born", JsonSchemaType.STRING_TIMESTAMP_WITH_TIMEZONE))
|
||||
.withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL))
|
||||
.withSourceDefinedPrimaryKey(List.of(List.of("id")))));
|
||||
|
||||
private JsonNode configWithoutDbName;
|
||||
private JsonNode config;
|
||||
|
||||
private static MSSQLServerContainer<?> db;
|
||||
|
||||
@BeforeAll
|
||||
static void init() {
|
||||
db = new MSSQLServerContainer<>("mcr.microsoft.com/mssql/server:2019-latest").acceptLicense();
|
||||
db.start();
|
||||
}
|
||||
private MsSQLTestDatabase testdb;
|
||||
|
||||
// how to interact with the mssql test container manaully.
|
||||
// 1. exec into mssql container (not the test container container)
|
||||
// 2. /opt/mssql-tools/bin/sqlcmd -S localhost -U SA -P "A_Str0ng_Required_Password"
|
||||
@BeforeEach
|
||||
void setup() throws SQLException {
|
||||
configWithoutDbName = getConfig(db);
|
||||
final String dbName = Strings.addRandomSuffix("db", "_", 10).toLowerCase();
|
||||
|
||||
try (final DSLContext dslContext = getDslContext(configWithoutDbName)) {
|
||||
final Database database = getDatabase(dslContext);
|
||||
database.query(ctx -> {
|
||||
ctx.fetch(String.format("CREATE DATABASE %s;", dbName));
|
||||
ctx.fetch(String.format("USE %s;", dbName));
|
||||
ctx.fetch("CREATE TABLE id_and_name(id INTEGER NOT NULL, name VARCHAR(200), born DATETIMEOFFSET(7));");
|
||||
ctx.fetch(
|
||||
"INSERT INTO id_and_name (id, name, born) VALUES (1,'picard', '2124-03-04T01:01:01Z'), (2, 'crusher', '2124-03-04T01:01:01Z'), (3, 'vash', '2124-03-04T01:01:01Z');");
|
||||
return null;
|
||||
});
|
||||
}
|
||||
|
||||
config = Jsons.clone(configWithoutDbName);
|
||||
((ObjectNode) config).put(JdbcUtils.DATABASE_KEY, dbName);
|
||||
((ObjectNode) config).put("ssl_method", Jsons.jsonNode(Map.of("ssl_method", "unencrypted")));
|
||||
void setup() {
|
||||
testdb = MsSQLTestDatabase.in("mcr.microsoft.com/mssql/server:2022-latest")
|
||||
.with("CREATE TABLE id_and_name(id INTEGER NOT NULL, name VARCHAR(200), born DATETIMEOFFSET(7));")
|
||||
.with("INSERT INTO id_and_name (id, name, born) VALUES (1,'picard', '2124-03-04T01:01:01Z'), (2, 'crusher', " +
|
||||
"'2124-03-04T01:01:01Z'), (3, 'vash', '2124-03-04T01:01:01Z');");
|
||||
}
|
||||
|
||||
@AfterAll
|
||||
static void cleanUp() {
|
||||
db.stop();
|
||||
db.close();
|
||||
@AfterEach
|
||||
void cleanUp() {
|
||||
testdb.close();
|
||||
}
|
||||
|
||||
private JsonNode getConfig() {
|
||||
return testdb.testConfigBuilder()
|
||||
.withoutSsl()
|
||||
.build();
|
||||
}
|
||||
|
||||
// if a column in mssql is used as a primary key and in a separate index the discover query returns
|
||||
@@ -97,82 +65,43 @@ class MssqlSourceTest {
|
||||
// this tests that this de-duplication is successful.
|
||||
@Test
|
||||
void testDiscoverWithPk() throws Exception {
|
||||
try (final DSLContext dslContext = getDslContext(configWithoutDbName)) {
|
||||
final Database database = getDatabase(dslContext);
|
||||
database.query(ctx -> {
|
||||
ctx.fetch(String.format("USE %s;", config.get(JdbcUtils.DATABASE_KEY)));
|
||||
ctx.execute("ALTER TABLE id_and_name ADD CONSTRAINT i3pk PRIMARY KEY CLUSTERED (id);");
|
||||
ctx.execute("CREATE INDEX i1 ON id_and_name (id);");
|
||||
return null;
|
||||
});
|
||||
}
|
||||
|
||||
final AirbyteCatalog actual = new MssqlSource().discover(config);
|
||||
testdb
|
||||
.with("ALTER TABLE id_and_name ADD CONSTRAINT i3pk PRIMARY KEY CLUSTERED (id);")
|
||||
.with("CREATE INDEX i1 ON id_and_name (id);");
|
||||
final AirbyteCatalog actual = new MssqlSource().discover(getConfig());
|
||||
assertEquals(CATALOG, actual);
|
||||
}
|
||||
|
||||
@Test
|
||||
@Disabled("See https://github.com/airbytehq/airbyte/pull/23908#issuecomment-1463753684, enable once communication is out")
|
||||
public void testTableWithNullCursorValueShouldThrowException() throws Exception {
|
||||
try (final DSLContext dslContext = getDslContext(configWithoutDbName)) {
|
||||
final Database database = getDatabase(dslContext);
|
||||
database.query(ctx -> {
|
||||
ctx.fetch(String.format("USE %s;", config.get(JdbcUtils.DATABASE_KEY)));
|
||||
ctx.execute("ALTER TABLE id_and_name ALTER COLUMN id INTEGER NULL");
|
||||
ctx.execute("INSERT INTO id_and_name(id) VALUES (7), (8), (NULL)");
|
||||
return null;
|
||||
});
|
||||
testdb
|
||||
.with("ALTER TABLE id_and_name ALTER COLUMN id INTEGER NULL")
|
||||
.with("INSERT INTO id_and_name(id) VALUES (7), (8), (NULL)");
|
||||
|
||||
ConfiguredAirbyteStream configuredAirbyteStream = new ConfiguredAirbyteStream().withSyncMode(
|
||||
SyncMode.INCREMENTAL)
|
||||
.withCursorField(Lists.newArrayList("id"))
|
||||
.withDestinationSyncMode(DestinationSyncMode.APPEND)
|
||||
.withSyncMode(SyncMode.INCREMENTAL)
|
||||
.withStream(CatalogHelpers.createAirbyteStream(
|
||||
STREAM_NAME,
|
||||
DB_NAME,
|
||||
Field.of("id", JsonSchemaType.INTEGER),
|
||||
Field.of("name", JsonSchemaType.STRING),
|
||||
Field.of("born", JsonSchemaType.STRING))
|
||||
.withSupportedSyncModes(
|
||||
Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL))
|
||||
.withSourceDefinedPrimaryKey(List.of(List.of("id"))));
|
||||
ConfiguredAirbyteStream configuredAirbyteStream = new ConfiguredAirbyteStream().withSyncMode(
|
||||
SyncMode.INCREMENTAL)
|
||||
.withCursorField(Lists.newArrayList("id"))
|
||||
.withDestinationSyncMode(DestinationSyncMode.APPEND)
|
||||
.withSyncMode(SyncMode.INCREMENTAL)
|
||||
.withStream(CatalogHelpers.createAirbyteStream(
|
||||
STREAM_NAME,
|
||||
testdb.getDatabaseName(),
|
||||
Field.of("id", JsonSchemaType.INTEGER),
|
||||
Field.of("name", JsonSchemaType.STRING),
|
||||
Field.of("born", JsonSchemaType.STRING))
|
||||
.withSupportedSyncModes(
|
||||
Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL))
|
||||
.withSourceDefinedPrimaryKey(List.of(List.of("id"))));
|
||||
|
||||
final ConfiguredAirbyteCatalog catalog = new ConfiguredAirbyteCatalog().withStreams(
|
||||
Collections.singletonList(configuredAirbyteStream));
|
||||
final ConfiguredAirbyteCatalog catalog = new ConfiguredAirbyteCatalog().withStreams(
|
||||
Collections.singletonList(configuredAirbyteStream));
|
||||
|
||||
final Throwable throwable = catchThrowable(() -> MoreIterators.toSet(
|
||||
new MssqlSource().read(config, catalog, null)));
|
||||
assertThat(throwable).isInstanceOf(ConfigErrorException.class)
|
||||
.hasMessageContaining(
|
||||
"The following tables have invalid columns selected as cursor, please select a column with a well-defined ordering with no null values as a cursor. {tableName='dbo.id_and_name', cursorColumnName='id', cursorSqlType=INTEGER, cause=Cursor column contains NULL value}");
|
||||
}
|
||||
}
|
||||
|
||||
private JsonNode getConfig(final MSSQLServerContainer<?> db) {
|
||||
return Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, db.getHost())
|
||||
.put(JdbcUtils.PORT_KEY, db.getFirstMappedPort())
|
||||
.put(JdbcUtils.USERNAME_KEY, db.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, db.getPassword())
|
||||
.build());
|
||||
}
|
||||
|
||||
private static DSLContext getDslContext(final JsonNode config) {
|
||||
return DSLContextFactory.create(DataSourceFactory.create(
|
||||
config.get(JdbcUtils.USERNAME_KEY).asText(),
|
||||
config.get(JdbcUtils.PASSWORD_KEY).asText(),
|
||||
DatabaseDriver.MSSQLSERVER.getDriverClassName(),
|
||||
String.format("jdbc:sqlserver://%s:%d;",
|
||||
config.get(JdbcUtils.HOST_KEY).asText(),
|
||||
config.get(JdbcUtils.PORT_KEY).asInt()),
|
||||
Map.of("encrypt", "false")), null);
|
||||
}
|
||||
|
||||
public static Database getDatabase(final DSLContext dslContext) {
|
||||
// todo (cgardens) - rework this abstraction so that we do not have to pass a null into the
|
||||
// constructor. at least explicitly handle it, even if the impl doesn't change.
|
||||
return new Database(dslContext);
|
||||
final Throwable throwable = catchThrowable(() -> MoreIterators.toSet(
|
||||
new MssqlSource().read(getConfig(), catalog, null)));
|
||||
assertThat(throwable).isInstanceOf(ConfigErrorException.class)
|
||||
.hasMessageContaining(
|
||||
"The following tables have invalid columns selected as cursor, please select a column with a well-defined ordering with no null values as a cursor. {tableName='dbo.id_and_name', cursorColumnName='id', cursorSqlType=INTEGER, cause=Cursor column contains NULL value}");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -0,0 +1,35 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.source.mssql;
|
||||
|
||||
import io.airbyte.cdk.testutils.ContainerFactory;
|
||||
import org.testcontainers.containers.MSSQLServerContainer;
|
||||
import org.testcontainers.containers.Network;
|
||||
import org.testcontainers.utility.DockerImageName;
|
||||
|
||||
public class MsSQLContainerFactory implements ContainerFactory<MSSQLServerContainer<?>> {
|
||||
|
||||
@Override
|
||||
public MSSQLServerContainer<?> createNewContainer(DockerImageName imageName) {
|
||||
return new MSSQLServerContainer<>(imageName.asCompatibleSubstituteFor("mcr.microsoft.com/mssql/server")).acceptLicense();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<?> getContainerClass() {
|
||||
return MSSQLServerContainer.class;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new network and bind it to the container.
|
||||
*/
|
||||
public void withNetwork(MSSQLServerContainer<?> container) {
|
||||
container.withNetwork(Network.newNetwork());
|
||||
}
|
||||
|
||||
public void withAgent(MSSQLServerContainer<?> container) {
|
||||
container.addEnv("MSSQL_AGENT_ENABLED", "True");
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,211 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.source.mssql;
|
||||
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.testutils.TestDatabase;
|
||||
import io.debezium.connector.sqlserver.Lsn;
|
||||
import java.sql.SQLException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.jooq.SQLDialect;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.testcontainers.containers.MSSQLServerContainer;
|
||||
|
||||
public class MsSQLTestDatabase extends TestDatabase<MSSQLServerContainer<?>, MsSQLTestDatabase, MsSQLTestDatabase.MsSQLConfigBuilder> {
|
||||
|
||||
static private final Logger LOGGER = LoggerFactory.getLogger(MsSQLTestDatabase.class);
|
||||
|
||||
static public final int MAX_RETRIES = 60;
|
||||
|
||||
static public MsSQLTestDatabase in(String imageName, String... methods) {
|
||||
final var container = new MsSQLContainerFactory().shared(imageName, methods);
|
||||
final var testdb = new MsSQLTestDatabase(container);
|
||||
return testdb
|
||||
.withConnectionProperty("encrypt", "false")
|
||||
.withConnectionProperty("databaseName", testdb.getDatabaseName())
|
||||
.initialized();
|
||||
}
|
||||
|
||||
public MsSQLTestDatabase(MSSQLServerContainer<?> container) {
|
||||
super(container);
|
||||
}
|
||||
|
||||
public MsSQLTestDatabase withSnapshotIsolation() {
|
||||
return with("ALTER DATABASE %s SET ALLOW_SNAPSHOT_ISOLATION ON;", getDatabaseName());
|
||||
}
|
||||
|
||||
public MsSQLTestDatabase withoutSnapshotIsolation() {
|
||||
return with("ALTER DATABASE %s SET ALLOW_SNAPSHOT_ISOLATION OFF;", getDatabaseName());
|
||||
}
|
||||
|
||||
public MsSQLTestDatabase withCdc() {
|
||||
return with("EXEC sys.sp_cdc_enable_db;");
|
||||
}
|
||||
|
||||
public MsSQLTestDatabase withoutCdc() {
|
||||
return with("EXEC sys.sp_cdc_disable_db;");
|
||||
}
|
||||
|
||||
public MsSQLTestDatabase withAgentStarted() {
|
||||
return with("EXEC master.dbo.xp_servicecontrol N'START', N'SQLServerAGENT';");
|
||||
}
|
||||
|
||||
public MsSQLTestDatabase withAgentStopped() {
|
||||
return with("EXEC master.dbo.xp_servicecontrol N'STOP', N'SQLServerAGENT';");
|
||||
}
|
||||
|
||||
public MsSQLTestDatabase withWaitUntilAgentRunning() {
|
||||
waitForAgentState(true);
|
||||
return self();
|
||||
}
|
||||
|
||||
public MsSQLTestDatabase withWaitUntilAgentStopped() {
|
||||
waitForAgentState(false);
|
||||
return self();
|
||||
}
|
||||
|
||||
private void waitForAgentState(final boolean running) {
|
||||
final String expectedValue = running ? "Running." : "Stopped.";
|
||||
LOGGER.debug("Waiting for SQLServerAgent state to change to '{}'.", expectedValue);
|
||||
for (int i = 0; i < MAX_RETRIES; i++) {
|
||||
try {
|
||||
final var r = query(ctx -> ctx.fetch("EXEC master.dbo.xp_servicecontrol 'QueryState', N'SQLServerAGENT';").get(0));
|
||||
if (expectedValue.equalsIgnoreCase(r.getValue(0).toString())) {
|
||||
LOGGER.debug("SQLServerAgent state is '{}', as expected.", expectedValue);
|
||||
return;
|
||||
}
|
||||
LOGGER.debug("Retrying, SQLServerAgent state {} does not match expected '{}'.", r, expectedValue);
|
||||
} catch (SQLException e) {
|
||||
LOGGER.debug("Retrying agent state query after catching exception {}.", e.getMessage());
|
||||
}
|
||||
try {
|
||||
Thread.sleep(1_000); // Wait one second between retries.
|
||||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
throw new RuntimeException("Exhausted retry attempts while polling for agent state");
|
||||
}
|
||||
|
||||
public MsSQLTestDatabase withWaitUntilMaxLsnAvailable() {
|
||||
LOGGER.debug("Waiting for max LSN to become available for database {}.", getDatabaseName());
|
||||
for (int i = 0; i < MAX_RETRIES; i++) {
|
||||
try {
|
||||
final var maxLSN = query(ctx -> ctx.fetch("SELECT sys.fn_cdc_get_max_lsn();").get(0).get(0, byte[].class));
|
||||
if (maxLSN != null) {
|
||||
LOGGER.debug("Max LSN available for database {}: {}", getDatabaseName(), Lsn.valueOf(maxLSN));
|
||||
return self();
|
||||
}
|
||||
LOGGER.debug("Retrying, max LSN still not available for database {}.", getDatabaseName());
|
||||
} catch (SQLException e) {
|
||||
LOGGER.warn("Retrying max LSN query after catching exception {}", e.getMessage());
|
||||
}
|
||||
try {
|
||||
Thread.sleep(1_000); // Wait one second between retries.
|
||||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
throw new RuntimeException("Exhausted retry attempts while polling for max LSN availability");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPassword() {
|
||||
return "S00p3rS33kr3tP4ssw0rd!";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getJdbcUrl() {
|
||||
return String.format("jdbc:sqlserver://%s:%d", getContainer().getHost(), getContainer().getFirstMappedPort());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Stream<Stream<String>> inContainerBootstrapCmd() {
|
||||
return Stream.of(
|
||||
mssqlCmd(Stream.of(String.format("CREATE DATABASE %s", getDatabaseName()))),
|
||||
mssqlCmd(Stream.of(
|
||||
String.format("USE %s", getDatabaseName()),
|
||||
String.format("CREATE LOGIN %s WITH PASSWORD = '%s', DEFAULT_DATABASE = %s", getUserName(), getPassword(), getDatabaseName()),
|
||||
String.format("ALTER SERVER ROLE [sysadmin] ADD MEMBER %s", getUserName()),
|
||||
String.format("CREATE USER %s FOR LOGIN %s WITH DEFAULT_SCHEMA = [dbo]", getUserName(), getUserName()),
|
||||
String.format("ALTER ROLE [db_owner] ADD MEMBER %s", getUserName()))));
|
||||
}
|
||||
|
||||
/**
|
||||
* Don't drop anything when closing the test database. Instead, if cleanup is required, call
|
||||
* {@link #dropDatabaseAndUser()} explicitly. Implicit cleanups may result in deadlocks and so
|
||||
* aren't really worth it.
|
||||
*/
|
||||
@Override
|
||||
protected Stream<String> inContainerUndoBootstrapCmd() {
|
||||
return Stream.empty();
|
||||
}
|
||||
|
||||
public void dropDatabaseAndUser() {
|
||||
execInContainer(mssqlCmd(Stream.of(
|
||||
String.format("USE master"),
|
||||
String.format("ALTER DATABASE %s SET single_user WITH ROLLBACK IMMEDIATE", getDatabaseName()),
|
||||
String.format("DROP DATABASE %s", getDatabaseName()))));
|
||||
}
|
||||
|
||||
public Stream<String> mssqlCmd(Stream<String> sql) {
|
||||
return Stream.of("/opt/mssql-tools/bin/sqlcmd",
|
||||
"-U", getContainer().getUsername(),
|
||||
"-P", getContainer().getPassword(),
|
||||
"-Q", sql.collect(Collectors.joining("; ")),
|
||||
"-b", "-e");
|
||||
}
|
||||
|
||||
@Override
|
||||
public DatabaseDriver getDatabaseDriver() {
|
||||
return DatabaseDriver.MSSQLSERVER;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SQLDialect getSqlDialect() {
|
||||
return SQLDialect.DEFAULT;
|
||||
}
|
||||
|
||||
@Override
|
||||
public MsSQLConfigBuilder configBuilder() {
|
||||
return new MsSQLConfigBuilder(this);
|
||||
}
|
||||
|
||||
static public class MsSQLConfigBuilder extends ConfigBuilder<MsSQLTestDatabase, MsSQLConfigBuilder> {
|
||||
|
||||
protected MsSQLConfigBuilder(MsSQLTestDatabase testDatabase) {
|
||||
super(testDatabase);
|
||||
}
|
||||
|
||||
public MsSQLConfigBuilder withCdcReplication() {
|
||||
return with("replication_method", Map.of(
|
||||
"method", "CDC",
|
||||
"data_to_sync", "Existing and New",
|
||||
"initial_waiting_seconds", DEFAULT_CDC_REPLICATION_INITIAL_WAIT.getSeconds(),
|
||||
"snapshot_isolation", "Snapshot"));
|
||||
}
|
||||
|
||||
public MsSQLConfigBuilder withSchemas(String... schemas) {
|
||||
return with(JdbcUtils.SCHEMAS_KEY, List.of(schemas));
|
||||
}
|
||||
|
||||
@Override
|
||||
public MsSQLConfigBuilder withoutSsl() {
|
||||
return withSsl(Map.of("ssl_method", "unencrypted"));
|
||||
}
|
||||
|
||||
@Override
|
||||
public MsSQLConfigBuilder withSsl(Map<Object, Object> sslMode) {
|
||||
return with("ssl_method", sslMode);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
@@ -4,12 +4,12 @@ plugins {
|
||||
}
|
||||
|
||||
airbyteJavaConnector {
|
||||
cdkVersionRequired = '0.4.1'
|
||||
cdkVersionRequired = '0.5.0'
|
||||
features = ['db-sources']
|
||||
useLocalCdk = false
|
||||
}
|
||||
|
||||
airbyteJavaConnector.addCdkDependencies()
|
||||
|
||||
|
||||
configurations.all {
|
||||
resolutionStrategy {
|
||||
@@ -26,6 +26,7 @@ dependencies {
|
||||
implementation project(':airbyte-integrations:connectors:source-mysql')
|
||||
implementation libs.jooq
|
||||
|
||||
testImplementation testFixtures(project(':airbyte-integrations:connectors:source-mysql'))
|
||||
testImplementation libs.junit.jupiter.system.stubs
|
||||
testImplementation 'org.hamcrest:hamcrest-all:1.3'
|
||||
testImplementation libs.testcontainers.mysql
|
||||
|
||||
@@ -0,0 +1 @@
|
||||
testExecutionConcurrency=-1
|
||||
@@ -11,7 +11,7 @@ data:
|
||||
connectorSubtype: database
|
||||
connectorType: source
|
||||
definitionId: 435bb9a5-7887-4809-aa58-28c27df0d7ad
|
||||
dockerImageTag: 3.1.7
|
||||
dockerImageTag: 3.1.8
|
||||
dockerRepository: airbyte/source-mysql-strict-encrypt
|
||||
githubIssueLabel: source-mysql
|
||||
icon: mysql.svg
|
||||
|
||||
@@ -41,7 +41,11 @@ public class MySqlStrictEncryptSource extends SpecModifyingSource implements Sou
|
||||
"<li><b>Verify Identity</b> - Always connect with SSL. Verify both CA and Hostname.</li></ul>Read more <a href=\"https://dev.mysql.com/doc/connector-j/8.0/en/connector-j-reference-using-ssl.html\"> in the docs</a>.";
|
||||
|
||||
MySqlStrictEncryptSource() {
|
||||
super(MySqlSource.sshWrappedSource());
|
||||
this(new MySqlSource());
|
||||
}
|
||||
|
||||
MySqlStrictEncryptSource(MySqlSource source) {
|
||||
super(MySqlSource.sshWrappedSource(source));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -1,79 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.source.mysql_strict_encrypt;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.MySqlUtils;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.base.ssh.SshHelpers;
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
|
||||
import io.airbyte.commons.features.EnvVariableFeatureFlags;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import org.jooq.DSLContext;
|
||||
import org.jooq.SQLDialect;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
|
||||
public abstract class AbstractMySqlSslCertificateStrictEncryptSourceAcceptanceTest extends MySqlStrictEncryptSourceAcceptanceTest {
|
||||
|
||||
protected static MySqlUtils.Certificate certs;
|
||||
protected static final String PASSWORD = "Passw0rd";
|
||||
|
||||
@Override
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) throws Exception {
|
||||
environmentVariables.set(EnvVariableFeatureFlags.USE_STREAM_CAPABLE_STATE, "true");
|
||||
container = new MySQLContainer<>("mysql:8.0");
|
||||
container.start();
|
||||
addTestData(container);
|
||||
certs = MySqlUtils.getCertificate(container, true);
|
||||
|
||||
final var sslMode = getSslConfig();
|
||||
final var innerContainerAddress = SshHelpers.getInnerContainerAddress(container);
|
||||
final JsonNode replicationMethod = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("method", "STANDARD")
|
||||
.build());
|
||||
config = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, innerContainerAddress.left)
|
||||
.put(JdbcUtils.PORT_KEY, innerContainerAddress.right)
|
||||
.put(JdbcUtils.DATABASE_KEY, container.getDatabaseName())
|
||||
.put(JdbcUtils.USERNAME_KEY, container.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, container.getPassword())
|
||||
.put(JdbcUtils.SSL_KEY, true)
|
||||
.put(JdbcUtils.SSL_MODE_KEY, sslMode)
|
||||
.put("replication_method", replicationMethod)
|
||||
.build());
|
||||
}
|
||||
|
||||
public abstract ImmutableMap getSslConfig();
|
||||
|
||||
private void addTestData(final MySQLContainer container) throws Exception {
|
||||
final var outerContainerAddress = SshHelpers.getOuterContainerAddress(container);
|
||||
try (final DSLContext dslContext = DSLContextFactory.create(
|
||||
container.getUsername(),
|
||||
container.getPassword(),
|
||||
DatabaseDriver.MYSQL.getDriverClassName(),
|
||||
String.format("jdbc:mysql://%s:%s/%s",
|
||||
outerContainerAddress.left,
|
||||
outerContainerAddress.right,
|
||||
container.getDatabaseName()),
|
||||
SQLDialect.MYSQL)) {
|
||||
final Database database = new Database(dslContext);
|
||||
|
||||
database.query(ctx -> {
|
||||
ctx.fetch("CREATE TABLE id_and_name(id INTEGER, name VARCHAR(200));");
|
||||
ctx.fetch(
|
||||
"INSERT INTO id_and_name (id, name) VALUES (1,'picard'), (2, 'crusher'), (3, 'vash');");
|
||||
ctx.fetch("CREATE TABLE starships(id INTEGER, name VARCHAR(200));");
|
||||
ctx.fetch(
|
||||
"INSERT INTO starships (id, name) VALUES (1,'enterprise-d'), (2, 'defiant'), (3, 'yamato');");
|
||||
return null;
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -4,17 +4,29 @@
|
||||
|
||||
package io.airbyte.integrations.source.mysql_strict_encrypt;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
public class MySqlSslCaCertificateStrictEncryptSourceAcceptanceTest extends AbstractMySqlSslCertificateStrictEncryptSourceAcceptanceTest {
|
||||
public class MySqlSslCaCertificateStrictEncryptSourceAcceptanceTest extends MySqlStrictEncryptSourceAcceptanceTest {
|
||||
|
||||
private static final String PASSWORD = "Passw0rd";
|
||||
|
||||
@Override
|
||||
public ImmutableMap getSslConfig() {
|
||||
return ImmutableMap.builder()
|
||||
.put(JdbcUtils.MODE_KEY, "verify_ca")
|
||||
.put("ca_certificate", certs.getCaCertificate())
|
||||
.put("client_key_password", PASSWORD)
|
||||
protected Stream<String> extraContainerFactoryMethods() {
|
||||
return Stream.of("withRootAndServerCertificates");
|
||||
}
|
||||
|
||||
@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();
|
||||
}
|
||||
|
||||
|
||||
@@ -4,19 +4,31 @@
|
||||
|
||||
package io.airbyte.integrations.source.mysql_strict_encrypt;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
public class MySqlSslFullCertificateStrictEncryptSourceAcceptanceTest extends AbstractMySqlSslCertificateStrictEncryptSourceAcceptanceTest {
|
||||
public class MySqlSslFullCertificateStrictEncryptSourceAcceptanceTest extends MySqlStrictEncryptSourceAcceptanceTest {
|
||||
|
||||
private static final String PASSWORD = "Passw0rd";
|
||||
|
||||
@Override
|
||||
public ImmutableMap getSslConfig() {
|
||||
return ImmutableMap.builder()
|
||||
.put(JdbcUtils.MODE_KEY, "verify_ca")
|
||||
.put("ca_certificate", certs.getCaCertificate())
|
||||
.put("client_certificate", certs.getClientCertificate())
|
||||
.put("client_key", certs.getClientKey())
|
||||
.put("client_key_password", PASSWORD)
|
||||
protected Stream<String> extraContainerFactoryMethods() {
|
||||
return Stream.of("withRootAndServerCertificates", "withClientCertificate");
|
||||
}
|
||||
|
||||
@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();
|
||||
}
|
||||
|
||||
|
||||
@@ -4,22 +4,19 @@
|
||||
|
||||
package io.airbyte.integrations.source.mysql_strict_encrypt;
|
||||
|
||||
import static io.airbyte.integrations.source.mysql.MySqlSource.SSL_PARAMETERS;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
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.cdk.integrations.util.HostPortResolver;
|
||||
import io.airbyte.commons.features.EnvVariableFeatureFlags;
|
||||
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.MySQLContainerFactory;
|
||||
import io.airbyte.integrations.source.mysql.MySQLTestDatabase;
|
||||
import io.airbyte.protocol.models.Field;
|
||||
import io.airbyte.protocol.models.JsonSchemaType;
|
||||
import io.airbyte.protocol.models.v0.CatalogHelpers;
|
||||
@@ -29,72 +26,40 @@ import io.airbyte.protocol.models.v0.ConnectorSpecification;
|
||||
import io.airbyte.protocol.models.v0.DestinationSyncMode;
|
||||
import io.airbyte.protocol.models.v0.SyncMode;
|
||||
import java.util.HashMap;
|
||||
import org.jooq.DSLContext;
|
||||
import org.jooq.SQLDialect;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
import uk.org.webcompere.systemstubs.environment.EnvironmentVariables;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStub;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStubsExtension;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
@ExtendWith(SystemStubsExtension.class)
|
||||
public class MySqlStrictEncryptSourceAcceptanceTest extends SourceAcceptanceTest {
|
||||
|
||||
@SystemStub
|
||||
public EnvironmentVariables environmentVariables;
|
||||
private static final String STREAM_NAME = "id_and_name";
|
||||
private static final String STREAM_NAME2 = "public.starships";
|
||||
|
||||
protected MySQLContainer<?> container;
|
||||
protected JsonNode config;
|
||||
protected MySQLTestDatabase testdb;
|
||||
|
||||
@Override
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) throws Exception {
|
||||
environmentVariables.set(EnvVariableFeatureFlags.USE_STREAM_CAPABLE_STATE, "true");
|
||||
container = new MySQLContainer<>("mysql:8.0");
|
||||
container.start();
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) {
|
||||
final var container = new MySQLContainerFactory().shared("mysql:8.0", extraContainerFactoryMethods().toArray(String[]::new));
|
||||
testdb = new MySQLTestDatabase(container)
|
||||
.withConnectionProperty("useSSL", "true")
|
||||
.withConnectionProperty("requireSSL", "true")
|
||||
.initialized()
|
||||
.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');");
|
||||
}
|
||||
|
||||
var sslMode = ImmutableMap.builder()
|
||||
.put(JdbcUtils.MODE_KEY, "required")
|
||||
.build();
|
||||
final JsonNode replicationMethod = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("method", "STANDARD")
|
||||
.build());
|
||||
config = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, HostPortResolver.resolveHost(container))
|
||||
.put(JdbcUtils.PORT_KEY, HostPortResolver.resolvePort(container))
|
||||
.put(JdbcUtils.DATABASE_KEY, container.getDatabaseName())
|
||||
.put(JdbcUtils.USERNAME_KEY, container.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, container.getPassword())
|
||||
.put(JdbcUtils.SSL_MODE_KEY, sslMode)
|
||||
.put("replication_method", replicationMethod)
|
||||
.build());
|
||||
protected Stream<String> extraContainerFactoryMethods() {
|
||||
return Stream.empty();
|
||||
}
|
||||
|
||||
try (final DSLContext dslContext = DSLContextFactory.create(
|
||||
config.get(JdbcUtils.USERNAME_KEY).asText(),
|
||||
config.get(JdbcUtils.PASSWORD_KEY).asText(),
|
||||
DatabaseDriver.MYSQL.getDriverClassName(),
|
||||
String.format("jdbc:mysql://%s:%s/%s?%s",
|
||||
container.getHost(),
|
||||
container.getFirstMappedPort(),
|
||||
config.get(JdbcUtils.DATABASE_KEY).asText(),
|
||||
String.join("&", SSL_PARAMETERS)),
|
||||
SQLDialect.MYSQL)) {
|
||||
final Database database = new Database(dslContext);
|
||||
|
||||
database.query(ctx -> {
|
||||
ctx.fetch("CREATE TABLE id_and_name(id INTEGER, name VARCHAR(200));");
|
||||
ctx.fetch("INSERT INTO id_and_name (id, name) VALUES (1,'picard'), (2, 'crusher'), (3, 'vash');");
|
||||
ctx.fetch("CREATE TABLE starships(id INTEGER, name VARCHAR(200));");
|
||||
ctx.fetch("INSERT INTO starships (id, name) VALUES (1,'enterprise-d'), (2, 'defiant'), (3, 'yamato');");
|
||||
return null;
|
||||
});
|
||||
}
|
||||
@Override
|
||||
protected FeatureFlags featureFlags() {
|
||||
return FeatureFlagsWrapper.overridingUseStreamCapableState(super.featureFlags(), true);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void tearDown(final TestDestinationEnv testEnv) {
|
||||
container.close();
|
||||
testdb.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -109,7 +74,10 @@ public class MySqlStrictEncryptSourceAcceptanceTest extends SourceAcceptanceTest
|
||||
|
||||
@Override
|
||||
protected JsonNode getConfig() {
|
||||
return config;
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.withSsl(ImmutableMap.of(JdbcUtils.MODE_KEY, "required"))
|
||||
.withStandardReplication()
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -120,7 +88,7 @@ public class MySqlStrictEncryptSourceAcceptanceTest extends SourceAcceptanceTest
|
||||
.withCursorField(Lists.newArrayList("id"))
|
||||
.withDestinationSyncMode(DestinationSyncMode.APPEND)
|
||||
.withStream(CatalogHelpers.createAirbyteStream(
|
||||
String.format("%s.%s", config.get(JdbcUtils.DATABASE_KEY).asText(), STREAM_NAME),
|
||||
String.format("%s.%s", testdb.getDatabaseName(), STREAM_NAME),
|
||||
Field.of("id", JsonSchemaType.NUMBER),
|
||||
Field.of("name", JsonSchemaType.STRING))
|
||||
.withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL))),
|
||||
@@ -129,7 +97,7 @@ public class MySqlStrictEncryptSourceAcceptanceTest extends SourceAcceptanceTest
|
||||
.withCursorField(Lists.newArrayList("id"))
|
||||
.withDestinationSyncMode(DestinationSyncMode.APPEND)
|
||||
.withStream(CatalogHelpers.createAirbyteStream(
|
||||
String.format("%s.%s", config.get(JdbcUtils.DATABASE_KEY).asText(), STREAM_NAME2),
|
||||
String.format("%s.%s", testdb.getDatabaseName(), STREAM_NAME2),
|
||||
Field.of("id", JsonSchemaType.NUMBER),
|
||||
Field.of("name", JsonSchemaType.STRING))
|
||||
.withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL)))));
|
||||
|
||||
@@ -8,43 +8,33 @@ package io.airbyte.integrations.source.mysql_strict_encrypt;
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
import static io.airbyte.integrations.source.mysql.MySqlSource.SSL_PARAMETERS;
|
||||
import static io.airbyte.integrations.source.mysql.initialsync.MySqlInitialLoadStateManager.STATE_TYPE_KEY;
|
||||
import static java.util.stream.Collectors.toList;
|
||||
import static org.assertj.core.api.Assertions.assertThat;
|
||||
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.node.ObjectNode;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.MySqlUtils;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.base.Source;
|
||||
import io.airbyte.cdk.integrations.base.ssh.SshBastionContainer;
|
||||
import io.airbyte.cdk.integrations.base.ssh.SshHelpers;
|
||||
import io.airbyte.cdk.integrations.base.ssh.SshTunnel;
|
||||
import io.airbyte.cdk.integrations.source.jdbc.test.JdbcSourceAcceptanceTest;
|
||||
import io.airbyte.cdk.integrations.source.relationaldb.models.DbStreamState;
|
||||
import io.airbyte.commons.features.EnvVariableFeatureFlags;
|
||||
import io.airbyte.commons.features.FeatureFlagsWrapper;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.commons.resources.MoreResources;
|
||||
import io.airbyte.commons.string.Strings;
|
||||
import io.airbyte.commons.util.MoreIterators;
|
||||
import io.airbyte.integrations.source.mysql.MySQLContainerFactory;
|
||||
import io.airbyte.integrations.source.mysql.MySQLTestDatabase;
|
||||
import io.airbyte.integrations.source.mysql.MySqlSource;
|
||||
import io.airbyte.integrations.source.mysql.internal.models.CursorBasedStatus;
|
||||
import io.airbyte.integrations.source.mysql.internal.models.InternalModels.StateType;
|
||||
import io.airbyte.protocol.models.Field;
|
||||
import io.airbyte.protocol.models.JsonSchemaType;
|
||||
import io.airbyte.protocol.models.v0.AirbyteCatalog;
|
||||
import io.airbyte.protocol.models.v0.AirbyteConnectionStatus;
|
||||
import io.airbyte.protocol.models.v0.AirbyteConnectionStatus.Status;
|
||||
import io.airbyte.protocol.models.v0.AirbyteMessage;
|
||||
import io.airbyte.protocol.models.v0.AirbyteMessage.Type;
|
||||
import io.airbyte.protocol.models.v0.AirbyteRecordMessage;
|
||||
@@ -59,97 +49,41 @@ import io.airbyte.protocol.models.v0.ConnectorSpecification;
|
||||
import io.airbyte.protocol.models.v0.DestinationSyncMode;
|
||||
import io.airbyte.protocol.models.v0.StreamDescriptor;
|
||||
import io.airbyte.protocol.models.v0.SyncMode;
|
||||
import java.sql.Connection;
|
||||
import java.sql.DriverManager;
|
||||
import java.sql.SQLException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.jooq.DSLContext;
|
||||
import org.jooq.SQLDialect;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
import org.testcontainers.containers.Network;
|
||||
import uk.org.webcompere.systemstubs.environment.EnvironmentVariables;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStub;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStubsExtension;
|
||||
|
||||
@ExtendWith(SystemStubsExtension.class)
|
||||
class MySqlStrictEncryptJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest {
|
||||
class MySqlStrictEncryptJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest<MySqlStrictEncryptSource, MySQLTestDatabase> {
|
||||
|
||||
@SystemStub
|
||||
private EnvironmentVariables environmentVariables;
|
||||
|
||||
protected static final String TEST_USER = "test";
|
||||
protected static final String TEST_PASSWORD = "test";
|
||||
protected static MySQLContainer<?> container;
|
||||
private static final SshBastionContainer bastion = new SshBastionContainer();
|
||||
private static final Network network = Network.newNetwork();
|
||||
|
||||
protected Database database;
|
||||
protected DSLContext dslContext;
|
||||
|
||||
@BeforeAll
|
||||
static void init() throws SQLException {
|
||||
container = new MySQLContainer<>("mysql:8.0")
|
||||
.withUsername(TEST_USER)
|
||||
.withPassword(TEST_PASSWORD)
|
||||
.withEnv("MYSQL_ROOT_HOST", "%")
|
||||
.withEnv("MYSQL_ROOT_PASSWORD", TEST_PASSWORD);
|
||||
container.start();
|
||||
final Connection connection = DriverManager.getConnection(container.getJdbcUrl(), "root", container.getPassword());
|
||||
connection.createStatement().execute("GRANT ALL PRIVILEGES ON *.* TO '" + TEST_USER + "'@'%';\n");
|
||||
@Override
|
||||
protected JsonNode config() {
|
||||
return testdb.testConfigBuilder().build();
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
public void setup() throws Exception {
|
||||
environmentVariables.set(EnvVariableFeatureFlags.USE_STREAM_CAPABLE_STATE, "true");
|
||||
config = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, container.getHost())
|
||||
.put(JdbcUtils.PORT_KEY, container.getFirstMappedPort())
|
||||
.put(JdbcUtils.DATABASE_KEY, Strings.addRandomSuffix("db", "_", 10))
|
||||
.put(JdbcUtils.USERNAME_KEY, container.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, container.getPassword())
|
||||
.build());
|
||||
|
||||
dslContext = DSLContextFactory.create(
|
||||
config.get(JdbcUtils.USERNAME_KEY).asText(),
|
||||
config.get(JdbcUtils.PASSWORD_KEY).asText(),
|
||||
DatabaseDriver.MYSQL.getDriverClassName(),
|
||||
String.format("jdbc:mysql://%s:%s?%s",
|
||||
container.getHost(),
|
||||
container.getFirstMappedPort(),
|
||||
String.join("&", SSL_PARAMETERS)),
|
||||
SQLDialect.MYSQL);
|
||||
database = new Database(dslContext);
|
||||
|
||||
database.query(ctx -> {
|
||||
ctx.fetch("CREATE DATABASE " + config.get(JdbcUtils.DATABASE_KEY).asText());
|
||||
return null;
|
||||
});
|
||||
|
||||
super.setup();
|
||||
@Override
|
||||
protected MySqlStrictEncryptSource source() {
|
||||
final var source = new MySqlSource();
|
||||
source.setFeatureFlags(FeatureFlagsWrapper.overridingUseStreamCapableState(new EnvVariableFeatureFlags(), true));
|
||||
return new MySqlStrictEncryptSource(source);
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
void tearDownMySql() throws Exception {
|
||||
dslContext.close();
|
||||
super.tearDown();
|
||||
@Override
|
||||
protected MySQLTestDatabase createTestDatabase() {
|
||||
final var container = new MySQLContainerFactory().shared("mysql:8.0");
|
||||
return new MySQLTestDatabase(container)
|
||||
.withConnectionProperty("useSSL", "true")
|
||||
.withConnectionProperty("requireSSL", "true")
|
||||
.initialized();
|
||||
}
|
||||
|
||||
@AfterAll
|
||||
static void cleanUp() {
|
||||
container.close();
|
||||
@Override
|
||||
protected void maybeSetShorterConnectionTimeout(final JsonNode config) {
|
||||
((ObjectNode) config).put(JdbcUtils.JDBC_URL_PARAMS_KEY, "connectTimeout=1000");
|
||||
}
|
||||
|
||||
// MySql does not support schemas in the way most dbs do. Instead we namespace by db name.
|
||||
@@ -158,29 +92,9 @@ class MySqlStrictEncryptJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTes
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public MySqlSource getJdbcSource() {
|
||||
return new MySqlSource();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source getSource() {
|
||||
return new MySqlStrictEncryptSource();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDriverClass() {
|
||||
return MySqlSource.DRIVER_CLASS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public JsonNode getConfig() {
|
||||
return Jsons.clone(config);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSpec() throws Exception {
|
||||
final ConnectorSpecification actual = source.spec();
|
||||
final ConnectorSpecification actual = source().spec();
|
||||
final ConnectorSpecification expected =
|
||||
SshHelpers.injectSshIntoSpec(Jsons.deserialize(MoreResources.readResource("expected_spec.json"), ConnectorSpecification.class));
|
||||
assertEquals(expected, actual);
|
||||
@@ -216,176 +130,42 @@ class MySqlStrictEncryptJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTes
|
||||
List.of(List.of(COL_FIRST_NAME), List.of(COL_LAST_NAME)))));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testStrictSSLUnsecuredNoTunnel() throws Exception {
|
||||
final String PASSWORD = "Passw0rd";
|
||||
final var certs = MySqlUtils.getCertificate(container, true);
|
||||
final var sslMode = ImmutableMap.builder()
|
||||
.put(JdbcUtils.MODE_KEY, "preferred")
|
||||
.build();
|
||||
|
||||
final var tunnelMode = ImmutableMap.builder()
|
||||
.put("tunnel_method", "NO_TUNNEL")
|
||||
.build();
|
||||
((ObjectNode) config).put(JdbcUtils.PASSWORD_KEY, "fake")
|
||||
.put(JdbcUtils.SSL_KEY, true)
|
||||
.putIfAbsent(JdbcUtils.SSL_MODE_KEY, Jsons.jsonNode(sslMode));
|
||||
((ObjectNode) config).putIfAbsent("tunnel_method", Jsons.jsonNode(tunnelMode));
|
||||
|
||||
final AirbyteConnectionStatus actual = source.check(config);
|
||||
assertEquals(Status.FAILED, actual.getStatus());
|
||||
assertTrue(actual.getMessage().contains("Unsecured connection not allowed"));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testStrictSSLSecuredNoTunnel() throws Exception {
|
||||
final String PASSWORD = "Passw0rd";
|
||||
final var certs = MySqlUtils.getCertificate(container, true);
|
||||
final var sslMode = ImmutableMap.builder()
|
||||
.put(JdbcUtils.MODE_KEY, "verify_ca")
|
||||
.put("ca_certificate", certs.getCaCertificate())
|
||||
.put("client_certificate", certs.getClientCertificate())
|
||||
.put("client_key", certs.getClientKey())
|
||||
.put("client_key_password", PASSWORD)
|
||||
.build();
|
||||
|
||||
final var tunnelMode = ImmutableMap.builder()
|
||||
.put("tunnel_method", "NO_TUNNEL")
|
||||
.build();
|
||||
((ObjectNode) config).put(JdbcUtils.PASSWORD_KEY, "fake")
|
||||
.put(JdbcUtils.SSL_KEY, true)
|
||||
.putIfAbsent(JdbcUtils.SSL_MODE_KEY, Jsons.jsonNode(sslMode));
|
||||
((ObjectNode) config).putIfAbsent("tunnel_method", Jsons.jsonNode(tunnelMode));
|
||||
|
||||
final AirbyteConnectionStatus actual = source.check(config);
|
||||
assertEquals(Status.FAILED, actual.getStatus());
|
||||
assertFalse(actual.getMessage().contains("Unsecured connection not allowed"));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testStrictSSLSecuredWithTunnel() throws Exception {
|
||||
final String PASSWORD = "Passw0rd";
|
||||
final var certs = MySqlUtils.getCertificate(container, true);
|
||||
final var sslMode = ImmutableMap.builder()
|
||||
.put(JdbcUtils.MODE_KEY, "verify_ca")
|
||||
.put("ca_certificate", certs.getCaCertificate())
|
||||
.put("client_certificate", certs.getClientCertificate())
|
||||
.put("client_key", certs.getClientKey())
|
||||
.put("client_key_password", PASSWORD)
|
||||
.build();
|
||||
|
||||
final var tunnelMode = ImmutableMap.builder()
|
||||
.put("tunnel_method", "SSH_KEY_AUTH")
|
||||
.build();
|
||||
((ObjectNode) config).put(JdbcUtils.PASSWORD_KEY, "fake")
|
||||
.put(JdbcUtils.SSL_KEY, true)
|
||||
.putIfAbsent(JdbcUtils.SSL_MODE_KEY, Jsons.jsonNode(sslMode));
|
||||
((ObjectNode) config).putIfAbsent("tunnel_method", Jsons.jsonNode(tunnelMode));
|
||||
|
||||
final AirbyteConnectionStatus actual = source.check(config);
|
||||
assertEquals(Status.FAILED, actual.getStatus());
|
||||
assertTrue(actual.getMessage().contains("Could not connect with provided SSH configuration."));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testStrictSSLUnsecuredWithTunnel() throws Exception {
|
||||
final String PASSWORD = "Passw0rd";
|
||||
final var certs = MySqlUtils.getCertificate(container, true);
|
||||
final var sslMode = ImmutableMap.builder()
|
||||
.put(JdbcUtils.MODE_KEY, "preferred")
|
||||
.build();
|
||||
|
||||
final var tunnelMode = ImmutableMap.builder()
|
||||
.put("tunnel_method", "SSH_KEY_AUTH")
|
||||
.build();
|
||||
((ObjectNode) config).put(JdbcUtils.PASSWORD_KEY, "fake")
|
||||
.put(JdbcUtils.SSL_KEY, true)
|
||||
.putIfAbsent(JdbcUtils.SSL_MODE_KEY, Jsons.jsonNode(sslMode));
|
||||
((ObjectNode) config).putIfAbsent("tunnel_method", Jsons.jsonNode(tunnelMode));
|
||||
|
||||
final AirbyteConnectionStatus actual = source.check(config);
|
||||
assertEquals(Status.FAILED, actual.getStatus());
|
||||
assertTrue(actual.getMessage().contains("Could not connect with provided SSH configuration."));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testCheckWithSSlModeDisabled() throws Exception {
|
||||
try (final MySQLContainer<?> db = new MySQLContainer<>("mysql:8.0").withNetwork(network)) {
|
||||
bastion.initAndStartBastion(network);
|
||||
db.start();
|
||||
final JsonNode configWithSSLModeDisabled = bastion.getTunnelConfig(SshTunnel.TunnelMethod.SSH_PASSWORD_AUTH, ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, Objects.requireNonNull(db.getContainerInfo()
|
||||
.getNetworkSettings()
|
||||
.getNetworks()
|
||||
.entrySet().stream()
|
||||
.findFirst()
|
||||
.get().getValue().getIpAddress()))
|
||||
.put(JdbcUtils.PORT_KEY, db.getExposedPorts().get(0))
|
||||
.put(JdbcUtils.DATABASE_KEY, db.getDatabaseName())
|
||||
.put(JdbcUtils.SCHEMAS_KEY, List.of("public"))
|
||||
.put(JdbcUtils.USERNAME_KEY, db.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, db.getPassword())
|
||||
.put(JdbcUtils.SSL_MODE_KEY, Map.of(JdbcUtils.MODE_KEY, "disable")), false);
|
||||
|
||||
final AirbyteConnectionStatus actual = source.check(configWithSSLModeDisabled);
|
||||
assertEquals(AirbyteConnectionStatus.Status.SUCCEEDED, actual.getStatus());
|
||||
} finally {
|
||||
bastion.stopAndClose();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testReadMultipleTablesIncrementally() throws Exception {
|
||||
final var config = config();
|
||||
((ObjectNode) config).put("sync_checkpoint_records", 1);
|
||||
final String namespace = getDefaultNamespace();
|
||||
final String streamOneName = TABLE_NAME + "one";
|
||||
// Create a fresh first table
|
||||
database.query(connection -> {
|
||||
connection.fetch(String.format("USE %s;", getDefaultNamespace()));
|
||||
connection.fetch(String.format("CREATE TABLE %s (\n"
|
||||
+ " id int PRIMARY KEY,\n"
|
||||
+ " name VARCHAR(200) NOT NULL,\n"
|
||||
+ " updated_at VARCHAR(200) NOT NULL\n"
|
||||
+ ");", streamOneName));
|
||||
connection.execute(
|
||||
String.format(
|
||||
"INSERT INTO %s(id, name, updated_at) VALUES (1,'picard', '2004-10-19')",
|
||||
getFullyQualifiedTableName(streamOneName)));
|
||||
connection.execute(
|
||||
String.format(
|
||||
"INSERT INTO %s(id, name, updated_at) VALUES (2, 'crusher', '2005-10-19')",
|
||||
getFullyQualifiedTableName(streamOneName)));
|
||||
connection.execute(
|
||||
String.format(
|
||||
"INSERT INTO %s(id, name, updated_at) VALUES (3, 'vash', '2006-10-19')",
|
||||
getFullyQualifiedTableName(streamOneName)));
|
||||
return null;
|
||||
});
|
||||
testdb.with("""
|
||||
CREATE TABLE %s (
|
||||
id int PRIMARY KEY,
|
||||
name VARCHAR(200) NOT NULL,
|
||||
updated_at VARCHAR(200) NOT NULL
|
||||
);""", streamOneName)
|
||||
.with("INSERT INTO %s(id, name, updated_at) VALUES (1,'picard', '2004-10-19')",
|
||||
getFullyQualifiedTableName(streamOneName))
|
||||
.with("INSERT INTO %s(id, name, updated_at) VALUES (2, 'crusher', '2005-10-19')",
|
||||
getFullyQualifiedTableName(streamOneName))
|
||||
.with("INSERT INTO %s(id, name, updated_at) VALUES (3, 'vash', '2006-10-19')",
|
||||
getFullyQualifiedTableName(streamOneName));
|
||||
|
||||
// Create a fresh second table
|
||||
final String streamTwoName = TABLE_NAME + "two";
|
||||
final String streamTwoFullyQualifiedName = getFullyQualifiedTableName(streamTwoName);
|
||||
// Insert records into second table
|
||||
database.query(ctx -> {
|
||||
ctx.fetch(String.format("CREATE TABLE %s (\n"
|
||||
+ " id int PRIMARY KEY,\n"
|
||||
+ " name VARCHAR(200) NOT NULL,\n"
|
||||
+ " updated_at DATE NOT NULL\n"
|
||||
+ ");", streamTwoName));
|
||||
ctx.execute(
|
||||
String.format("INSERT INTO %s(id, name, updated_at)"
|
||||
+ "VALUES (40,'Jean Luc','2006-10-19')",
|
||||
streamTwoFullyQualifiedName));
|
||||
ctx.execute(
|
||||
String.format("INSERT INTO %s(id, name, updated_at)"
|
||||
+ "VALUES (41, 'Groot', '2006-10-19')",
|
||||
streamTwoFullyQualifiedName));
|
||||
ctx.execute(
|
||||
String.format("INSERT INTO %s(id, name, updated_at)"
|
||||
+ "VALUES (42, 'Thanos','2006-10-19')",
|
||||
streamTwoFullyQualifiedName));
|
||||
return null;
|
||||
});
|
||||
testdb.with("""
|
||||
CREATE TABLE %s (
|
||||
id int PRIMARY KEY,
|
||||
name VARCHAR(200) NOT NULL,
|
||||
updated_at DATE NOT NULL
|
||||
);""", streamTwoName)
|
||||
.with("INSERT INTO %s(id, name, updated_at) VALUES (40,'Jean Luc','2006-10-19')",
|
||||
streamTwoFullyQualifiedName)
|
||||
.with("INSERT INTO %s(id, name, updated_at) VALUES (41, 'Groot', '2006-10-19')",
|
||||
streamTwoFullyQualifiedName)
|
||||
.with("INSERT INTO %s(id, name, updated_at) VALUES (42, 'Thanos','2006-10-19')",
|
||||
streamTwoFullyQualifiedName);
|
||||
// Create records list that we expect to see in the state message
|
||||
final List<AirbyteMessage> streamTwoExpectedRecords = Arrays.asList(
|
||||
createRecord(streamTwoName, namespace, ImmutableMap.of(
|
||||
@@ -416,7 +196,7 @@ class MySqlStrictEncryptJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTes
|
||||
|
||||
// Perform initial sync
|
||||
final List<AirbyteMessage> messagesFromFirstSync = MoreIterators
|
||||
.toList(source.read(config, configuredCatalog, null));
|
||||
.toList(source().read(config, configuredCatalog, null));
|
||||
|
||||
final List<AirbyteMessage> recordsFromFirstSync = filterRecords(messagesFromFirstSync);
|
||||
|
||||
@@ -483,7 +263,7 @@ class MySqlStrictEncryptJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTes
|
||||
// - stream two state being the Primary Key state before the final emitted state before the cursor
|
||||
// switch
|
||||
final List<AirbyteMessage> messagesFromSecondSyncWithMixedStates = MoreIterators
|
||||
.toList(source.read(config, configuredCatalog,
|
||||
.toList(source().read(config, configuredCatalog,
|
||||
Jsons.jsonNode(List.of(streamOneStateMessagesFromFirstSync.get(0),
|
||||
streamTwoStateMessagesFromFirstSync.get(1)))));
|
||||
|
||||
@@ -510,21 +290,13 @@ class MySqlStrictEncryptJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTes
|
||||
|
||||
// Add some data to each table and perform a third read.
|
||||
// Expect to see all records be synced via cursorBased method and not primaryKey
|
||||
|
||||
database.query(ctx -> {
|
||||
ctx.execute(
|
||||
String.format("INSERT INTO %s(id, name, updated_at)"
|
||||
+ "VALUES (4,'Hooper','2006-10-19')",
|
||||
getFullyQualifiedTableName(streamOneName)));
|
||||
ctx.execute(
|
||||
String.format("INSERT INTO %s(id, name, updated_at)"
|
||||
+ "VALUES (43, 'Iron Man', '2006-10-19')",
|
||||
streamTwoFullyQualifiedName));
|
||||
return null;
|
||||
});
|
||||
testdb.with("INSERT INTO %s(id, name, updated_at) VALUES (4,'Hooper','2006-10-19')",
|
||||
getFullyQualifiedTableName(streamOneName))
|
||||
.with("INSERT INTO %s(id, name, updated_at) VALUES (43, 'Iron Man', '2006-10-19')",
|
||||
streamTwoFullyQualifiedName);
|
||||
|
||||
final List<AirbyteMessage> messagesFromThirdSync = MoreIterators
|
||||
.toList(source.read(config, configuredCatalog,
|
||||
.toList(source().read(config, configuredCatalog,
|
||||
Jsons.jsonNode(List.of(streamOneStateMessagesFromSecondSync.get(1),
|
||||
streamTwoStateMessagesFromSecondSync.get(0)))));
|
||||
|
||||
@@ -615,13 +387,13 @@ class MySqlStrictEncryptJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTes
|
||||
protected List<AirbyteMessage> getExpectedAirbyteMessagesSecondSync(final String namespace) {
|
||||
final List<AirbyteMessage> expectedMessages = new ArrayList<>();
|
||||
expectedMessages.add(new AirbyteMessage().withType(AirbyteMessage.Type.RECORD)
|
||||
.withRecord(new AirbyteRecordMessage().withStream(streamName).withNamespace(namespace)
|
||||
.withRecord(new AirbyteRecordMessage().withStream(streamName()).withNamespace(namespace)
|
||||
.withData(Jsons.jsonNode(ImmutableMap
|
||||
.of(COL_ID, ID_VALUE_4,
|
||||
COL_NAME, "riker",
|
||||
COL_UPDATED_AT, "2006-10-19")))));
|
||||
expectedMessages.add(new AirbyteMessage().withType(AirbyteMessage.Type.RECORD)
|
||||
.withRecord(new AirbyteRecordMessage().withStream(streamName).withNamespace(namespace)
|
||||
.withRecord(new AirbyteRecordMessage().withStream(streamName()).withNamespace(namespace)
|
||||
.withData(Jsons.jsonNode(ImmutableMap
|
||||
.of(COL_ID, ID_VALUE_5,
|
||||
COL_NAME, "data",
|
||||
@@ -629,7 +401,7 @@ class MySqlStrictEncryptJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTes
|
||||
final DbStreamState state = new CursorBasedStatus()
|
||||
.withStateType(StateType.CURSOR_BASED)
|
||||
.withVersion(2L)
|
||||
.withStreamName(streamName)
|
||||
.withStreamName(streamName())
|
||||
.withStreamNamespace(namespace)
|
||||
.withCursorField(ImmutableList.of(COL_ID))
|
||||
.withCursor("5")
|
||||
@@ -641,7 +413,7 @@ class MySqlStrictEncryptJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTes
|
||||
|
||||
@Override
|
||||
protected List<AirbyteMessage> getTestMessages() {
|
||||
return getTestMessages(streamName);
|
||||
return getTestMessages(streamName());
|
||||
}
|
||||
|
||||
protected List<AirbyteMessage> getTestMessages(final String streamName) {
|
||||
|
||||
@@ -0,0 +1,129 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.source.mysql_strict_encrypt;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.base.ssh.SshBastionContainer;
|
||||
import io.airbyte.cdk.integrations.base.ssh.SshTunnel;
|
||||
import io.airbyte.integrations.source.mysql.MySQLContainerFactory;
|
||||
import io.airbyte.integrations.source.mysql.MySQLTestDatabase;
|
||||
import io.airbyte.protocol.models.v0.AirbyteConnectionStatus;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.parallel.Execution;
|
||||
import org.junit.jupiter.api.parallel.ExecutionMode;
|
||||
|
||||
@Execution(ExecutionMode.CONCURRENT)
|
||||
public class MySqlStrictEncryptSslTest {
|
||||
|
||||
private MySQLTestDatabase createTestDatabase(String... containerFactoryMethods) {
|
||||
final var container = new MySQLContainerFactory().shared("mysql:8.0", containerFactoryMethods);
|
||||
return new MySQLTestDatabase(container)
|
||||
.withConnectionProperty("useSSL", "true")
|
||||
.withConnectionProperty("requireSSL", "true")
|
||||
.initialized();
|
||||
}
|
||||
|
||||
@Test
|
||||
void testStrictSSLUnsecuredNoTunnel() throws Exception {
|
||||
try (final var testdb = createTestDatabase()) {
|
||||
final var config = testdb.configBuilder()
|
||||
.withHostAndPort()
|
||||
.withDatabase()
|
||||
.with(JdbcUtils.USERNAME_KEY, testdb.getUserName())
|
||||
.with(JdbcUtils.PASSWORD_KEY, "fake")
|
||||
.with("tunnel_method", ImmutableMap.builder().put("tunnel_method", "NO_TUNNEL").build())
|
||||
.withSsl(ImmutableMap.builder()
|
||||
.put(JdbcUtils.MODE_KEY, "preferred")
|
||||
.build())
|
||||
.build();
|
||||
final AirbyteConnectionStatus actual = new MySqlStrictEncryptSource().check(config);
|
||||
assertEquals(AirbyteConnectionStatus.Status.FAILED, actual.getStatus());
|
||||
assertTrue(actual.getMessage().contains("Unsecured connection not allowed"), actual.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testStrictSSLSecuredNoTunnel() throws Exception {
|
||||
final String PASSWORD = "Passw0rd";
|
||||
try (final var testdb = createTestDatabase("withRootAndServerCertificates", "withClientCertificate")) {
|
||||
final var config = testdb.testConfigBuilder()
|
||||
.with("tunnel_method", ImmutableMap.builder().put("tunnel_method", "NO_TUNNEL").build())
|
||||
.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();
|
||||
final AirbyteConnectionStatus actual = new MySqlStrictEncryptSource().check(config);
|
||||
assertEquals(AirbyteConnectionStatus.Status.FAILED, actual.getStatus());
|
||||
assertTrue(actual.getMessage().contains("Failed to create keystore for Client certificate"), actual.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testStrictSSLSecuredWithTunnel() throws Exception {
|
||||
final String PASSWORD = "Passw0rd";
|
||||
try (final var testdb = createTestDatabase("withRootAndServerCertificates", "withClientCertificate")) {
|
||||
final var config = testdb.configBuilder()
|
||||
.withHostAndPort()
|
||||
.withDatabase()
|
||||
.with(JdbcUtils.USERNAME_KEY, testdb.getUserName())
|
||||
.with(JdbcUtils.PASSWORD_KEY, "fake")
|
||||
.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())
|
||||
.with("tunnel_method", ImmutableMap.builder().put("tunnel_method", "SSH_KEY_AUTH").build())
|
||||
.build();
|
||||
final AirbyteConnectionStatus actual = new MySqlStrictEncryptSource().check(config);
|
||||
assertEquals(AirbyteConnectionStatus.Status.FAILED, actual.getStatus());
|
||||
assertTrue(actual.getMessage().contains("Could not connect with provided SSH configuration."), actual.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testStrictSSLUnsecuredWithTunnel() throws Exception {
|
||||
try (final var testdb = createTestDatabase()) {
|
||||
final var config = testdb.configBuilder()
|
||||
.withHostAndPort()
|
||||
.withDatabase()
|
||||
.with(JdbcUtils.USERNAME_KEY, testdb.getUserName())
|
||||
.with(JdbcUtils.PASSWORD_KEY, "fake")
|
||||
.withSsl(ImmutableMap.builder()
|
||||
.put(JdbcUtils.MODE_KEY, "preferred")
|
||||
.build())
|
||||
.with("tunnel_method", ImmutableMap.builder().put("tunnel_method", "SSH_KEY_AUTH").build())
|
||||
.build();
|
||||
final AirbyteConnectionStatus actual = new MySqlStrictEncryptSource().check(config);
|
||||
assertEquals(AirbyteConnectionStatus.Status.FAILED, actual.getStatus());
|
||||
assertTrue(actual.getMessage().contains("Could not connect with provided SSH configuration."), actual.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testCheckWithSslModeDisabled() throws Exception {
|
||||
try (final var testdb = createTestDatabase("withNetwork")) {
|
||||
try (final SshBastionContainer bastion = new SshBastionContainer()) {
|
||||
bastion.initAndStartBastion(testdb.getContainer().getNetwork());
|
||||
final var config = testdb.integrationTestConfigBuilder()
|
||||
.with("tunnel_method", bastion.getTunnelMethod(SshTunnel.TunnelMethod.SSH_PASSWORD_AUTH, false))
|
||||
.withoutSsl()
|
||||
.build();
|
||||
final AirbyteConnectionStatus actual = new MySqlStrictEncryptSource().check(config);
|
||||
assertEquals(AirbyteConnectionStatus.Status.SUCCEEDED, actual.getStatus());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -7,7 +7,7 @@ plugins {
|
||||
}
|
||||
|
||||
airbyteJavaConnector {
|
||||
cdkVersionRequired = '0.4.1'
|
||||
cdkVersionRequired = '0.5.0'
|
||||
features = ['db-sources']
|
||||
useLocalCdk = false
|
||||
}
|
||||
@@ -18,7 +18,7 @@ configurations.all {
|
||||
}
|
||||
}
|
||||
|
||||
airbyteJavaConnector.addCdkDependencies()
|
||||
|
||||
|
||||
application {
|
||||
mainClass = 'io.airbyte.integrations.source.mysql.MySqlSource'
|
||||
@@ -35,7 +35,7 @@ dependencies {
|
||||
testImplementation 'org.hamcrest:hamcrest-all:1.3'
|
||||
testImplementation libs.junit.jupiter.system.stubs
|
||||
testImplementation libs.testcontainers.mysql
|
||||
|
||||
testFixturesImplementation libs.testcontainers.mysql
|
||||
performanceTestJavaImplementation project(':airbyte-integrations:connectors:source-mysql')
|
||||
|
||||
}
|
||||
|
||||
@@ -0,0 +1 @@
|
||||
testExecutionConcurrency=-1
|
||||
@@ -9,7 +9,7 @@ data:
|
||||
connectorSubtype: database
|
||||
connectorType: source
|
||||
definitionId: 435bb9a5-7887-4809-aa58-28c27df0d7ad
|
||||
dockerImageTag: 3.1.7
|
||||
dockerImageTag: 3.1.8
|
||||
dockerRepository: airbyte/source-mysql
|
||||
documentationUrl: https://docs.airbyte.com/integrations/sources/mysql
|
||||
githubIssueLabel: source-mysql
|
||||
|
||||
@@ -26,7 +26,10 @@ import org.slf4j.LoggerFactory;
|
||||
public class MySqlCdcProperties {
|
||||
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(MySqlCdcProperties.class);
|
||||
private static final Duration HEARTBEAT_FREQUENCY = Duration.ofSeconds(10);
|
||||
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();
|
||||
@@ -61,7 +64,12 @@ public class MySqlCdcProperties {
|
||||
props.setProperty("converters", "boolean, datetime");
|
||||
props.setProperty("boolean.type", CustomMySQLTinyIntOneToBooleanConverter.class.getName());
|
||||
props.setProperty("datetime.type", MySQLDateTimeConverter.class.getName());
|
||||
props.setProperty("heartbeat.interval.ms", Long.toString(HEARTBEAT_FREQUENCY.toMillis()));
|
||||
|
||||
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
|
||||
|
||||
@@ -34,7 +34,7 @@ import io.airbyte.cdk.db.jdbc.StreamingJdbcDatabase;
|
||||
import io.airbyte.cdk.integrations.base.IntegrationRunner;
|
||||
import io.airbyte.cdk.integrations.base.Source;
|
||||
import io.airbyte.cdk.integrations.base.ssh.SshWrappedSource;
|
||||
import io.airbyte.cdk.integrations.debezium.internals.FirstRecordWaitTimeUtil;
|
||||
import io.airbyte.cdk.integrations.debezium.internals.RecordWaitTimeUtil;
|
||||
import io.airbyte.cdk.integrations.source.jdbc.AbstractJdbcSource;
|
||||
import io.airbyte.cdk.integrations.source.jdbc.JdbcDataSourceUtils;
|
||||
import io.airbyte.cdk.integrations.source.jdbc.JdbcSSLConnectionUtils;
|
||||
@@ -46,8 +46,6 @@ import io.airbyte.cdk.integrations.source.relationaldb.state.StateManager;
|
||||
import io.airbyte.cdk.integrations.source.relationaldb.state.StateManagerFactory;
|
||||
import io.airbyte.cdk.integrations.util.HostPortResolver;
|
||||
import io.airbyte.commons.exceptions.ConfigErrorException;
|
||||
import io.airbyte.commons.features.EnvVariableFeatureFlags;
|
||||
import io.airbyte.commons.features.FeatureFlags;
|
||||
import io.airbyte.commons.functional.CheckedConsumer;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.commons.map.MoreMaps;
|
||||
@@ -116,15 +114,12 @@ public class MySqlSource extends AbstractJdbcSource<MysqlType> implements Source
|
||||
"useSSL=true",
|
||||
"requireSSL=true");
|
||||
|
||||
private final FeatureFlags featureFlags;
|
||||
|
||||
public static Source sshWrappedSource() {
|
||||
return new SshWrappedSource(new MySqlSource(), JdbcUtils.HOST_LIST_KEY, JdbcUtils.PORT_LIST_KEY);
|
||||
public static Source sshWrappedSource(MySqlSource source) {
|
||||
return new SshWrappedSource(source, JdbcUtils.HOST_LIST_KEY, JdbcUtils.PORT_LIST_KEY);
|
||||
}
|
||||
|
||||
public MySqlSource() {
|
||||
super(DRIVER_CLASS, MySqlStreamingQueryConfig::new, new MySqlSourceOperations());
|
||||
this.featureFlags = new EnvVariableFeatureFlags();
|
||||
}
|
||||
|
||||
private static AirbyteStream overrideSyncModes(final AirbyteStream stream) {
|
||||
@@ -182,7 +177,7 @@ public class MySqlSource extends AbstractJdbcSource<MysqlType> implements Source
|
||||
checkOperations.addAll(CdcConfigurationHelper.getCheckOperations());
|
||||
|
||||
checkOperations.add(database -> {
|
||||
FirstRecordWaitTimeUtil.checkFirstRecordWaitTime(config);
|
||||
RecordWaitTimeUtil.checkFirstRecordWaitTime(config);
|
||||
CdcConfigurationHelper.checkServerTimeZoneConfig(config);
|
||||
});
|
||||
}
|
||||
@@ -530,7 +525,7 @@ public class MySqlSource extends AbstractJdbcSource<MysqlType> implements Source
|
||||
}
|
||||
|
||||
public static void main(final String[] args) throws Exception {
|
||||
final Source source = MySqlSource.sshWrappedSource();
|
||||
final Source source = MySqlSource.sshWrappedSource(new MySqlSource());
|
||||
LOGGER.info("starting source: {}", MySqlSource.class);
|
||||
new IntegrationRunner(source).run(args);
|
||||
LOGGER.info("completed source: {}", MySqlSource.class);
|
||||
|
||||
@@ -17,7 +17,7 @@ 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.DebeziumPropertiesManager;
|
||||
import io.airbyte.cdk.integrations.debezium.internals.FirstRecordWaitTimeUtil;
|
||||
import io.airbyte.cdk.integrations.debezium.internals.RecordWaitTimeUtil;
|
||||
import io.airbyte.cdk.integrations.debezium.internals.mysql.MySqlCdcPosition;
|
||||
import io.airbyte.cdk.integrations.debezium.internals.mysql.MySqlCdcTargetPosition;
|
||||
import io.airbyte.cdk.integrations.debezium.internals.mysql.MySqlDebeziumStateUtil;
|
||||
@@ -84,7 +84,8 @@ public class MySqlInitialReadUtil {
|
||||
final Instant emittedAt,
|
||||
final String quoteString) {
|
||||
final JsonNode sourceConfig = database.getSourceConfig();
|
||||
final Duration firstRecordWaitTime = FirstRecordWaitTimeUtil.getFirstRecordWaitTime(sourceConfig);
|
||||
final Duration firstRecordWaitTime = RecordWaitTimeUtil.getFirstRecordWaitTime(sourceConfig);
|
||||
final Duration subsequentRecordWaitTime = RecordWaitTimeUtil.getSubsequentRecordWaitTime(sourceConfig);
|
||||
LOGGER.info("First record waiting time: {} seconds", firstRecordWaitTime.getSeconds());
|
||||
// Determine the streams that need to be loaded via primary key sync.
|
||||
final List<AutoCloseableIterator<AirbyteMessage>> initialLoadIterator = new ArrayList<>();
|
||||
@@ -149,8 +150,13 @@ public class MySqlInitialReadUtil {
|
||||
}
|
||||
|
||||
// Build the incremental CDC iterators.
|
||||
final AirbyteDebeziumHandler<MySqlCdcPosition> handler =
|
||||
new AirbyteDebeziumHandler<>(sourceConfig, MySqlCdcTargetPosition.targetPosition(database), true, firstRecordWaitTime, OptionalInt.empty());
|
||||
final AirbyteDebeziumHandler<MySqlCdcPosition> handler = new AirbyteDebeziumHandler<>(
|
||||
sourceConfig,
|
||||
MySqlCdcTargetPosition.targetPosition(database),
|
||||
true,
|
||||
firstRecordWaitTime,
|
||||
subsequentRecordWaitTime,
|
||||
OptionalInt.empty());
|
||||
|
||||
final Supplier<AutoCloseableIterator<AirbyteMessage>> incrementalIteratorSupplier = () -> handler.getIncrementalIterators(catalog,
|
||||
new MySqlCdcSavedInfoFetcher(stateToBeUsed),
|
||||
|
||||
@@ -4,11 +4,11 @@
|
||||
|
||||
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.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;
|
||||
@@ -21,18 +21,21 @@ import org.apache.commons.lang3.RandomStringUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
|
||||
public abstract class AbstractMySqlSourceDatatypeTest extends AbstractSourceDatabaseTypeTest {
|
||||
|
||||
protected static final Logger LOGGER = LoggerFactory.getLogger(AbstractMySqlSourceDatatypeTest.class);
|
||||
|
||||
protected MySQLContainer<?> container;
|
||||
protected JsonNode config;
|
||||
protected MySQLTestDatabase testdb;
|
||||
|
||||
@Override
|
||||
protected JsonNode getConfig() {
|
||||
return config;
|
||||
protected String getNameSpace() {
|
||||
return testdb.getDatabaseName();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void tearDown(final TestDestinationEnv testEnv) {
|
||||
testdb.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -40,14 +43,6 @@ public abstract class AbstractMySqlSourceDatatypeTest extends AbstractSourceData
|
||||
return "airbyte/source-mysql:dev";
|
||||
}
|
||||
|
||||
@Override
|
||||
protected abstract Database setupDatabase() throws Exception;
|
||||
|
||||
@Override
|
||||
protected String getNameSpace() {
|
||||
return container.getDatabaseName();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void initTests() {
|
||||
// bit defaults to bit(1), which is equivalent to boolean
|
||||
|
||||
@@ -1,80 +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.Database;
|
||||
import io.airbyte.cdk.db.MySqlUtils;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
|
||||
import io.airbyte.cdk.integrations.util.HostPortResolver;
|
||||
import io.airbyte.commons.features.EnvVariableFeatureFlags;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import java.io.IOException;
|
||||
import org.jooq.DSLContext;
|
||||
import org.jooq.SQLDialect;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
|
||||
public abstract class AbstractMySqlSslCertificateSourceAcceptanceTest extends MySqlSourceAcceptanceTest {
|
||||
|
||||
protected static MySqlUtils.Certificate certs;
|
||||
protected static final String PASSWORD = "Passw0rd";
|
||||
|
||||
@Override
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) throws Exception {
|
||||
environmentVariables.set(EnvVariableFeatureFlags.USE_STREAM_CAPABLE_STATE, "true");
|
||||
container = new MySQLContainer<>("mysql:8.0");
|
||||
container.start();
|
||||
addTestData(container);
|
||||
certs = getCertificates();
|
||||
|
||||
var sslMode = getSslConfig();
|
||||
final JsonNode replicationMethod = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("method", "STANDARD")
|
||||
.build());
|
||||
config = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, HostPortResolver.resolveHost(container))
|
||||
.put(JdbcUtils.PORT_KEY, HostPortResolver.resolvePort(container))
|
||||
.put(JdbcUtils.DATABASE_KEY, container.getDatabaseName())
|
||||
.put(JdbcUtils.USERNAME_KEY, container.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, container.getPassword())
|
||||
.put(JdbcUtils.SSL_KEY, true)
|
||||
.put(JdbcUtils.SSL_MODE_KEY, sslMode)
|
||||
.put("replication_method", replicationMethod)
|
||||
.build());
|
||||
}
|
||||
|
||||
public abstract MySqlUtils.Certificate getCertificates() throws IOException, InterruptedException;
|
||||
|
||||
public abstract ImmutableMap getSslConfig();
|
||||
|
||||
private void addTestData(MySQLContainer container) throws Exception {
|
||||
try (final DSLContext dslContext = DSLContextFactory.create(
|
||||
container.getUsername(),
|
||||
container.getPassword(),
|
||||
DatabaseDriver.MYSQL.getDriverClassName(),
|
||||
String.format("jdbc:mysql://%s:%s/%s",
|
||||
container.getHost(),
|
||||
container.getFirstMappedPort(),
|
||||
container.getDatabaseName()),
|
||||
SQLDialect.MYSQL)) {
|
||||
final Database database = new Database(dslContext);
|
||||
|
||||
database.query(ctx -> {
|
||||
ctx.fetch("CREATE TABLE id_and_name(id INTEGER, name VARCHAR(200));");
|
||||
ctx.fetch(
|
||||
"INSERT INTO id_and_name (id, name) VALUES (1,'picard'), (2, 'crusher'), (3, 'vash');");
|
||||
ctx.fetch("CREATE TABLE starships(id INTEGER, name VARCHAR(200));");
|
||||
ctx.fetch(
|
||||
"INSERT INTO starships (id, name) VALUES (1,'enterprise-d'), (2, 'defiant'), (3, 'yamato');");
|
||||
return null;
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -10,6 +10,8 @@ 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.features.FeatureFlags;
|
||||
import io.airbyte.commons.features.FeatureFlagsWrapper;
|
||||
import io.airbyte.commons.io.IOs;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.protocol.models.Field;
|
||||
@@ -28,19 +30,22 @@ public abstract class AbstractSshMySqlSourceAcceptanceTest extends SourceAccepta
|
||||
private static final String STREAM_NAME = "id_and_name";
|
||||
private static final String STREAM_NAME2 = "starships";
|
||||
|
||||
protected static JsonNode config;
|
||||
@Override
|
||||
protected FeatureFlags featureFlags() {
|
||||
return FeatureFlagsWrapper.overridingUseStreamCapableState(super.featureFlags(), true);
|
||||
}
|
||||
|
||||
private JsonNode config;
|
||||
|
||||
public abstract Path getConfigFilePath();
|
||||
|
||||
@Override
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) throws Exception {
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) {
|
||||
config = Jsons.deserialize(IOs.readFile(getConfigFilePath()));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void tearDown(final TestDestinationEnv testEnv) {
|
||||
|
||||
}
|
||||
protected void tearDown(final TestDestinationEnv testEnv) {}
|
||||
|
||||
@Override
|
||||
protected String getImageName() {
|
||||
|
||||
@@ -4,98 +4,25 @@
|
||||
|
||||
package io.airbyte.integrations.io.airbyte.integration_tests.sources;
|
||||
|
||||
import static io.airbyte.integrations.io.airbyte.integration_tests.sources.utils.TestConstants.INITIAL_CDC_WAITING_SECONDS;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
|
||||
import io.airbyte.cdk.integrations.util.HostPortResolver;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import org.jooq.DSLContext;
|
||||
import org.jooq.SQLDialect;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
import io.airbyte.integrations.source.mysql.MySQLTestDatabase;
|
||||
|
||||
public class CDCMySqlDatatypeAccuracyTest extends MySqlDatatypeAccuracyTest {
|
||||
|
||||
private DSLContext dslContext;
|
||||
|
||||
@Override
|
||||
protected void tearDown(final TestDestinationEnv testEnv) {
|
||||
dslContext.close();
|
||||
super.tearDown(testEnv);
|
||||
protected JsonNode getConfig() {
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.withoutSsl()
|
||||
.withCdcReplication()
|
||||
.with("snapshot_mode", "initial_only")
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Database setupDatabase() throws Exception {
|
||||
container = new MySQLContainer<>("mysql:8.0");
|
||||
container.start();
|
||||
final JsonNode replicationMethod = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("method", "CDC")
|
||||
.put("initial_waiting_seconds", INITIAL_CDC_WAITING_SECONDS)
|
||||
.build());
|
||||
config = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, HostPortResolver.resolveHost(container))
|
||||
.put(JdbcUtils.PORT_KEY, HostPortResolver.resolvePort(container))
|
||||
.put(JdbcUtils.DATABASE_KEY, container.getDatabaseName())
|
||||
.put(JdbcUtils.USERNAME_KEY, container.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, container.getPassword())
|
||||
.put("replication_method", replicationMethod)
|
||||
.put("snapshot_mode", "initial_only")
|
||||
.put("is_test", true)
|
||||
.build());
|
||||
|
||||
dslContext = DSLContextFactory.create(
|
||||
config.get(JdbcUtils.USERNAME_KEY).asText(),
|
||||
config.get(JdbcUtils.PASSWORD_KEY).asText(),
|
||||
DatabaseDriver.MYSQL.getDriverClassName(),
|
||||
String.format(DatabaseDriver.MYSQL.getUrlFormatString(),
|
||||
container.getHost(),
|
||||
container.getFirstMappedPort(),
|
||||
config.get(JdbcUtils.DATABASE_KEY).asText()),
|
||||
SQLDialect.MYSQL);
|
||||
final Database database = new Database(dslContext);
|
||||
|
||||
// It disable strict mode in the DB and allows to insert specific values.
|
||||
// For example, it's possible to insert date with zero values "2021-00-00"
|
||||
database.query(ctx -> ctx.fetch("SET @@sql_mode=''"));
|
||||
|
||||
revokeAllPermissions();
|
||||
grantCorrectPermissions();
|
||||
|
||||
return database;
|
||||
}
|
||||
|
||||
private void revokeAllPermissions() {
|
||||
executeQuery("REVOKE ALL PRIVILEGES, GRANT OPTION FROM " + container.getUsername() + "@'%';");
|
||||
}
|
||||
|
||||
private void grantCorrectPermissions() {
|
||||
executeQuery(
|
||||
"GRANT SELECT, RELOAD, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT ON *.* TO "
|
||||
+ container.getUsername() + "@'%';");
|
||||
}
|
||||
|
||||
private void executeQuery(final String query) {
|
||||
try (final DSLContext dslContext = DSLContextFactory.create(
|
||||
"root",
|
||||
"test",
|
||||
DatabaseDriver.MYSQL.getDriverClassName(),
|
||||
String.format(DatabaseDriver.MYSQL.getUrlFormatString(),
|
||||
container.getHost(),
|
||||
container.getFirstMappedPort(),
|
||||
container.getDatabaseName()),
|
||||
SQLDialect.MYSQL)) {
|
||||
final Database database = new Database(dslContext);
|
||||
database.query(
|
||||
ctx -> ctx
|
||||
.execute(query));
|
||||
} catch (final Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
protected Database setupDatabase() {
|
||||
testdb = MySQLTestDatabase.in("mysql:8.0").withoutStrictMode().withCdcPermissions();
|
||||
return testdb.getDatabase();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -4,47 +4,42 @@
|
||||
|
||||
package io.airbyte.integrations.io.airbyte.integration_tests.sources;
|
||||
|
||||
import static io.airbyte.integrations.io.airbyte.integration_tests.sources.utils.TestConstants.INITIAL_CDC_WAITING_SECONDS;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDataHolder;
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
|
||||
import io.airbyte.cdk.integrations.util.HostPortResolver;
|
||||
import io.airbyte.commons.features.EnvVariableFeatureFlags;
|
||||
import io.airbyte.commons.features.FeatureFlags;
|
||||
import io.airbyte.commons.features.FeatureFlagsWrapper;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.integrations.source.mysql.MySQLTestDatabase;
|
||||
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;
|
||||
import org.jooq.DSLContext;
|
||||
import org.jooq.SQLDialect;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
import uk.org.webcompere.systemstubs.environment.EnvironmentVariables;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStub;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStubsExtension;
|
||||
|
||||
@ExtendWith(SystemStubsExtension.class)
|
||||
public class CdcBinlogsMySqlSourceDatatypeTest extends AbstractMySqlSourceDatatypeTest {
|
||||
|
||||
@SystemStub
|
||||
private EnvironmentVariables environmentVariables;
|
||||
|
||||
private DSLContext dslContext;
|
||||
private JsonNode stateAfterFirstSync;
|
||||
|
||||
@Override
|
||||
protected void tearDown(final TestDestinationEnv testEnv) {
|
||||
dslContext.close();
|
||||
container.close();
|
||||
protected FeatureFlags featureFlags() {
|
||||
return FeatureFlagsWrapper.overridingUseStreamCapableState(super.featureFlags(), true);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected JsonNode getConfig() {
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.withoutSsl()
|
||||
.withCdcReplication()
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Database setupDatabase() {
|
||||
testdb = MySQLTestDatabase.in("mysql:8.0").withoutStrictMode().withCdcPermissions();
|
||||
return testdb.getDatabase();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -57,11 +52,10 @@ public class CdcBinlogsMySqlSourceDatatypeTest extends AbstractMySqlSourceDataty
|
||||
|
||||
@Override
|
||||
protected void postSetup() throws Exception {
|
||||
final Database database = setupDatabase();
|
||||
initTests();
|
||||
final var database = testdb.getDatabase();
|
||||
for (final TestDataHolder test : testDataHolders) {
|
||||
database.query(ctx -> {
|
||||
ctx.fetch(test.getCreateSqlQuery());
|
||||
ctx.execute("TRUNCATE TABLE " + test.getNameWithTestPrefix() + ";");
|
||||
return null;
|
||||
});
|
||||
}
|
||||
@@ -84,75 +78,6 @@ public class CdcBinlogsMySqlSourceDatatypeTest extends AbstractMySqlSourceDataty
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Database setupDatabase() throws Exception {
|
||||
container = new MySQLContainer<>("mysql:8.0");
|
||||
container.start();
|
||||
environmentVariables.set(EnvVariableFeatureFlags.USE_STREAM_CAPABLE_STATE, "true");
|
||||
final JsonNode replicationMethod = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("method", "CDC")
|
||||
.put("initial_waiting_seconds", INITIAL_CDC_WAITING_SECONDS)
|
||||
.build());
|
||||
config = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, HostPortResolver.resolveHost(container))
|
||||
.put(JdbcUtils.PORT_KEY, HostPortResolver.resolvePort(container))
|
||||
.put(JdbcUtils.DATABASE_KEY, container.getDatabaseName())
|
||||
.put(JdbcUtils.USERNAME_KEY, container.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, container.getPassword())
|
||||
.put("replication_method", replicationMethod)
|
||||
.put("is_test", true)
|
||||
.build());
|
||||
|
||||
dslContext = DSLContextFactory.create(
|
||||
config.get(JdbcUtils.USERNAME_KEY).asText(),
|
||||
config.get(JdbcUtils.PASSWORD_KEY).asText(),
|
||||
DatabaseDriver.MYSQL.getDriverClassName(),
|
||||
String.format(DatabaseDriver.MYSQL.getUrlFormatString(),
|
||||
container.getHost(),
|
||||
container.getFirstMappedPort(),
|
||||
config.get(JdbcUtils.DATABASE_KEY).asText()),
|
||||
SQLDialect.MYSQL);
|
||||
final Database database = new Database(dslContext);
|
||||
|
||||
// It disable strict mode in the DB and allows to insert specific values.
|
||||
// For example, it's possible to insert date with zero values "2021-00-00"
|
||||
database.query(ctx -> ctx.fetch("SET @@sql_mode=''"));
|
||||
|
||||
revokeAllPermissions();
|
||||
grantCorrectPermissions();
|
||||
|
||||
return database;
|
||||
}
|
||||
|
||||
private void revokeAllPermissions() {
|
||||
executeQuery("REVOKE ALL PRIVILEGES, GRANT OPTION FROM " + container.getUsername() + "@'%';");
|
||||
}
|
||||
|
||||
private void grantCorrectPermissions() {
|
||||
executeQuery(
|
||||
"GRANT SELECT, RELOAD, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT ON *.* TO "
|
||||
+ container.getUsername() + "@'%';");
|
||||
}
|
||||
|
||||
private void executeQuery(final String query) {
|
||||
try (final DSLContext dslContext = DSLContextFactory.create(
|
||||
"root",
|
||||
"test",
|
||||
DatabaseDriver.MYSQL.getDriverClassName(),
|
||||
String.format(DatabaseDriver.MYSQL.getUrlFormatString(),
|
||||
container.getHost(),
|
||||
container.getFirstMappedPort(),
|
||||
container.getDatabaseName()),
|
||||
SQLDialect.MYSQL)) {
|
||||
final Database database = new Database(dslContext);
|
||||
database.query(
|
||||
ctx -> ctx
|
||||
.execute(query));
|
||||
} catch (final Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean testCatalog() {
|
||||
return true;
|
||||
|
||||
@@ -4,108 +4,32 @@
|
||||
|
||||
package io.airbyte.integrations.io.airbyte.integration_tests.sources;
|
||||
|
||||
import static io.airbyte.integrations.io.airbyte.integration_tests.sources.utils.TestConstants.INITIAL_CDC_WAITING_SECONDS;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
|
||||
import io.airbyte.cdk.integrations.util.HostPortResolver;
|
||||
import io.airbyte.commons.features.EnvVariableFeatureFlags;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import org.jooq.DSLContext;
|
||||
import org.jooq.SQLDialect;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
import uk.org.webcompere.systemstubs.environment.EnvironmentVariables;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStub;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStubsExtension;
|
||||
import io.airbyte.commons.features.FeatureFlags;
|
||||
import io.airbyte.commons.features.FeatureFlagsWrapper;
|
||||
import io.airbyte.integrations.source.mysql.MySQLTestDatabase;
|
||||
|
||||
@ExtendWith(SystemStubsExtension.class)
|
||||
public class CdcInitialSnapshotMySqlSourceDatatypeTest extends AbstractMySqlSourceDatatypeTest {
|
||||
|
||||
@SystemStub
|
||||
private EnvironmentVariables environmentVariables;
|
||||
|
||||
private DSLContext dslContext;
|
||||
|
||||
@Override
|
||||
protected void tearDown(final TestDestinationEnv testEnv) {
|
||||
dslContext.close();
|
||||
container.close();
|
||||
protected FeatureFlags featureFlags() {
|
||||
return FeatureFlagsWrapper.overridingUseStreamCapableState(super.featureFlags(), true);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Database setupDatabase() throws Exception {
|
||||
container = new MySQLContainer<>("mysql:8.0");
|
||||
container.start();
|
||||
environmentVariables.set(EnvVariableFeatureFlags.USE_STREAM_CAPABLE_STATE, "true");
|
||||
final JsonNode replicationMethod = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("method", "CDC")
|
||||
.put("initial_waiting_seconds", INITIAL_CDC_WAITING_SECONDS)
|
||||
.build());
|
||||
config = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, HostPortResolver.resolveHost(container))
|
||||
.put(JdbcUtils.PORT_KEY, HostPortResolver.resolvePort(container))
|
||||
.put(JdbcUtils.DATABASE_KEY, container.getDatabaseName())
|
||||
.put(JdbcUtils.USERNAME_KEY, container.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, container.getPassword())
|
||||
.put("replication_method", replicationMethod)
|
||||
.put("snapshot_mode", "initial_only")
|
||||
.put("is_test", true)
|
||||
.build());
|
||||
|
||||
dslContext = DSLContextFactory.create(
|
||||
config.get(JdbcUtils.USERNAME_KEY).asText(),
|
||||
config.get(JdbcUtils.PASSWORD_KEY).asText(),
|
||||
DatabaseDriver.MYSQL.getDriverClassName(),
|
||||
String.format(DatabaseDriver.MYSQL.getUrlFormatString(),
|
||||
container.getHost(),
|
||||
container.getFirstMappedPort(),
|
||||
config.get(JdbcUtils.DATABASE_KEY).asText()),
|
||||
SQLDialect.MYSQL);
|
||||
final Database database = new Database(dslContext);
|
||||
|
||||
// It disable strict mode in the DB and allows to insert specific values.
|
||||
// For example, it's possible to insert date with zero values "2021-00-00"
|
||||
database.query(ctx -> ctx.fetch("SET @@sql_mode=''"));
|
||||
|
||||
revokeAllPermissions();
|
||||
grantCorrectPermissions();
|
||||
|
||||
return database;
|
||||
protected JsonNode getConfig() {
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.withoutSsl()
|
||||
.withCdcReplication()
|
||||
.with("snapshot_mode", "initial_only")
|
||||
.build();
|
||||
}
|
||||
|
||||
private void revokeAllPermissions() {
|
||||
executeQuery("REVOKE ALL PRIVILEGES, GRANT OPTION FROM " + container.getUsername() + "@'%';");
|
||||
}
|
||||
|
||||
private void grantCorrectPermissions() {
|
||||
executeQuery(
|
||||
"GRANT SELECT, RELOAD, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT ON *.* TO "
|
||||
+ container.getUsername() + "@'%';");
|
||||
}
|
||||
|
||||
private void executeQuery(final String query) {
|
||||
try (final DSLContext dslContext = DSLContextFactory.create(
|
||||
"root",
|
||||
"test",
|
||||
DatabaseDriver.MYSQL.getDriverClassName(),
|
||||
String.format(DatabaseDriver.MYSQL.getUrlFormatString(),
|
||||
container.getHost(),
|
||||
container.getFirstMappedPort(),
|
||||
container.getDatabaseName()),
|
||||
SQLDialect.MYSQL)) {
|
||||
final Database database = new Database(dslContext);
|
||||
database.query(
|
||||
ctx -> ctx
|
||||
.execute(query));
|
||||
} catch (final Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
@Override
|
||||
protected Database setupDatabase() {
|
||||
testdb = MySQLTestDatabase.in("mysql:8.0").withoutStrictMode().withCdcPermissions();
|
||||
return testdb.getDatabase();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -4,26 +4,21 @@
|
||||
|
||||
package io.airbyte.integrations.io.airbyte.integration_tests.sources;
|
||||
|
||||
import static io.airbyte.integrations.io.airbyte.integration_tests.sources.utils.TestConstants.INITIAL_CDC_WAITING_SECONDS;
|
||||
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.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
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.cdk.integrations.util.HostPortResolver;
|
||||
import io.airbyte.commons.features.EnvVariableFeatureFlags;
|
||||
import io.airbyte.commons.features.FeatureFlags;
|
||||
import io.airbyte.commons.features.FeatureFlagsWrapper;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.integrations.source.mysql.MySQLTestDatabase;
|
||||
import io.airbyte.protocol.models.Field;
|
||||
import io.airbyte.protocol.models.JsonSchemaType;
|
||||
import io.airbyte.protocol.models.v0.AirbyteMessage;
|
||||
@@ -37,25 +32,20 @@ import io.airbyte.protocol.models.v0.DestinationSyncMode;
|
||||
import io.airbyte.protocol.models.v0.SyncMode;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import org.jooq.DSLContext;
|
||||
import org.jooq.SQLDialect;
|
||||
import java.util.stream.Stream;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
import uk.org.webcompere.systemstubs.environment.EnvironmentVariables;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStub;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStubsExtension;
|
||||
|
||||
@ExtendWith(SystemStubsExtension.class)
|
||||
public class CdcMySqlSourceAcceptanceTest extends SourceAcceptanceTest {
|
||||
|
||||
@SystemStub
|
||||
protected EnvironmentVariables environmentVariables;
|
||||
|
||||
protected static final String STREAM_NAME = "id_and_name";
|
||||
protected static final String STREAM_NAME2 = "starships";
|
||||
protected MySQLContainer<?> container;
|
||||
protected JsonNode config;
|
||||
|
||||
protected MySQLTestDatabase testdb;
|
||||
|
||||
@Override
|
||||
protected FeatureFlags featureFlags() {
|
||||
return FeatureFlagsWrapper.overridingUseStreamCapableState(super.featureFlags(), true);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getImageName() {
|
||||
@@ -69,7 +59,10 @@ public class CdcMySqlSourceAcceptanceTest extends SourceAcceptanceTest {
|
||||
|
||||
@Override
|
||||
protected JsonNode getConfig() {
|
||||
return config;
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.withCdcReplication()
|
||||
.withoutSsl()
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -80,7 +73,7 @@ public class CdcMySqlSourceAcceptanceTest extends SourceAcceptanceTest {
|
||||
.withDestinationSyncMode(DestinationSyncMode.APPEND)
|
||||
.withStream(CatalogHelpers.createAirbyteStream(
|
||||
String.format("%s", STREAM_NAME),
|
||||
String.format("%s", config.get(JdbcUtils.DATABASE_KEY).asText()),
|
||||
testdb.getDatabaseName(),
|
||||
Field.of("id", JsonSchemaType.NUMBER),
|
||||
Field.of("name", JsonSchemaType.STRING))
|
||||
.withSourceDefinedCursor(true)
|
||||
@@ -92,7 +85,7 @@ public class CdcMySqlSourceAcceptanceTest extends SourceAcceptanceTest {
|
||||
.withDestinationSyncMode(DestinationSyncMode.APPEND)
|
||||
.withStream(CatalogHelpers.createAirbyteStream(
|
||||
String.format("%s", STREAM_NAME2),
|
||||
String.format("%s", config.get(JdbcUtils.DATABASE_KEY).asText()),
|
||||
testdb.getDatabaseName(),
|
||||
Field.of("id", JsonSchemaType.NUMBER),
|
||||
Field.of("name", JsonSchemaType.STRING))
|
||||
.withSourceDefinedCursor(true)
|
||||
@@ -107,70 +100,22 @@ public class CdcMySqlSourceAcceptanceTest extends SourceAcceptanceTest {
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) throws Exception {
|
||||
container = new MySQLContainer<>("mysql:8.0");
|
||||
container.start();
|
||||
final JsonNode replicationMethod = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("method", "CDC")
|
||||
.put("initial_waiting_seconds", INITIAL_CDC_WAITING_SECONDS)
|
||||
.build());
|
||||
environmentVariables.set(EnvVariableFeatureFlags.USE_STREAM_CAPABLE_STATE, "true");
|
||||
config = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, HostPortResolver.resolveHost(container))
|
||||
.put(JdbcUtils.PORT_KEY, HostPortResolver.resolvePort(container))
|
||||
.put(JdbcUtils.DATABASE_KEY, container.getDatabaseName())
|
||||
.put(JdbcUtils.USERNAME_KEY, container.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, container.getPassword())
|
||||
.put("replication_method", replicationMethod)
|
||||
.put("is_test", true)
|
||||
.build());
|
||||
|
||||
revokeAllPermissions();
|
||||
grantCorrectPermissions();
|
||||
createAndPopulateTables();
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) {
|
||||
testdb = MySQLTestDatabase.in("mysql:8.0", extraContainerFactoryMethods().toArray(String[]::new))
|
||||
.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');");
|
||||
}
|
||||
|
||||
protected void createAndPopulateTables() {
|
||||
executeQuery("CREATE TABLE id_and_name(id INTEGER PRIMARY KEY, name VARCHAR(200));");
|
||||
executeQuery(
|
||||
"INSERT INTO id_and_name (id, name) VALUES (1,'picard'), (2, 'crusher'), (3, 'vash');");
|
||||
executeQuery("CREATE TABLE starships(id INTEGER PRIMARY KEY, name VARCHAR(200));");
|
||||
executeQuery(
|
||||
"INSERT INTO starships (id, name) VALUES (1,'enterprise-d'), (2, 'defiant'), (3, 'yamato');");
|
||||
}
|
||||
|
||||
protected void revokeAllPermissions() {
|
||||
executeQuery("REVOKE ALL PRIVILEGES, GRANT OPTION FROM " + container.getUsername() + "@'%';");
|
||||
}
|
||||
|
||||
protected void grantCorrectPermissions() {
|
||||
executeQuery(
|
||||
"GRANT SELECT, RELOAD, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT ON *.* TO "
|
||||
+ container.getUsername() + "@'%';");
|
||||
}
|
||||
|
||||
protected void executeQuery(final String query) {
|
||||
try (final DSLContext dslContext = DSLContextFactory.create(
|
||||
"root",
|
||||
"test",
|
||||
DatabaseDriver.MYSQL.getDriverClassName(),
|
||||
String.format(DatabaseDriver.MYSQL.getUrlFormatString(),
|
||||
container.getHost(),
|
||||
container.getFirstMappedPort(),
|
||||
container.getDatabaseName()),
|
||||
SQLDialect.MYSQL)) {
|
||||
final Database database = new Database(dslContext);
|
||||
database.query(
|
||||
ctx -> ctx
|
||||
.execute(query));
|
||||
} catch (final Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
protected Stream<String> extraContainerFactoryMethods() {
|
||||
return Stream.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void tearDown(final TestDestinationEnv testEnv) {
|
||||
container.close();
|
||||
testdb.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -195,7 +140,7 @@ public class CdcMySqlSourceAcceptanceTest extends SourceAcceptanceTest {
|
||||
final JsonNode latestState = Jsons.jsonNode(supportsPerStream() ? stateMessages : 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
|
||||
executeQuery("RESET MASTER;");
|
||||
testdb.with("RESET MASTER;");
|
||||
|
||||
assertEquals(6, filterRecords(runRead(configuredCatalog, latestState)).size());
|
||||
}
|
||||
@@ -219,7 +164,7 @@ public class CdcMySqlSourceAcceptanceTest extends SourceAcceptanceTest {
|
||||
.withDestinationSyncMode(DestinationSyncMode.APPEND)
|
||||
.withStream(CatalogHelpers.createAirbyteStream(
|
||||
String.format("%s", STREAM_NAME),
|
||||
String.format("%s", config.get(JdbcUtils.DATABASE_KEY).asText()),
|
||||
testdb.getDatabaseName(),
|
||||
Field.of("id", JsonSchemaType.NUMBER)
|
||||
/* no name field */)
|
||||
.withSourceDefinedCursor(true)
|
||||
@@ -231,7 +176,7 @@ public class CdcMySqlSourceAcceptanceTest extends SourceAcceptanceTest {
|
||||
.withDestinationSyncMode(DestinationSyncMode.APPEND)
|
||||
.withStream(CatalogHelpers.createAirbyteStream(
|
||||
String.format("%s", STREAM_NAME2),
|
||||
String.format("%s", config.get(JdbcUtils.DATABASE_KEY).asText()),
|
||||
testdb.getDatabaseName(),
|
||||
/* no name field */
|
||||
Field.of("id", JsonSchemaType.NUMBER))
|
||||
.withSourceDefinedCursor(true)
|
||||
@@ -241,13 +186,8 @@ public class CdcMySqlSourceAcceptanceTest extends SourceAcceptanceTest {
|
||||
}
|
||||
|
||||
private void verifyFieldNotExist(final List<AirbyteRecordMessage> records, final String stream, final String field) {
|
||||
assertTrue(records.stream()
|
||||
.filter(r -> {
|
||||
return r.getStream().equals(stream)
|
||||
&& r.getData().get(field) != null;
|
||||
})
|
||||
.collect(Collectors.toList())
|
||||
.isEmpty(), "Records contain unselected columns [%s:%s]".formatted(stream, field));
|
||||
assertTrue(records.stream().noneMatch(r -> r.getStream().equals(stream) && r.getData().get(field) != null),
|
||||
"Records contain unselected columns [%s:%s]".formatted(stream, field));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -4,56 +4,27 @@
|
||||
|
||||
package io.airbyte.integrations.io.airbyte.integration_tests.sources;
|
||||
|
||||
import static io.airbyte.integrations.io.airbyte.integration_tests.sources.utils.TestConstants.INITIAL_CDC_WAITING_SECONDS;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.airbyte.cdk.db.MySqlUtils;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
|
||||
import io.airbyte.cdk.integrations.util.HostPortResolver;
|
||||
import io.airbyte.commons.features.EnvVariableFeatureFlags;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
public class CdcMySqlSslCaCertificateSourceAcceptanceTest extends CdcMySqlSourceAcceptanceTest {
|
||||
|
||||
private static MySqlUtils.Certificate certs;
|
||||
@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 void setupEnvironment(final TestDestinationEnv environment) throws Exception {
|
||||
container = new MySQLContainer<>("mysql:8.0");
|
||||
container.start();
|
||||
environmentVariables.set(EnvVariableFeatureFlags.USE_STREAM_CAPABLE_STATE, "true");
|
||||
certs = MySqlUtils.getCertificate(container, true);
|
||||
|
||||
final var sslMode = ImmutableMap.builder()
|
||||
.put(JdbcUtils.MODE_KEY, "verify_ca")
|
||||
.put("ca_certificate", certs.getCaCertificate())
|
||||
.put("client_certificate", certs.getClientCertificate())
|
||||
.put("client_key", certs.getClientKey())
|
||||
.put("client_key_password", "Passw0rd")
|
||||
.build();
|
||||
final JsonNode replicationMethod = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("method", "CDC")
|
||||
.put("initial_waiting_seconds", INITIAL_CDC_WAITING_SECONDS)
|
||||
.build());
|
||||
|
||||
config = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, HostPortResolver.resolveHost(container))
|
||||
.put(JdbcUtils.PORT_KEY, HostPortResolver.resolvePort(container))
|
||||
.put(JdbcUtils.DATABASE_KEY, container.getDatabaseName())
|
||||
.put(JdbcUtils.USERNAME_KEY, container.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, container.getPassword())
|
||||
.put(JdbcUtils.SSL_KEY, true)
|
||||
.put(JdbcUtils.SSL_MODE_KEY, sslMode)
|
||||
.put("replication_method", replicationMethod)
|
||||
.put("is_test", true)
|
||||
.build());
|
||||
|
||||
revokeAllPermissions();
|
||||
grantCorrectPermissions();
|
||||
createAndPopulateTables();
|
||||
protected Stream<String> extraContainerFactoryMethods() {
|
||||
return Stream.of("withRootAndServerCertificates");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -4,53 +4,31 @@
|
||||
|
||||
package io.airbyte.integrations.io.airbyte.integration_tests.sources;
|
||||
|
||||
import static io.airbyte.integrations.io.airbyte.integration_tests.sources.utils.TestConstants.INITIAL_CDC_WAITING_SECONDS;
|
||||
|
||||
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.cdk.integrations.util.HostPortResolver;
|
||||
import io.airbyte.commons.features.EnvVariableFeatureFlags;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
public class CdcMySqlSslRequiredSourceAcceptanceTest extends CdcMySqlSourceAcceptanceTest {
|
||||
|
||||
@Override
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) {
|
||||
container = new MySQLContainer<>("mysql:8.0");
|
||||
container.start();
|
||||
environmentVariables.set(EnvVariableFeatureFlags.USE_STREAM_CAPABLE_STATE, "true");
|
||||
|
||||
final var sslMode = ImmutableMap.builder()
|
||||
.put(JdbcUtils.MODE_KEY, "required")
|
||||
protected JsonNode getConfig() {
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.withCdcReplication()
|
||||
.withSsl(ImmutableMap.builder().put(JdbcUtils.MODE_KEY, "required").build())
|
||||
.build();
|
||||
final JsonNode replicationMethod = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("method", "CDC")
|
||||
.put("initial_waiting_seconds", INITIAL_CDC_WAITING_SECONDS)
|
||||
.build());
|
||||
|
||||
config = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, HostPortResolver.resolveHost(container))
|
||||
.put(JdbcUtils.PORT_KEY, HostPortResolver.resolvePort(container))
|
||||
.put(JdbcUtils.DATABASE_KEY, container.getDatabaseName())
|
||||
.put(JdbcUtils.USERNAME_KEY, container.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, container.getPassword())
|
||||
.put(JdbcUtils.SSL_KEY, true)
|
||||
.put(JdbcUtils.SSL_MODE_KEY, sslMode)
|
||||
.put("replication_method", replicationMethod)
|
||||
.put("is_test", true)
|
||||
.build());
|
||||
|
||||
revokeAllPermissions();
|
||||
grantCorrectPermissions();
|
||||
alterUserRequireSsl();
|
||||
createAndPopulateTables();
|
||||
}
|
||||
|
||||
private void alterUserRequireSsl() {
|
||||
executeQuery("ALTER USER " + container.getUsername() + " REQUIRE SSL;");
|
||||
@Override
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) {
|
||||
super.setupEnvironment(environment);
|
||||
testdb.with("ALTER USER %s REQUIRE SSL;", testdb.getUserName());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Stream<String> extraContainerFactoryMethods() {
|
||||
return Stream.of("withRootAndServerCertificates", "withClientCertificate");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -5,38 +5,42 @@
|
||||
package io.airbyte.integrations.io.airbyte.integration_tests.sources;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.mysql.cj.MysqlType;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDataHolder;
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
|
||||
import io.airbyte.cdk.integrations.util.HostPortResolver;
|
||||
import io.airbyte.commons.features.EnvVariableFeatureFlags;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.commons.features.FeatureFlags;
|
||||
import io.airbyte.commons.features.FeatureFlagsWrapper;
|
||||
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;
|
||||
import org.jooq.SQLDialect;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
import uk.org.webcompere.systemstubs.environment.EnvironmentVariables;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStub;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStubsExtension;
|
||||
|
||||
@ExtendWith(SystemStubsExtension.class)
|
||||
public class MySqlDatatypeAccuracyTest extends AbstractMySqlSourceDatatypeTest {
|
||||
|
||||
@SystemStub
|
||||
private EnvironmentVariables environmentVariables;
|
||||
@Override
|
||||
protected FeatureFlags featureFlags() {
|
||||
return FeatureFlagsWrapper.overridingUseStreamCapableState(super.featureFlags(), true);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void tearDown(final TestDestinationEnv testEnv) {
|
||||
container.close();
|
||||
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<String, List<String>> charsetsCollationsMap = Map.of(
|
||||
@@ -46,42 +50,6 @@ public class MySqlDatatypeAccuracyTest extends AbstractMySqlSourceDatatypeTest {
|
||||
"binary", Arrays.asList("binary"),
|
||||
"CP1250", Arrays.asList("CP1250_general_ci", "cp1250_czech_cs"));
|
||||
|
||||
@Override
|
||||
protected Database setupDatabase() throws Exception {
|
||||
environmentVariables.set(EnvVariableFeatureFlags.USE_STREAM_CAPABLE_STATE, "true");
|
||||
container = new MySQLContainer<>("mysql:8.0");
|
||||
container.start();
|
||||
final JsonNode replicationMethod = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("method", "STANDARD")
|
||||
.build());
|
||||
config = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, HostPortResolver.resolveHost(container))
|
||||
.put(JdbcUtils.PORT_KEY, HostPortResolver.resolvePort(container))
|
||||
.put(JdbcUtils.DATABASE_KEY, container.getDatabaseName())
|
||||
.put(JdbcUtils.USERNAME_KEY, container.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, container.getPassword())
|
||||
.put("replication_method", replicationMethod)
|
||||
.build());
|
||||
|
||||
final Database database = new Database(
|
||||
DSLContextFactory.create(
|
||||
config.get(JdbcUtils.USERNAME_KEY).asText(),
|
||||
config.get(JdbcUtils.PASSWORD_KEY).asText(),
|
||||
DatabaseDriver.MYSQL.getDriverClassName(),
|
||||
String.format(DatabaseDriver.MYSQL.getUrlFormatString(),
|
||||
container.getHost(),
|
||||
container.getFirstMappedPort(),
|
||||
config.get(JdbcUtils.DATABASE_KEY).asText()),
|
||||
SQLDialect.MYSQL,
|
||||
Map.of("zeroDateTimeBehavior", "convertToNull")));
|
||||
|
||||
// It disable strict mode in the DB and allows to insert specific values.
|
||||
// For example, it's possible to insert date with zero values "2021-00-00"
|
||||
database.query(ctx -> ctx.fetch("SET @@sql_mode=''"));
|
||||
|
||||
return database;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean testCatalog() {
|
||||
return true;
|
||||
|
||||
@@ -5,18 +5,14 @@
|
||||
package io.airbyte.integrations.io.airbyte.integration_tests.sources;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
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.cdk.integrations.util.HostPortResolver;
|
||||
import io.airbyte.commons.features.EnvVariableFeatureFlags;
|
||||
import io.airbyte.commons.features.FeatureFlags;
|
||||
import io.airbyte.commons.features.FeatureFlagsWrapper;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.integrations.source.mysql.MySQLTestDatabase;
|
||||
import io.airbyte.protocol.models.Field;
|
||||
import io.airbyte.protocol.models.JsonSchemaType;
|
||||
import io.airbyte.protocol.models.v0.CatalogHelpers;
|
||||
@@ -26,66 +22,36 @@ import io.airbyte.protocol.models.v0.ConnectorSpecification;
|
||||
import io.airbyte.protocol.models.v0.DestinationSyncMode;
|
||||
import io.airbyte.protocol.models.v0.SyncMode;
|
||||
import java.util.HashMap;
|
||||
import org.jooq.DSLContext;
|
||||
import org.jooq.SQLDialect;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
import uk.org.webcompere.systemstubs.environment.EnvironmentVariables;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStub;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStubsExtension;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
@ExtendWith(SystemStubsExtension.class)
|
||||
public class MySqlSourceAcceptanceTest extends SourceAcceptanceTest {
|
||||
|
||||
@SystemStub
|
||||
public EnvironmentVariables environmentVariables;
|
||||
protected MySQLTestDatabase testdb;
|
||||
|
||||
private static final String STREAM_NAME = "id_and_name";
|
||||
private static final String STREAM_NAME2 = "public.starships";
|
||||
|
||||
protected MySQLContainer<?> container;
|
||||
protected JsonNode config;
|
||||
|
||||
@Override
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) throws Exception {
|
||||
container = new MySQLContainer<>("mysql:8.0");
|
||||
container.start();
|
||||
final JsonNode replicationMethod = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("method", "STANDARD")
|
||||
.build());
|
||||
environmentVariables.set(EnvVariableFeatureFlags.USE_STREAM_CAPABLE_STATE, "true");
|
||||
config = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, HostPortResolver.resolveHost(container))
|
||||
.put(JdbcUtils.PORT_KEY, HostPortResolver.resolvePort(container))
|
||||
.put(JdbcUtils.DATABASE_KEY, container.getDatabaseName())
|
||||
.put(JdbcUtils.USERNAME_KEY, container.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, container.getPassword())
|
||||
.put("replication_method", replicationMethod)
|
||||
.build());
|
||||
testdb = MySQLTestDatabase.in("mysql:8.0", extraContainerFactoryMethods().toArray(String[]::new))
|
||||
.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');");
|
||||
}
|
||||
|
||||
try (final DSLContext dslContext = DSLContextFactory.create(
|
||||
config.get(JdbcUtils.USERNAME_KEY).asText(),
|
||||
config.get(JdbcUtils.PASSWORD_KEY).asText(),
|
||||
DatabaseDriver.MYSQL.getDriverClassName(),
|
||||
String.format(DatabaseDriver.MYSQL.getUrlFormatString(),
|
||||
container.getHost(),
|
||||
container.getFirstMappedPort(),
|
||||
config.get(JdbcUtils.DATABASE_KEY).asText()),
|
||||
SQLDialect.MYSQL)) {
|
||||
final Database database = new Database(dslContext);
|
||||
@Override
|
||||
protected FeatureFlags featureFlags() {
|
||||
return FeatureFlagsWrapper.overridingUseStreamCapableState(super.featureFlags(), true);
|
||||
}
|
||||
|
||||
database.query(ctx -> {
|
||||
ctx.fetch("CREATE TABLE id_and_name(id INTEGER, name VARCHAR(200));");
|
||||
ctx.fetch("INSERT INTO id_and_name (id, name) VALUES (1,'picard'), (2, 'crusher'), (3, 'vash');");
|
||||
ctx.fetch("CREATE TABLE starships(id INTEGER, name VARCHAR(200));");
|
||||
ctx.fetch("INSERT INTO starships (id, name) VALUES (1,'enterprise-d'), (2, 'defiant'), (3, 'yamato');");
|
||||
return null;
|
||||
});
|
||||
}
|
||||
protected Stream<String> extraContainerFactoryMethods() {
|
||||
return Stream.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void tearDown(final TestDestinationEnv testEnv) {
|
||||
container.close();
|
||||
testdb.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -100,7 +66,10 @@ public class MySqlSourceAcceptanceTest extends SourceAcceptanceTest {
|
||||
|
||||
@Override
|
||||
protected JsonNode getConfig() {
|
||||
return config;
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.withStandardReplication()
|
||||
.withoutSsl()
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -111,7 +80,7 @@ public class MySqlSourceAcceptanceTest extends SourceAcceptanceTest {
|
||||
.withCursorField(Lists.newArrayList("id"))
|
||||
.withDestinationSyncMode(DestinationSyncMode.APPEND)
|
||||
.withStream(CatalogHelpers.createAirbyteStream(
|
||||
String.format("%s.%s", config.get(JdbcUtils.DATABASE_KEY).asText(), STREAM_NAME),
|
||||
String.format("%s.%s", testdb.getDatabaseName(), STREAM_NAME),
|
||||
Field.of("id", JsonSchemaType.NUMBER),
|
||||
Field.of("name", JsonSchemaType.STRING))
|
||||
.withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL))),
|
||||
@@ -120,7 +89,7 @@ public class MySqlSourceAcceptanceTest extends SourceAcceptanceTest {
|
||||
.withCursorField(Lists.newArrayList("id"))
|
||||
.withDestinationSyncMode(DestinationSyncMode.APPEND)
|
||||
.withStream(CatalogHelpers.createAirbyteStream(
|
||||
String.format("%s.%s", config.get(JdbcUtils.DATABASE_KEY).asText(), STREAM_NAME2),
|
||||
String.format("%s.%s", testdb.getDatabaseName(), STREAM_NAME2),
|
||||
Field.of("id", JsonSchemaType.NUMBER),
|
||||
Field.of("name", JsonSchemaType.STRING))
|
||||
.withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL)))));
|
||||
|
||||
@@ -5,68 +5,35 @@
|
||||
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.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
|
||||
import io.airbyte.cdk.integrations.util.HostPortResolver;
|
||||
import io.airbyte.commons.features.EnvVariableFeatureFlags;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import java.util.Map;
|
||||
import org.jooq.SQLDialect;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
import uk.org.webcompere.systemstubs.environment.EnvironmentVariables;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStub;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStubsExtension;
|
||||
import io.airbyte.commons.features.FeatureFlags;
|
||||
import io.airbyte.commons.features.FeatureFlagsWrapper;
|
||||
import io.airbyte.integrations.source.mysql.MySQLContainerFactory;
|
||||
import io.airbyte.integrations.source.mysql.MySQLTestDatabase;
|
||||
|
||||
@ExtendWith(SystemStubsExtension.class)
|
||||
public class MySqlSourceDatatypeTest extends AbstractMySqlSourceDatatypeTest {
|
||||
|
||||
@SystemStub
|
||||
private EnvironmentVariables environmentVariables;
|
||||
|
||||
@Override
|
||||
protected void tearDown(final TestDestinationEnv testEnv) {
|
||||
container.close();
|
||||
protected FeatureFlags featureFlags() {
|
||||
return FeatureFlagsWrapper.overridingUseStreamCapableState(super.featureFlags(), true);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Database setupDatabase() throws Exception {
|
||||
environmentVariables.set(EnvVariableFeatureFlags.USE_STREAM_CAPABLE_STATE, "true");
|
||||
container = new MySQLContainer<>("mysql:8.0");
|
||||
container.start();
|
||||
final JsonNode replicationMethod = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("method", "STANDARD")
|
||||
.build());
|
||||
config = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, HostPortResolver.resolveHost(container))
|
||||
.put(JdbcUtils.PORT_KEY, HostPortResolver.resolvePort(container))
|
||||
.put(JdbcUtils.DATABASE_KEY, container.getDatabaseName())
|
||||
.put(JdbcUtils.USERNAME_KEY, container.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, container.getPassword())
|
||||
.put("replication_method", replicationMethod)
|
||||
.build());
|
||||
protected JsonNode getConfig() {
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.withoutSsl()
|
||||
.withStandardReplication()
|
||||
.build();
|
||||
}
|
||||
|
||||
final Database database = new Database(
|
||||
DSLContextFactory.create(
|
||||
config.get(JdbcUtils.USERNAME_KEY).asText(),
|
||||
config.get(JdbcUtils.PASSWORD_KEY).asText(),
|
||||
DatabaseDriver.MYSQL.getDriverClassName(),
|
||||
String.format(DatabaseDriver.MYSQL.getUrlFormatString(),
|
||||
container.getHost(),
|
||||
container.getFirstMappedPort(),
|
||||
config.get(JdbcUtils.DATABASE_KEY).asText()),
|
||||
SQLDialect.MYSQL,
|
||||
Map.of("zeroDateTimeBehavior", "convertToNull")));
|
||||
|
||||
// It disable strict mode in the DB and allows to insert specific values.
|
||||
// For example, it's possible to insert date with zero values "2021-00-00"
|
||||
database.query(ctx -> ctx.fetch("SET @@sql_mode=''"));
|
||||
|
||||
return database;
|
||||
@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
|
||||
|
||||
@@ -4,24 +4,29 @@
|
||||
|
||||
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.MySqlUtils;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import java.io.IOException;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
public class MySqlSslCaCertificateSourceAcceptanceTest extends AbstractMySqlSslCertificateSourceAcceptanceTest {
|
||||
public class MySqlSslCaCertificateSourceAcceptanceTest extends MySqlSourceAcceptanceTest {
|
||||
|
||||
private static final String PASSWORD = "Passw0rd";
|
||||
|
||||
@Override
|
||||
public MySqlUtils.Certificate getCertificates() throws IOException, InterruptedException {
|
||||
return MySqlUtils.getCertificate(container, false);
|
||||
protected Stream<String> extraContainerFactoryMethods() {
|
||||
return Stream.of("withRootAndServerCertificates");
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableMap getSslConfig() {
|
||||
return ImmutableMap.builder()
|
||||
.put(JdbcUtils.MODE_KEY, "verify_ca")
|
||||
.put("ca_certificate", certs.getCaCertificate())
|
||||
.put("client_key_password", PASSWORD)
|
||||
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();
|
||||
}
|
||||
|
||||
|
||||
@@ -4,26 +4,31 @@
|
||||
|
||||
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.MySqlUtils;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import java.io.IOException;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
public class MySqlSslFullCertificateSourceAcceptanceTest extends AbstractMySqlSslCertificateSourceAcceptanceTest {
|
||||
public class MySqlSslFullCertificateSourceAcceptanceTest extends MySqlSourceAcceptanceTest {
|
||||
|
||||
private static final String PASSWORD = "Passw0rd";
|
||||
|
||||
@Override
|
||||
public MySqlUtils.Certificate getCertificates() throws IOException, InterruptedException {
|
||||
return MySqlUtils.getCertificate(container, true);
|
||||
protected Stream<String> extraContainerFactoryMethods() {
|
||||
return Stream.of("withRootAndServerCertificates", "withClientCertificate");
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableMap getSslConfig() {
|
||||
return ImmutableMap.builder()
|
||||
.put(JdbcUtils.MODE_KEY, "verify_ca")
|
||||
.put("ca_certificate", certs.getCaCertificate())
|
||||
.put("client_certificate", certs.getClientCertificate())
|
||||
.put("client_key", certs.getClientKey())
|
||||
.put("client_key_password", PASSWORD)
|
||||
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();
|
||||
}
|
||||
|
||||
|
||||
@@ -6,65 +6,16 @@ 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.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
|
||||
import io.airbyte.cdk.integrations.util.HostPortResolver;
|
||||
import io.airbyte.commons.features.EnvVariableFeatureFlags;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import org.jooq.DSLContext;
|
||||
import org.jooq.SQLDialect;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
|
||||
public class MySqlSslSourceAcceptanceTest extends MySqlSourceAcceptanceTest {
|
||||
|
||||
@Override
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) throws Exception {
|
||||
environmentVariables.set(EnvVariableFeatureFlags.USE_STREAM_CAPABLE_STATE, "true");
|
||||
container = new MySQLContainer<>("mysql:8.0");
|
||||
container.start();
|
||||
final JsonNode replicationMethod = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("method", "STANDARD")
|
||||
.build());
|
||||
|
||||
var sslMode = ImmutableMap.builder()
|
||||
.put(JdbcUtils.MODE_KEY, "required")
|
||||
protected JsonNode getConfig() {
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.withStandardReplication()
|
||||
.withSsl(ImmutableMap.builder().put(JdbcUtils.MODE_KEY, "required").build())
|
||||
.build();
|
||||
|
||||
config = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, HostPortResolver.resolveHost(container))
|
||||
.put(JdbcUtils.PORT_KEY, HostPortResolver.resolvePort(container))
|
||||
.put(JdbcUtils.DATABASE_KEY, container.getDatabaseName())
|
||||
.put(JdbcUtils.USERNAME_KEY, container.getUsername())
|
||||
.put(JdbcUtils.PASSWORD_KEY, container.getPassword())
|
||||
.put(JdbcUtils.SSL_KEY, true)
|
||||
.put(JdbcUtils.SSL_MODE_KEY, sslMode)
|
||||
.put("replication_method", replicationMethod)
|
||||
.build());
|
||||
|
||||
try (final DSLContext dslContext = DSLContextFactory.create(
|
||||
config.get(JdbcUtils.USERNAME_KEY).asText(),
|
||||
config.get(JdbcUtils.PASSWORD_KEY).asText(),
|
||||
DatabaseDriver.MYSQL.getDriverClassName(),
|
||||
String.format("jdbc:mysql://%s:%s/%s",
|
||||
container.getHost(),
|
||||
container.getFirstMappedPort(),
|
||||
config.get(JdbcUtils.DATABASE_KEY).asText()),
|
||||
SQLDialect.MYSQL)) {
|
||||
final Database database = new Database(dslContext);
|
||||
|
||||
database.query(ctx -> {
|
||||
ctx.fetch("CREATE TABLE id_and_name(id INTEGER, name VARCHAR(200));");
|
||||
ctx.fetch(
|
||||
"INSERT INTO id_and_name (id, name) VALUES (1,'picard'), (2, 'crusher'), (3, 'vash');");
|
||||
ctx.fetch("CREATE TABLE starships(id INTEGER, name VARCHAR(200));");
|
||||
ctx.fetch(
|
||||
"INSERT INTO starships (id, name) VALUES (1,'enterprise-d'), (2, 'defiant'), (3, 'yamato');");
|
||||
return null;
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -4,26 +4,10 @@
|
||||
|
||||
package io.airbyte.integrations.io.airbyte.integration_tests.sources;
|
||||
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
|
||||
import io.airbyte.commons.features.EnvVariableFeatureFlags;
|
||||
import java.nio.file.Path;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import uk.org.webcompere.systemstubs.environment.EnvironmentVariables;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStub;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStubsExtension;
|
||||
|
||||
@ExtendWith(SystemStubsExtension.class)
|
||||
public class SshKeyMySqlSourceAcceptanceTest extends AbstractSshMySqlSourceAcceptanceTest {
|
||||
|
||||
@SystemStub
|
||||
private EnvironmentVariables environmentVariables;
|
||||
|
||||
@Override
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) throws Exception {
|
||||
environmentVariables.set(EnvVariableFeatureFlags.USE_STREAM_CAPABLE_STATE, "true");
|
||||
super.setupEnvironment(environment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path getConfigFilePath() {
|
||||
return Path.of("secrets/ssh-key-repl-config.json");
|
||||
|
||||
@@ -10,32 +10,14 @@ 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.cdk.integrations.standardtest.source.TestDestinationEnv;
|
||||
import io.airbyte.commons.exceptions.ConfigErrorException;
|
||||
import io.airbyte.commons.features.EnvVariableFeatureFlags;
|
||||
import io.airbyte.integrations.source.mysql.MySQLTestDatabase;
|
||||
import io.airbyte.integrations.source.mysql.MySqlSource;
|
||||
import java.nio.file.Path;
|
||||
import java.util.List;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
import org.testcontainers.containers.Network;
|
||||
import uk.org.webcompere.systemstubs.environment.EnvironmentVariables;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStub;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStubsExtension;
|
||||
|
||||
@ExtendWith(SystemStubsExtension.class)
|
||||
public class SshPasswordMySqlSourceAcceptanceTest extends AbstractSshMySqlSourceAcceptanceTest {
|
||||
|
||||
@SystemStub
|
||||
private EnvironmentVariables environmentVariables;
|
||||
|
||||
@Override
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) throws Exception {
|
||||
environmentVariables.set(EnvVariableFeatureFlags.USE_STREAM_CAPABLE_STATE, "true");
|
||||
super.setupEnvironment(environment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path getConfigFilePath() {
|
||||
return Path.of("secrets/ssh-pwd-repl-config.json");
|
||||
@@ -43,30 +25,23 @@ public class SshPasswordMySqlSourceAcceptanceTest extends AbstractSshMySqlSource
|
||||
|
||||
@Test
|
||||
public void sshTimeoutExceptionMarkAsConfigErrorTest() throws Exception {
|
||||
final SshBastionContainer bastion = new SshBastionContainer();
|
||||
final Network network = Network.newNetwork();
|
||||
// set up env
|
||||
final MySQLContainer<?> db = startTestContainers(bastion, network);
|
||||
config = bastion.getTunnelConfig(SshTunnel.TunnelMethod.SSH_PASSWORD_AUTH, bastion.getBasicDbConfigBuider(db, List.of("public")), true);
|
||||
bastion.stopAndClose();
|
||||
final Source sshWrappedSource = MySqlSource.sshWrappedSource();
|
||||
final Exception exception = assertThrows(ConfigErrorException.class, () -> sshWrappedSource.discover(config));
|
||||
try (final var testdb = MySQLTestDatabase.in("mysql:8.0", "withNetwork")) {
|
||||
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 String expectedMessage = "Timed out while opening a SSH Tunnel. Please double check the given SSH configurations and try again.";
|
||||
final String actualMessage = exception.getMessage();
|
||||
final Source sshWrappedSource = MySqlSource.sshWrappedSource(new MySqlSource());
|
||||
final Exception exception = assertThrows(ConfigErrorException.class, () -> sshWrappedSource.discover(config));
|
||||
|
||||
assertTrue(actualMessage.contains(expectedMessage));
|
||||
}
|
||||
|
||||
private MySQLContainer startTestContainers(final SshBastionContainer bastion, final Network network) {
|
||||
bastion.initAndStartBastion(network);
|
||||
return initAndStartJdbcContainer(network);
|
||||
}
|
||||
|
||||
private MySQLContainer initAndStartJdbcContainer(final Network network) {
|
||||
final MySQLContainer<?> db = new MySQLContainer<>("mysql:8.0").withNetwork(network);
|
||||
db.start();
|
||||
return db;
|
||||
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));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -1,11 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.io.airbyte.integration_tests.sources.utils;
|
||||
|
||||
public class TestConstants {
|
||||
|
||||
public static final int INITIAL_CDC_WAITING_SECONDS = 10;
|
||||
|
||||
}
|
||||
@@ -13,7 +13,6 @@ import static io.airbyte.cdk.integrations.debezium.internals.mysql.MysqlCdcState
|
||||
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 static io.airbyte.integrations.source.mysql.MySqlSource.DRIVER_CLASS;
|
||||
import static io.airbyte.integrations.source.mysql.initialsync.MySqlInitialLoadStateManager.PRIMARY_KEY_STATE_TYPE;
|
||||
import static io.airbyte.integrations.source.mysql.initialsync.MySqlInitialLoadStateManager.STATE_TYPE_KEY;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
@@ -30,16 +29,13 @@ import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Streams;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DataSourceFactory;
|
||||
import io.airbyte.cdk.db.jdbc.DefaultJdbcDatabase;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcDatabase;
|
||||
import io.airbyte.cdk.integrations.base.Source;
|
||||
import io.airbyte.cdk.integrations.debezium.CdcSourceTest;
|
||||
import io.airbyte.cdk.integrations.debezium.internals.AirbyteSchemaHistoryStorage;
|
||||
import io.airbyte.cdk.integrations.debezium.internals.mysql.MySqlCdcTargetPosition;
|
||||
import io.airbyte.commons.features.EnvVariableFeatureFlags;
|
||||
import io.airbyte.commons.features.FeatureFlagsWrapper;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.commons.util.AutoCloseableIterator;
|
||||
import io.airbyte.commons.util.AutoCloseableIterators;
|
||||
@@ -59,127 +55,68 @@ 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.sql.SQLException;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import javax.sql.DataSource;
|
||||
import org.jooq.SQLDialect;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Tag;
|
||||
import org.junit.jupiter.api.Tags;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.TestInfo;
|
||||
import org.junit.jupiter.api.Timeout;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
import uk.org.webcompere.systemstubs.environment.EnvironmentVariables;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStub;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStubsExtension;
|
||||
|
||||
@ExtendWith(SystemStubsExtension.class)
|
||||
public class CdcMysqlSourceTest extends CdcSourceTest {
|
||||
@Order(1)
|
||||
public class CdcMysqlSourceTest extends CdcSourceTest<MySqlSource, MySQLTestDatabase> {
|
||||
|
||||
private static final String START_DB_CONTAINER_WITH_INVALID_TIMEZONE = "START-DB-CONTAINER-WITH-INVALID-TIMEZONE";
|
||||
private static final String INVALID_TIMEZONE_CEST = "CEST";
|
||||
|
||||
@SystemStub
|
||||
private EnvironmentVariables environmentVariables;
|
||||
|
||||
private static final String DB_NAME = MODELS_SCHEMA;
|
||||
private MySQLContainer<?> container;
|
||||
private Database database;
|
||||
private MySqlSource source;
|
||||
private JsonNode config;
|
||||
private static final Random RANDOM = new Random();
|
||||
|
||||
@BeforeEach
|
||||
public void setup(final TestInfo testInfo) throws SQLException {
|
||||
environmentVariables.set(EnvVariableFeatureFlags.USE_STREAM_CAPABLE_STATE, "true");
|
||||
init(testInfo);
|
||||
revokeAllPermissions();
|
||||
grantCorrectPermissions();
|
||||
super.setup();
|
||||
@Override
|
||||
protected MySQLTestDatabase createTestDatabase() {
|
||||
return MySQLTestDatabase.in("mysql:8.0", "withInvalidTimezoneCEST").withCdcPermissions();
|
||||
}
|
||||
|
||||
private void init(final TestInfo testInfo) {
|
||||
container = new MySQLContainer<>("mysql:8.0");
|
||||
if (testInfo.getTags().contains(START_DB_CONTAINER_WITH_INVALID_TIMEZONE)) {
|
||||
container.withEnv(Map.of("TZ", INVALID_TIMEZONE_CEST));
|
||||
}
|
||||
container.start();
|
||||
source = new MySqlSource();
|
||||
database = new Database(DSLContextFactory.create(
|
||||
"root",
|
||||
"test",
|
||||
DRIVER_CLASS,
|
||||
String.format("jdbc:mysql://%s:%s",
|
||||
container.getHost(),
|
||||
container.getFirstMappedPort()),
|
||||
SQLDialect.MYSQL));
|
||||
|
||||
final JsonNode replicationMethod = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("method", "CDC")
|
||||
.put("initial_waiting_seconds", INITIAL_WAITING_SECONDS)
|
||||
.put("server_time_zone", "America/Los_Angeles")
|
||||
.build());
|
||||
|
||||
config = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("host", container.getHost())
|
||||
.put("port", container.getFirstMappedPort())
|
||||
.put("database", DB_NAME)
|
||||
.put("username", container.getUsername())
|
||||
.put("password", container.getPassword())
|
||||
.put("replication_method", replicationMethod)
|
||||
.put(SYNC_CHECKPOINT_RECORDS_PROPERTY, 1)
|
||||
.put("is_test", true)
|
||||
.build());
|
||||
@Override
|
||||
protected MySqlSource source() {
|
||||
final var source = new MySqlSource();
|
||||
source.setFeatureFlags(FeatureFlagsWrapper.overridingUseStreamCapableState(new EnvVariableFeatureFlags(), true));
|
||||
return source;
|
||||
}
|
||||
|
||||
private void revokeAllPermissions() {
|
||||
executeQuery("REVOKE ALL PRIVILEGES, GRANT OPTION FROM " + container.getUsername() + "@'%';");
|
||||
}
|
||||
|
||||
private void revokeReplicationClientPermission() {
|
||||
executeQuery("REVOKE REPLICATION CLIENT ON *.* FROM " + container.getUsername() + "@'%';");
|
||||
}
|
||||
|
||||
private void grantCorrectPermissions() {
|
||||
executeQuery("GRANT SELECT, RELOAD, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT ON *.* TO " + container.getUsername() + "@'%';");
|
||||
@Override
|
||||
protected JsonNode config() {
|
||||
return testdb.testConfigBuilder()
|
||||
.withCdcReplication()
|
||||
.with(SYNC_CHECKPOINT_RECORDS_PROPERTY, 1)
|
||||
.build();
|
||||
}
|
||||
|
||||
protected void purgeAllBinaryLogs() {
|
||||
executeQuery("RESET MASTER;");
|
||||
testdb.with("RESET MASTER;");
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() {
|
||||
try {
|
||||
container.close();
|
||||
} catch (final Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
@Override
|
||||
protected String createSchemaSqlFmt() {
|
||||
return "CREATE DATABASE IF NOT EXISTS %s;";
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String modelsSchema() {
|
||||
return testdb.getDatabaseName();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String randomSchema() {
|
||||
return testdb.getDatabaseName();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected MySqlCdcTargetPosition cdcLatestTargetPosition() {
|
||||
final DataSource dataSource = DataSourceFactory.create(
|
||||
"root",
|
||||
"test",
|
||||
DRIVER_CLASS,
|
||||
String.format("jdbc:mysql://%s:%s",
|
||||
container.getHost(),
|
||||
container.getFirstMappedPort()),
|
||||
Collections.emptyMap());
|
||||
return MySqlCdcTargetPosition.targetPosition(new DefaultJdbcDatabase(dataSource));
|
||||
return MySqlCdcTargetPosition.targetPosition(new DefaultJdbcDatabase(testdb.getDataSource()));
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -240,30 +177,10 @@ public class CdcMysqlSourceTest extends CdcSourceTest {
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Source getSource() {
|
||||
return source;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected JsonNode getConfig() {
|
||||
return config;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Database getDatabase() {
|
||||
return database;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String randomTableSchema() {
|
||||
return MODELS_SCHEMA;
|
||||
}
|
||||
|
||||
@Test
|
||||
protected void syncWithReplicationClientPrivilegeRevokedFailsCheck() throws Exception {
|
||||
revokeReplicationClientPermission();
|
||||
final AirbyteConnectionStatus status = getSource().check(getConfig());
|
||||
testdb.with("REVOKE REPLICATION CLIENT ON *.* FROM %s@'%%';", testdb.getUserName());
|
||||
final AirbyteConnectionStatus status = source().check(config());
|
||||
final String expectedErrorMessage = "Please grant REPLICATION CLIENT privilege, so that binary log files are available"
|
||||
+ " for CDC mode.";
|
||||
assertTrue(status.getStatus().equals(Status.FAILED));
|
||||
@@ -283,8 +200,8 @@ public class CdcMysqlSourceTest extends CdcSourceTest {
|
||||
writeModelRecord(record);
|
||||
}
|
||||
|
||||
final AutoCloseableIterator<AirbyteMessage> firstBatchIterator = getSource()
|
||||
.read(getConfig(), CONFIGURED_CATALOG, null);
|
||||
final AutoCloseableIterator<AirbyteMessage> firstBatchIterator = source()
|
||||
.read(config(), getConfiguredCatalog(), null);
|
||||
final List<AirbyteMessage> dataFromFirstBatch = AutoCloseableIterators
|
||||
.toListAndClose(firstBatchIterator);
|
||||
final List<AirbyteStateMessage> stateAfterFirstBatch = extractStateMessages(dataFromFirstBatch);
|
||||
@@ -314,8 +231,8 @@ public class CdcMysqlSourceTest extends CdcSourceTest {
|
||||
purgeAllBinaryLogs();
|
||||
|
||||
final JsonNode state = Jsons.jsonNode(Collections.singletonList(stateAfterFirstBatch.get(stateAfterFirstBatch.size() - 1)));
|
||||
final AutoCloseableIterator<AirbyteMessage> secondBatchIterator = getSource()
|
||||
.read(getConfig(), CONFIGURED_CATALOG, state);
|
||||
final AutoCloseableIterator<AirbyteMessage> secondBatchIterator = source()
|
||||
.read(config(), getConfiguredCatalog(), state);
|
||||
final List<AirbyteMessage> dataFromSecondBatch = AutoCloseableIterators
|
||||
.toListAndClose(secondBatchIterator);
|
||||
|
||||
@@ -338,10 +255,10 @@ public class CdcMysqlSourceTest extends CdcSourceTest {
|
||||
@Test
|
||||
protected void verifyCheckpointStatesByRecords() throws Exception {
|
||||
// We require a huge amount of records, otherwise Debezium will notify directly the last offset.
|
||||
final int recordsToCreate = 20000;
|
||||
final int recordsToCreate = 20_000;
|
||||
|
||||
final AutoCloseableIterator<AirbyteMessage> firstBatchIterator = getSource()
|
||||
.read(getConfig(), CONFIGURED_CATALOG, null);
|
||||
final AutoCloseableIterator<AirbyteMessage> firstBatchIterator = source()
|
||||
.read(config(), getConfiguredCatalog(), null);
|
||||
final List<AirbyteMessage> dataFromFirstBatch = AutoCloseableIterators
|
||||
.toListAndClose(firstBatchIterator);
|
||||
final List<AirbyteStateMessage> stateMessages = extractStateMessages(dataFromFirstBatch);
|
||||
@@ -351,16 +268,14 @@ public class CdcMysqlSourceTest extends CdcSourceTest {
|
||||
assertExpectedStateMessages(stateMessages);
|
||||
|
||||
for (int recordsCreated = 0; recordsCreated < recordsToCreate; recordsCreated++) {
|
||||
final JsonNode record =
|
||||
Jsons.jsonNode(ImmutableMap
|
||||
.of(COL_ID, 200 + recordsCreated, COL_MAKE_ID, 1, COL_MODEL,
|
||||
"F-" + recordsCreated));
|
||||
final JsonNode record = Jsons.jsonNode(ImmutableMap
|
||||
.of(COL_ID, 200 + recordsCreated, COL_MAKE_ID, 1, COL_MODEL, "F-" + recordsCreated));
|
||||
writeModelRecord(record);
|
||||
}
|
||||
|
||||
final JsonNode stateAfterFirstSync = Jsons.jsonNode(Collections.singletonList(stateMessages.get(stateMessages.size() - 1)));
|
||||
final AutoCloseableIterator<AirbyteMessage> secondBatchIterator = getSource()
|
||||
.read(getConfig(), CONFIGURED_CATALOG, stateAfterFirstSync);
|
||||
final AutoCloseableIterator<AirbyteMessage> secondBatchIterator = source()
|
||||
.read(config(), getConfiguredCatalog(), stateAfterFirstSync);
|
||||
final List<AirbyteMessage> dataFromSecondBatch = AutoCloseableIterators
|
||||
.toListAndClose(secondBatchIterator);
|
||||
assertEquals(recordsToCreate, extractRecordMessages(dataFromSecondBatch).size());
|
||||
@@ -449,14 +364,14 @@ public class CdcMysqlSourceTest extends CdcSourceTest {
|
||||
.collect(Collectors.toSet());
|
||||
assertEquals(2, streamsInSnapshotState.size());
|
||||
assertTrue(
|
||||
streamsInSnapshotState.contains(new StreamDescriptor().withName(MODELS_STREAM_NAME + "_random").withNamespace(randomTableSchema())));
|
||||
assertTrue(streamsInSnapshotState.contains(new StreamDescriptor().withName(MODELS_STREAM_NAME).withNamespace(MODELS_SCHEMA)));
|
||||
streamsInSnapshotState.contains(new StreamDescriptor().withName(MODELS_STREAM_NAME + "_random").withNamespace(randomSchema())));
|
||||
assertTrue(streamsInSnapshotState.contains(new StreamDescriptor().withName(MODELS_STREAM_NAME).withNamespace(testdb.getDatabaseName())));
|
||||
|
||||
stateMessage.getGlobal().getStreamStates().forEach(s -> {
|
||||
final JsonNode streamState = s.getStreamState();
|
||||
if (s.getStreamDescriptor().equals(new StreamDescriptor().withName(MODELS_STREAM_NAME + "_random").withNamespace(randomTableSchema()))) {
|
||||
if (s.getStreamDescriptor().equals(new StreamDescriptor().withName(MODELS_STREAM_NAME + "_random").withNamespace(randomSchema()))) {
|
||||
assertEquals(PRIMARY_KEY_STATE_TYPE, streamState.get(STATE_TYPE_KEY).asText());
|
||||
} else if (s.getStreamDescriptor().equals(new StreamDescriptor().withName(MODELS_STREAM_NAME).withNamespace(MODELS_SCHEMA))) {
|
||||
} else if (s.getStreamDescriptor().equals(new StreamDescriptor().withName(MODELS_STREAM_NAME).withNamespace(testdb.getDatabaseName()))) {
|
||||
assertFalse(streamState.has(STATE_TYPE_KEY));
|
||||
} else {
|
||||
throw new RuntimeException("Unknown stream");
|
||||
@@ -474,8 +389,8 @@ public class CdcMysqlSourceTest extends CdcSourceTest {
|
||||
.collect(Collectors.toSet());
|
||||
assertEquals(2, streamsInSnapshotState.size());
|
||||
assertTrue(
|
||||
streamsInSnapshotState.contains(new StreamDescriptor().withName(MODELS_STREAM_NAME + "_random").withNamespace(randomTableSchema())));
|
||||
assertTrue(streamsInSnapshotState.contains(new StreamDescriptor().withName(MODELS_STREAM_NAME).withNamespace(MODELS_SCHEMA)));
|
||||
streamsInSnapshotState.contains(new StreamDescriptor().withName(MODELS_STREAM_NAME + "_random").withNamespace(randomSchema())));
|
||||
assertTrue(streamsInSnapshotState.contains(new StreamDescriptor().withName(MODELS_STREAM_NAME).withNamespace(testdb.getDatabaseName())));
|
||||
secondLastSateMessage.getGlobal().getStreamStates().forEach(s -> {
|
||||
final JsonNode streamState = s.getStreamState();
|
||||
assertFalse(streamState.has(STATE_TYPE_KEY));
|
||||
@@ -492,17 +407,16 @@ public class CdcMysqlSourceTest extends CdcSourceTest {
|
||||
assertEquals(2, streamsInSnapshotState.size());
|
||||
assertTrue(
|
||||
streamsInSyncCompletionState.contains(
|
||||
new StreamDescriptor().withName(MODELS_STREAM_NAME + "_random").withNamespace(randomTableSchema())));
|
||||
assertTrue(streamsInSyncCompletionState.contains(new StreamDescriptor().withName(MODELS_STREAM_NAME).withNamespace(MODELS_SCHEMA)));
|
||||
new StreamDescriptor().withName(MODELS_STREAM_NAME + "_random").withNamespace(randomSchema())));
|
||||
assertTrue(streamsInSyncCompletionState.contains(new StreamDescriptor().withName(MODELS_STREAM_NAME).withNamespace(testdb.getDatabaseName())));
|
||||
assertNotNull(stateMessageEmittedAfterSecondSyncCompletion.getData());
|
||||
}
|
||||
|
||||
@Test
|
||||
@Timeout(value = 60)
|
||||
@Tags(value = {@Tag(START_DB_CONTAINER_WITH_INVALID_TIMEZONE)})
|
||||
public void syncWouldWorkWithDBWithInvalidTimezone() throws Exception {
|
||||
final String systemTimeZone = "@@system_time_zone";
|
||||
final JdbcDatabase jdbcDatabase = ((MySqlSource) getSource()).createDatabase(getConfig());
|
||||
final JdbcDatabase jdbcDatabase = source().createDatabase(config());
|
||||
final Properties properties = MySqlCdcProperties.getDebeziumProperties(jdbcDatabase);
|
||||
final String databaseTimezone = jdbcDatabase.unsafeQuery(String.format("SELECT %s;", systemTimeZone)).toList().get(0).get(systemTimeZone)
|
||||
.asText();
|
||||
@@ -511,8 +425,8 @@ public class CdcMysqlSourceTest extends CdcSourceTest {
|
||||
assertEquals(INVALID_TIMEZONE_CEST, databaseTimezone);
|
||||
assertEquals("America/Los_Angeles", debeziumEngineTimezone);
|
||||
|
||||
final AutoCloseableIterator<AirbyteMessage> read = getSource()
|
||||
.read(getConfig(), CONFIGURED_CATALOG, null);
|
||||
final AutoCloseableIterator<AirbyteMessage> read = source()
|
||||
.read(config(), getConfiguredCatalog(), null);
|
||||
|
||||
final List<AirbyteMessage> actualRecords = AutoCloseableIterators.toListAndClose(read);
|
||||
|
||||
@@ -526,12 +440,12 @@ public class CdcMysqlSourceTest extends CdcSourceTest {
|
||||
@Test
|
||||
public void testCompositeIndexInitialLoad() throws Exception {
|
||||
// Simulate adding a composite index by modifying the catalog.
|
||||
final ConfiguredAirbyteCatalog configuredCatalog = Jsons.clone(CONFIGURED_CATALOG);
|
||||
final ConfiguredAirbyteCatalog configuredCatalog = Jsons.clone(getConfiguredCatalog());
|
||||
final List<List<String>> primaryKeys = configuredCatalog.getStreams().get(0).getStream().getSourceDefinedPrimaryKey();
|
||||
primaryKeys.add(List.of("make_id"));
|
||||
|
||||
final AutoCloseableIterator<AirbyteMessage> read1 = getSource()
|
||||
.read(getConfig(), configuredCatalog, null);
|
||||
final AutoCloseableIterator<AirbyteMessage> read1 = source()
|
||||
.read(config(), configuredCatalog, null);
|
||||
|
||||
final List<AirbyteMessage> actualRecords1 = AutoCloseableIterators.toListAndClose(read1);
|
||||
|
||||
@@ -546,8 +460,8 @@ public class CdcMysqlSourceTest extends CdcSourceTest {
|
||||
// load, and
|
||||
// the last one indicating the cdc position we have synced until.
|
||||
final JsonNode state = Jsons.jsonNode(Collections.singletonList(stateMessages1.get(4)));
|
||||
final AutoCloseableIterator<AirbyteMessage> read2 = getSource()
|
||||
.read(getConfig(), configuredCatalog, state);
|
||||
final AutoCloseableIterator<AirbyteMessage> read2 = source()
|
||||
.read(config(), configuredCatalog, state);
|
||||
|
||||
final List<AirbyteMessage> actualRecords2 = AutoCloseableIterators.toListAndClose(read2);
|
||||
final Set<AirbyteRecordMessage> recordMessages2 = extractRecordMessages(actualRecords2);
|
||||
@@ -561,7 +475,7 @@ public class CdcMysqlSourceTest extends CdcSourceTest {
|
||||
@Test
|
||||
public void testTwoStreamSync() throws Exception {
|
||||
// Add another stream models_2 and read that one as well.
|
||||
final ConfiguredAirbyteCatalog configuredCatalog = Jsons.clone(CONFIGURED_CATALOG);
|
||||
final ConfiguredAirbyteCatalog configuredCatalog = Jsons.clone(getConfiguredCatalog());
|
||||
|
||||
final List<JsonNode> MODEL_RECORDS_2 = ImmutableList.of(
|
||||
Jsons.jsonNode(ImmutableMap.of(COL_ID, 110, COL_MAKE_ID, 1, COL_MODEL, "Fiesta-2")),
|
||||
@@ -571,18 +485,18 @@ public class CdcMysqlSourceTest extends CdcSourceTest {
|
||||
Jsons.jsonNode(ImmutableMap.of(COL_ID, 150, COL_MAKE_ID, 2, COL_MODEL, "A 220-2")),
|
||||
Jsons.jsonNode(ImmutableMap.of(COL_ID, 160, COL_MAKE_ID, 2, COL_MODEL, "E 350-2")));
|
||||
|
||||
createTable(MODELS_SCHEMA, MODELS_STREAM_NAME + "_2",
|
||||
testdb.with(createTableSqlFmt(), testdb.getDatabaseName(), MODELS_STREAM_NAME + "_2",
|
||||
columnClause(ImmutableMap.of(COL_ID, "INTEGER", COL_MAKE_ID, "INTEGER", COL_MODEL, "VARCHAR(200)"), Optional.of(COL_ID)));
|
||||
|
||||
for (final JsonNode recordJson : MODEL_RECORDS_2) {
|
||||
writeRecords(recordJson, MODELS_SCHEMA, MODELS_STREAM_NAME + "_2", COL_ID,
|
||||
writeRecords(recordJson, testdb.getDatabaseName(), MODELS_STREAM_NAME + "_2", COL_ID,
|
||||
COL_MAKE_ID, COL_MODEL);
|
||||
}
|
||||
|
||||
final ConfiguredAirbyteStream airbyteStream = new ConfiguredAirbyteStream()
|
||||
.withStream(CatalogHelpers.createAirbyteStream(
|
||||
MODELS_STREAM_NAME + "_2",
|
||||
MODELS_SCHEMA,
|
||||
testdb.getDatabaseName(),
|
||||
Field.of(COL_ID, JsonSchemaType.INTEGER),
|
||||
Field.of(COL_MAKE_ID, JsonSchemaType.INTEGER),
|
||||
Field.of(COL_MODEL, JsonSchemaType.STRING))
|
||||
@@ -595,8 +509,8 @@ public class CdcMysqlSourceTest extends CdcSourceTest {
|
||||
streams.add(airbyteStream);
|
||||
configuredCatalog.withStreams(streams);
|
||||
|
||||
final AutoCloseableIterator<AirbyteMessage> read1 = getSource()
|
||||
.read(getConfig(), configuredCatalog, null);
|
||||
final AutoCloseableIterator<AirbyteMessage> read1 = source()
|
||||
.read(config(), configuredCatalog, null);
|
||||
final List<AirbyteMessage> actualRecords1 = AutoCloseableIterators.toListAndClose(read1);
|
||||
|
||||
final Set<AirbyteRecordMessage> recordMessages1 = extractRecordMessages(actualRecords1);
|
||||
@@ -658,13 +572,13 @@ public class CdcMysqlSourceTest extends CdcSourceTest {
|
||||
recordMessages1,
|
||||
names,
|
||||
names,
|
||||
MODELS_SCHEMA);
|
||||
testdb.getDatabaseName());
|
||||
|
||||
assertEquals(new StreamDescriptor().withName(MODELS_STREAM_NAME).withNamespace(MODELS_SCHEMA), firstStreamInState);
|
||||
assertEquals(new StreamDescriptor().withName(MODELS_STREAM_NAME).withNamespace(testdb.getDatabaseName()), firstStreamInState);
|
||||
|
||||
// Triggering a sync with a primary_key state for 1 stream and complete state for other stream
|
||||
final AutoCloseableIterator<AirbyteMessage> read2 = getSource()
|
||||
.read(getConfig(), configuredCatalog, Jsons.jsonNode(Collections.singletonList(stateMessages1.get(6))));
|
||||
final AutoCloseableIterator<AirbyteMessage> read2 = source()
|
||||
.read(config(), configuredCatalog, Jsons.jsonNode(Collections.singletonList(stateMessages1.get(6))));
|
||||
final List<AirbyteMessage> actualRecords2 = AutoCloseableIterators.toListAndClose(read2);
|
||||
|
||||
final List<AirbyteStateMessage> stateMessages2 = extractStateMessages(actualRecords2);
|
||||
@@ -701,7 +615,7 @@ public class CdcMysqlSourceTest extends CdcSourceTest {
|
||||
recordMessages2,
|
||||
names,
|
||||
names,
|
||||
MODELS_SCHEMA);
|
||||
testdb.getDatabaseName());
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -714,8 +628,8 @@ public class CdcMysqlSourceTest extends CdcSourceTest {
|
||||
@Test
|
||||
public void testCompressedSchemaHistory() throws Exception {
|
||||
createTablesToIncreaseSchemaHistorySize();
|
||||
final AutoCloseableIterator<AirbyteMessage> firstBatchIterator = getSource()
|
||||
.read(getConfig(), CONFIGURED_CATALOG, null);
|
||||
final AutoCloseableIterator<AirbyteMessage> firstBatchIterator = source()
|
||||
.read(config(), getConfiguredCatalog(), null);
|
||||
final List<AirbyteMessage> dataFromFirstBatch = AutoCloseableIterators
|
||||
.toListAndClose(firstBatchIterator);
|
||||
final AirbyteStateMessage lastStateMessageFromFirstBatch = Iterables.getLast(extractStateMessages(dataFromFirstBatch));
|
||||
@@ -737,8 +651,8 @@ public class CdcMysqlSourceTest extends CdcSourceTest {
|
||||
writeModelRecord(record);
|
||||
}
|
||||
|
||||
final AutoCloseableIterator<AirbyteMessage> secondBatchIterator = getSource()
|
||||
.read(getConfig(), CONFIGURED_CATALOG, Jsons.jsonNode(Collections.singletonList(lastStateMessageFromFirstBatch)));
|
||||
final AutoCloseableIterator<AirbyteMessage> secondBatchIterator = source()
|
||||
.read(config(), getConfiguredCatalog(), Jsons.jsonNode(Collections.singletonList(lastStateMessageFromFirstBatch)));
|
||||
final List<AirbyteMessage> dataFromSecondBatch = AutoCloseableIterators
|
||||
.toListAndClose(secondBatchIterator);
|
||||
final AirbyteStateMessage lastStateMessageFromSecondBatch = Iterables.getLast(extractStateMessages(dataFromSecondBatch));
|
||||
@@ -758,7 +672,7 @@ public class CdcMysqlSourceTest extends CdcSourceTest {
|
||||
private void createTablesToIncreaseSchemaHistorySize() {
|
||||
for (int i = 0; i <= 200; i++) {
|
||||
final String tableName = generateRandomStringOf32Characters();
|
||||
final StringBuilder createTableQuery = new StringBuilder("CREATE TABLE models_schema." + tableName + "(");
|
||||
final StringBuilder createTableQuery = new StringBuilder("CREATE TABLE " + tableName + "(");
|
||||
String firstCol = null;
|
||||
for (int j = 1; j <= 250; j++) {
|
||||
final String columnName = generateRandomStringOf32Characters();
|
||||
@@ -769,7 +683,7 @@ public class CdcMysqlSourceTest extends CdcSourceTest {
|
||||
createTableQuery.append(columnName).append(" INTEGER, ");
|
||||
}
|
||||
createTableQuery.append("PRIMARY KEY (").append(firstCol).append("));");
|
||||
executeQuery(createTableQuery.toString());
|
||||
testdb.with(createTableQuery.toString());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -8,6 +8,7 @@ package io.airbyte.integrations.source.mysql;
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
import static io.airbyte.cdk.integrations.debezium.DebeziumIteratorConstants.SYNC_CHECKPOINT_RECORDS_PROPERTY;
|
||||
import static io.airbyte.integrations.source.mysql.initialsync.MySqlInitialLoadStateManager.STATE_TYPE_KEY;
|
||||
import static java.util.stream.Collectors.toList;
|
||||
import static org.assertj.core.api.Assertions.assertThat;
|
||||
@@ -19,18 +20,13 @@ import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.mysql.cj.MysqlType;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.source.jdbc.AbstractJdbcSource;
|
||||
import io.airbyte.cdk.integrations.source.jdbc.test.JdbcSourceAcceptanceTest;
|
||||
import io.airbyte.cdk.integrations.source.relationaldb.models.DbStreamState;
|
||||
import io.airbyte.commons.features.EnvVariableFeatureFlags;
|
||||
import io.airbyte.commons.features.FeatureFlagsWrapper;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.commons.resources.MoreResources;
|
||||
import io.airbyte.commons.string.Strings;
|
||||
import io.airbyte.commons.util.MoreIterators;
|
||||
import io.airbyte.integrations.source.mysql.internal.models.CursorBasedStatus;
|
||||
import io.airbyte.integrations.source.mysql.internal.models.InternalModels.StateType;
|
||||
@@ -52,187 +48,102 @@ import io.airbyte.protocol.models.v0.ConnectorSpecification;
|
||||
import io.airbyte.protocol.models.v0.DestinationSyncMode;
|
||||
import io.airbyte.protocol.models.v0.StreamDescriptor;
|
||||
import io.airbyte.protocol.models.v0.SyncMode;
|
||||
import java.sql.Connection;
|
||||
import java.sql.DriverManager;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.jooq.DSLContext;
|
||||
import org.jooq.SQLDialect;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
import uk.org.webcompere.systemstubs.environment.EnvironmentVariables;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStub;
|
||||
import uk.org.webcompere.systemstubs.jupiter.SystemStubsExtension;
|
||||
|
||||
@ExtendWith(SystemStubsExtension.class)
|
||||
class MySqlJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest {
|
||||
|
||||
@SystemStub
|
||||
private EnvironmentVariables environmentVariables;
|
||||
@Order(2)
|
||||
class MySqlJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest<MySqlSource, MySQLTestDatabase> {
|
||||
|
||||
protected static final String USERNAME_WITHOUT_PERMISSION = "new_user";
|
||||
protected static final String PASSWORD_WITHOUT_PERMISSION = "new_password";
|
||||
protected static final String TEST_USER = "test";
|
||||
protected static final Callable<String> TEST_PASSWORD = () -> "test";
|
||||
protected static MySQLContainer<?> container;
|
||||
|
||||
protected Database database;
|
||||
protected DSLContext dslContext;
|
||||
|
||||
@BeforeAll
|
||||
static void init() throws Exception {
|
||||
container = new MySQLContainer<>("mysql:8.0")
|
||||
.withUsername(TEST_USER)
|
||||
.withPassword(TEST_PASSWORD.call())
|
||||
.withEnv("MYSQL_ROOT_HOST", "%")
|
||||
.withEnv("MYSQL_ROOT_PASSWORD", TEST_PASSWORD.call());
|
||||
container.start();
|
||||
final Connection connection = DriverManager.getConnection(container.getJdbcUrl(), "root", TEST_PASSWORD.call());
|
||||
connection.createStatement().execute("GRANT ALL PRIVILEGES ON *.* TO '" + TEST_USER + "'@'%';\n");
|
||||
@Override
|
||||
protected JsonNode config() {
|
||||
return testdb.testConfigBuilder().build();
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
public void setup() throws Exception {
|
||||
environmentVariables.set(EnvVariableFeatureFlags.USE_STREAM_CAPABLE_STATE, "true");
|
||||
config = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, container.getHost())
|
||||
.put(JdbcUtils.PORT_KEY, container.getFirstMappedPort())
|
||||
.put(JdbcUtils.DATABASE_KEY, Strings.addRandomSuffix("db", "_", 10))
|
||||
.put(JdbcUtils.USERNAME_KEY, TEST_USER)
|
||||
.put(JdbcUtils.PASSWORD_KEY, TEST_PASSWORD.call())
|
||||
.build());
|
||||
|
||||
dslContext = DSLContextFactory.create(
|
||||
config.get(JdbcUtils.USERNAME_KEY).asText(),
|
||||
config.get(JdbcUtils.PASSWORD_KEY).asText(),
|
||||
DatabaseDriver.MYSQL.getDriverClassName(),
|
||||
String.format("jdbc:mysql://%s:%s",
|
||||
config.get(JdbcUtils.HOST_KEY).asText(),
|
||||
config.get(JdbcUtils.PORT_KEY).asText()),
|
||||
SQLDialect.MYSQL);
|
||||
database = new Database(dslContext);
|
||||
|
||||
database.query(ctx -> {
|
||||
ctx.fetch("CREATE DATABASE " + getDefaultNamespace());
|
||||
return null;
|
||||
});
|
||||
|
||||
super.setup();
|
||||
@Override
|
||||
protected MySqlSource source() {
|
||||
final var source = new MySqlSource();
|
||||
source.setFeatureFlags(FeatureFlagsWrapper.overridingUseStreamCapableState(new EnvVariableFeatureFlags(), true));
|
||||
return source;
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
void tearDownMySql() throws Exception {
|
||||
dslContext.close();
|
||||
super.tearDown();
|
||||
@Override
|
||||
protected MySQLTestDatabase createTestDatabase() {
|
||||
return MySQLTestDatabase.in("mysql:8.0");
|
||||
}
|
||||
|
||||
@AfterAll
|
||||
static void cleanUp() {
|
||||
container.close();
|
||||
@Override
|
||||
protected void maybeSetShorterConnectionTimeout(final JsonNode config) {
|
||||
((ObjectNode) config).put(JdbcUtils.JDBC_URL_PARAMS_KEY, "connectTimeout=1000");
|
||||
}
|
||||
|
||||
// MySql does not support schemas in the way most dbs do. Instead we namespace by db name.
|
||||
@Override
|
||||
public boolean supportsSchemas() {
|
||||
protected boolean supportsSchemas() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractJdbcSource<MysqlType> getJdbcSource() {
|
||||
return new MySqlSource();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDriverClass() {
|
||||
return MySqlSource.DRIVER_CLASS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public JsonNode getConfig() {
|
||||
return Jsons.clone(config);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testReadMultipleTablesIncrementally() throws Exception {
|
||||
((ObjectNode) config).put("sync_checkpoint_records", 1);
|
||||
final String namespace = getDefaultNamespace();
|
||||
final var config = config();
|
||||
((ObjectNode) config).put(SYNC_CHECKPOINT_RECORDS_PROPERTY, 1);
|
||||
final String streamOneName = TABLE_NAME + "one";
|
||||
// Create a fresh first table
|
||||
database.query(connection -> {
|
||||
connection.fetch(String.format("USE %s;", getDefaultNamespace()));
|
||||
connection.fetch(String.format("CREATE TABLE %s (\n"
|
||||
+ " id int PRIMARY KEY,\n"
|
||||
+ " name VARCHAR(200) NOT NULL,\n"
|
||||
+ " updated_at VARCHAR(200) NOT NULL\n"
|
||||
+ ");", streamOneName));
|
||||
connection.execute(
|
||||
String.format(
|
||||
"INSERT INTO %s(id, name, updated_at) VALUES (1,'picard', '2004-10-19')",
|
||||
getFullyQualifiedTableName(streamOneName)));
|
||||
connection.execute(
|
||||
String.format(
|
||||
"INSERT INTO %s(id, name, updated_at) VALUES (2, 'crusher', '2005-10-19')",
|
||||
getFullyQualifiedTableName(streamOneName)));
|
||||
connection.execute(
|
||||
String.format(
|
||||
"INSERT INTO %s(id, name, updated_at) VALUES (3, 'vash', '2006-10-19')",
|
||||
getFullyQualifiedTableName(streamOneName)));
|
||||
return null;
|
||||
});
|
||||
testdb.with("CREATE TABLE %s (\n"
|
||||
+ " id int PRIMARY KEY,\n"
|
||||
+ " name VARCHAR(200) NOT NULL,\n"
|
||||
+ " updated_at VARCHAR(200) NOT NULL\n"
|
||||
+ ");", streamOneName)
|
||||
.with("INSERT INTO %s(id, name, updated_at) VALUES (1,'picard', '2004-10-19')",
|
||||
getFullyQualifiedTableName(streamOneName))
|
||||
.with("INSERT INTO %s(id, name, updated_at) VALUES (2, 'crusher', '2005-10-19')",
|
||||
getFullyQualifiedTableName(streamOneName))
|
||||
.with("INSERT INTO %s(id, name, updated_at) VALUES (3, 'vash', '2006-10-19')",
|
||||
getFullyQualifiedTableName(streamOneName));
|
||||
|
||||
// Create a fresh second table
|
||||
final String streamTwoName = TABLE_NAME + "two";
|
||||
final String streamTwoFullyQualifiedName = getFullyQualifiedTableName(streamTwoName);
|
||||
// Insert records into second table
|
||||
database.query(ctx -> {
|
||||
ctx.fetch(String.format("CREATE TABLE %s (\n"
|
||||
+ " id int PRIMARY KEY,\n"
|
||||
+ " name VARCHAR(200) NOT NULL,\n"
|
||||
+ " updated_at DATE NOT NULL\n"
|
||||
+ ");", streamTwoName));
|
||||
ctx.execute(
|
||||
String.format("INSERT INTO %s(id, name, updated_at)"
|
||||
+ "VALUES (40,'Jean Luc','2006-10-19')",
|
||||
streamTwoFullyQualifiedName));
|
||||
ctx.execute(
|
||||
String.format("INSERT INTO %s(id, name, updated_at)"
|
||||
+ "VALUES (41, 'Groot', '2006-10-19')",
|
||||
streamTwoFullyQualifiedName));
|
||||
ctx.execute(
|
||||
String.format("INSERT INTO %s(id, name, updated_at)"
|
||||
+ "VALUES (42, 'Thanos','2006-10-19')",
|
||||
streamTwoFullyQualifiedName));
|
||||
return null;
|
||||
});
|
||||
testdb.with("CREATE TABLE %s (\n"
|
||||
+ " id int PRIMARY KEY,\n"
|
||||
+ " name VARCHAR(200) NOT NULL,\n"
|
||||
+ " updated_at DATE NOT NULL\n"
|
||||
+ ");", streamTwoName)
|
||||
.with("INSERT INTO %s(id, name, updated_at) VALUES (40,'Jean Luc','2006-10-19')",
|
||||
streamTwoFullyQualifiedName)
|
||||
.with("INSERT INTO %s(id, name, updated_at) VALUES (41, 'Groot', '2006-10-19')",
|
||||
streamTwoFullyQualifiedName)
|
||||
.with("INSERT INTO %s(id, name, updated_at) VALUES (42, 'Thanos','2006-10-19')",
|
||||
streamTwoFullyQualifiedName);
|
||||
|
||||
// Create records list that we expect to see in the state message
|
||||
final List<AirbyteMessage> streamTwoExpectedRecords = Arrays.asList(
|
||||
createRecord(streamTwoName, namespace, ImmutableMap.of(
|
||||
createRecord(streamTwoName, getDefaultNamespace(), ImmutableMap.of(
|
||||
COL_ID, 40,
|
||||
COL_NAME, "Jean Luc",
|
||||
COL_UPDATED_AT, "2006-10-19")),
|
||||
createRecord(streamTwoName, namespace, ImmutableMap.of(
|
||||
createRecord(streamTwoName, getDefaultNamespace(), ImmutableMap.of(
|
||||
COL_ID, 41,
|
||||
COL_NAME, "Groot",
|
||||
COL_UPDATED_AT, "2006-10-19")),
|
||||
createRecord(streamTwoName, namespace, ImmutableMap.of(
|
||||
createRecord(streamTwoName, getDefaultNamespace(), ImmutableMap.of(
|
||||
COL_ID, 42,
|
||||
COL_NAME, "Thanos",
|
||||
COL_UPDATED_AT, "2006-10-19")));
|
||||
|
||||
// Prep and create a configured catalog to perform sync
|
||||
final AirbyteStream streamOne = getAirbyteStream(streamOneName, namespace);
|
||||
final AirbyteStream streamTwo = getAirbyteStream(streamTwoName, namespace);
|
||||
final AirbyteStream streamOne = getAirbyteStream(streamOneName, getDefaultNamespace());
|
||||
final AirbyteStream streamTwo = getAirbyteStream(streamTwoName, getDefaultNamespace());
|
||||
|
||||
final ConfiguredAirbyteCatalog configuredCatalog = CatalogHelpers.toDefaultConfiguredCatalog(
|
||||
new AirbyteCatalog().withStreams(List.of(streamOne, streamTwo)));
|
||||
@@ -245,7 +156,7 @@ class MySqlJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest {
|
||||
|
||||
// Perform initial sync
|
||||
final List<AirbyteMessage> messagesFromFirstSync = MoreIterators
|
||||
.toList(source.read(config, configuredCatalog, null));
|
||||
.toList(source().read(config, configuredCatalog, null));
|
||||
|
||||
final List<AirbyteMessage> recordsFromFirstSync = filterRecords(messagesFromFirstSync);
|
||||
|
||||
@@ -312,7 +223,7 @@ class MySqlJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest {
|
||||
// - stream two state being the Primary Key state before the final emitted state before the cursor
|
||||
// switch
|
||||
final List<AirbyteMessage> messagesFromSecondSyncWithMixedStates = MoreIterators
|
||||
.toList(source.read(config, configuredCatalog,
|
||||
.toList(source().read(config, configuredCatalog,
|
||||
Jsons.jsonNode(List.of(streamOneStateMessagesFromFirstSync.get(0),
|
||||
streamTwoStateMessagesFromFirstSync.get(1)))));
|
||||
|
||||
@@ -339,21 +250,13 @@ class MySqlJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest {
|
||||
|
||||
// Add some data to each table and perform a third read.
|
||||
// Expect to see all records be synced via cursorBased method and not primaryKey
|
||||
|
||||
database.query(ctx -> {
|
||||
ctx.execute(
|
||||
String.format("INSERT INTO %s(id, name, updated_at)"
|
||||
+ "VALUES (4,'Hooper','2006-10-19')",
|
||||
getFullyQualifiedTableName(streamOneName)));
|
||||
ctx.execute(
|
||||
String.format("INSERT INTO %s(id, name, updated_at)"
|
||||
+ "VALUES (43, 'Iron Man', '2006-10-19')",
|
||||
streamTwoFullyQualifiedName));
|
||||
return null;
|
||||
});
|
||||
testdb.with("INSERT INTO %s(id, name, updated_at) VALUES (4,'Hooper','2006-10-19')",
|
||||
getFullyQualifiedTableName(streamOneName))
|
||||
.with("INSERT INTO %s(id, name, updated_at) VALUES (43, 'Iron Man', '2006-10-19')",
|
||||
streamTwoFullyQualifiedName);
|
||||
|
||||
final List<AirbyteMessage> messagesFromThirdSync = MoreIterators
|
||||
.toList(source.read(config, configuredCatalog,
|
||||
.toList(source().read(config, configuredCatalog,
|
||||
Jsons.jsonNode(List.of(streamOneStateMessagesFromSecondSync.get(1),
|
||||
streamTwoStateMessagesFromSecondSync.get(0)))));
|
||||
|
||||
@@ -386,7 +289,7 @@ class MySqlJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest {
|
||||
|
||||
@Test
|
||||
void testSpec() throws Exception {
|
||||
final ConnectorSpecification actual = source.spec();
|
||||
final ConnectorSpecification actual = source().spec();
|
||||
final ConnectorSpecification expected = Jsons.deserialize(MoreResources.readResource("spec.json"), ConnectorSpecification.class);
|
||||
|
||||
assertEquals(expected, actual);
|
||||
@@ -402,16 +305,20 @@ class MySqlJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest {
|
||||
*/
|
||||
@Test
|
||||
void testCheckIncorrectPasswordFailure() throws Exception {
|
||||
final var config = config();
|
||||
maybeSetShorterConnectionTimeout(config);
|
||||
((ObjectNode) config).put(JdbcUtils.PASSWORD_KEY, "fake");
|
||||
final AirbyteConnectionStatus status = source.check(config);
|
||||
final AirbyteConnectionStatus status = source().check(config);
|
||||
assertEquals(AirbyteConnectionStatus.Status.FAILED, status.getStatus());
|
||||
assertTrue(status.getMessage().contains("State code: 08001;"));
|
||||
assertTrue(status.getMessage().contains("State code: 08001;"), status.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckIncorrectUsernameFailure() throws Exception {
|
||||
final var config = config();
|
||||
maybeSetShorterConnectionTimeout(config);
|
||||
((ObjectNode) config).put(JdbcUtils.USERNAME_KEY, "fake");
|
||||
final AirbyteConnectionStatus status = source.check(config);
|
||||
final AirbyteConnectionStatus status = source().check(config);
|
||||
assertEquals(AirbyteConnectionStatus.Status.FAILED, status.getStatus());
|
||||
// do not test for message since there seems to be flakiness where sometimes the test will get the
|
||||
// message with
|
||||
@@ -420,38 +327,45 @@ class MySqlJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest {
|
||||
|
||||
@Test
|
||||
public void testCheckIncorrectHostFailure() throws Exception {
|
||||
final var config = config();
|
||||
maybeSetShorterConnectionTimeout(config);
|
||||
((ObjectNode) config).put(JdbcUtils.HOST_KEY, "localhost2");
|
||||
final AirbyteConnectionStatus status = source.check(config);
|
||||
final AirbyteConnectionStatus status = source().check(config);
|
||||
assertEquals(AirbyteConnectionStatus.Status.FAILED, status.getStatus());
|
||||
assertTrue(status.getMessage().contains("State code: 08S01;"));
|
||||
assertTrue(status.getMessage().contains("State code: 08S01;"), status.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckIncorrectPortFailure() throws Exception {
|
||||
final var config = config();
|
||||
maybeSetShorterConnectionTimeout(config);
|
||||
((ObjectNode) config).put(JdbcUtils.PORT_KEY, "0000");
|
||||
final AirbyteConnectionStatus status = source.check(config);
|
||||
final AirbyteConnectionStatus status = source().check(config);
|
||||
assertEquals(AirbyteConnectionStatus.Status.FAILED, status.getStatus());
|
||||
assertTrue(status.getMessage().contains("State code: 08S01;"));
|
||||
assertTrue(status.getMessage().contains("State code: 08S01;"), status.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckIncorrectDataBaseFailure() throws Exception {
|
||||
final var config = config();
|
||||
maybeSetShorterConnectionTimeout(config);
|
||||
((ObjectNode) config).put(JdbcUtils.DATABASE_KEY, "wrongdatabase");
|
||||
final AirbyteConnectionStatus status = source.check(config);
|
||||
final AirbyteConnectionStatus status = source().check(config);
|
||||
assertEquals(AirbyteConnectionStatus.Status.FAILED, status.getStatus());
|
||||
assertTrue(status.getMessage().contains("State code: 42000; Error code: 1049;"));
|
||||
assertTrue(status.getMessage().contains("State code: 42000; Error code: 1049;"), status.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUserHasNoPermissionToDataBase() throws Exception {
|
||||
final Connection connection = DriverManager.getConnection(container.getJdbcUrl(), "root", TEST_PASSWORD.call());
|
||||
connection.createStatement()
|
||||
.execute("create user '" + USERNAME_WITHOUT_PERMISSION + "'@'%' IDENTIFIED BY '" + PASSWORD_WITHOUT_PERMISSION + "';\n");
|
||||
((ObjectNode) config).put(JdbcUtils.USERNAME_KEY, USERNAME_WITHOUT_PERMISSION);
|
||||
final var config = config();
|
||||
maybeSetShorterConnectionTimeout(config);
|
||||
final String usernameWithoutPermission = testdb.withNamespace(USERNAME_WITHOUT_PERMISSION);
|
||||
testdb.with("CREATE USER '%s'@'%%' IDENTIFIED BY '%s';", usernameWithoutPermission, PASSWORD_WITHOUT_PERMISSION);
|
||||
((ObjectNode) config).put(JdbcUtils.USERNAME_KEY, usernameWithoutPermission);
|
||||
((ObjectNode) config).put(JdbcUtils.PASSWORD_KEY, PASSWORD_WITHOUT_PERMISSION);
|
||||
final AirbyteConnectionStatus status = source.check(config);
|
||||
final AirbyteConnectionStatus status = source().check(config);
|
||||
assertEquals(AirbyteConnectionStatus.Status.FAILED, status.getStatus());
|
||||
assertTrue(status.getMessage().contains("State code: 08001;"));
|
||||
assertTrue(status.getMessage().contains("State code: 08001;"), status.getMessage());
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -470,13 +384,13 @@ class MySqlJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest {
|
||||
protected List<AirbyteMessage> getExpectedAirbyteMessagesSecondSync(final String namespace) {
|
||||
final List<AirbyteMessage> expectedMessages = new ArrayList<>();
|
||||
expectedMessages.add(new AirbyteMessage().withType(AirbyteMessage.Type.RECORD)
|
||||
.withRecord(new AirbyteRecordMessage().withStream(streamName).withNamespace(namespace)
|
||||
.withRecord(new AirbyteRecordMessage().withStream(streamName()).withNamespace(namespace)
|
||||
.withData(Jsons.jsonNode(ImmutableMap
|
||||
.of(COL_ID, ID_VALUE_4,
|
||||
COL_NAME, "riker",
|
||||
COL_UPDATED_AT, "2006-10-19")))));
|
||||
expectedMessages.add(new AirbyteMessage().withType(AirbyteMessage.Type.RECORD)
|
||||
.withRecord(new AirbyteRecordMessage().withStream(streamName).withNamespace(namespace)
|
||||
.withRecord(new AirbyteRecordMessage().withStream(streamName()).withNamespace(namespace)
|
||||
.withData(Jsons.jsonNode(ImmutableMap
|
||||
.of(COL_ID, ID_VALUE_5,
|
||||
COL_NAME, "data",
|
||||
@@ -484,7 +398,7 @@ class MySqlJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest {
|
||||
final DbStreamState state = new CursorBasedStatus()
|
||||
.withStateType(StateType.CURSOR_BASED)
|
||||
.withVersion(2L)
|
||||
.withStreamName(streamName)
|
||||
.withStreamName(streamName())
|
||||
.withStreamNamespace(namespace)
|
||||
.withCursorField(ImmutableList.of(COL_ID))
|
||||
.withCursor("5")
|
||||
@@ -501,7 +415,7 @@ class MySqlJdbcSourceAcceptanceTest extends JdbcSourceAcceptanceTest {
|
||||
|
||||
@Override
|
||||
protected List<AirbyteMessage> getTestMessages() {
|
||||
return getTestMessages(streamName);
|
||||
return getTestMessages(streamName());
|
||||
}
|
||||
|
||||
protected List<AirbyteMessage> getTestMessages(final String streamName) {
|
||||
|
||||
@@ -4,15 +4,12 @@
|
||||
|
||||
package io.airbyte.integrations.source.mysql;
|
||||
|
||||
import static io.airbyte.integrations.source.mysql.MySqlSource.DRIVER_CLASS;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.hamcrest.Matchers.containsInAnyOrder;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||
import com.mysql.cj.MysqlType;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.jdbc.DateTimeConverter;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import java.sql.Connection;
|
||||
@@ -27,279 +24,106 @@ import java.time.LocalTime;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import org.jooq.SQLDialect;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.IntFunction;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
|
||||
public class MySqlSourceOperationsTest {
|
||||
|
||||
private final MySqlSourceOperations sqlSourceOperations = new MySqlSourceOperations();
|
||||
private MySQLContainer<?> container;
|
||||
private Database database;
|
||||
|
||||
@BeforeEach
|
||||
public void init() {
|
||||
container = new MySQLContainer<>("mysql:8.0");
|
||||
container.start();
|
||||
database = new Database(DSLContextFactory.create(
|
||||
"root",
|
||||
"test",
|
||||
DRIVER_CLASS,
|
||||
String.format("jdbc:mysql://%s:%s",
|
||||
container.getHost(),
|
||||
container.getFirstMappedPort()),
|
||||
SQLDialect.MYSQL));
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() {
|
||||
try {
|
||||
container.close();
|
||||
} catch (final Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void dateColumnAsCursor() throws SQLException {
|
||||
final String tableName = container.getDatabaseName() + ".table_with_date";
|
||||
final String cursorColumn = "cursor_column";
|
||||
executeQuery("CREATE TABLE " + tableName + "(id INTEGER PRIMARY KEY, " + cursorColumn + " DATE);");
|
||||
|
||||
final List<JsonNode> expectedRecords = new ArrayList<>();
|
||||
for (int i = 1; i <= 4; i++) {
|
||||
final ObjectNode jsonNode = (ObjectNode) Jsons.jsonNode(Collections.emptyMap());
|
||||
jsonNode.put("id", i);
|
||||
final LocalDate cursorValue = LocalDate.of(2019, 1, i);
|
||||
jsonNode.put("cursor_column", DateTimeConverter.convertToDate(cursorValue));
|
||||
executeQuery("INSERT INTO " + tableName + " VALUES (" + i + ", '" + cursorValue + "');");
|
||||
if (i >= 2) {
|
||||
expectedRecords.add(jsonNode);
|
||||
}
|
||||
}
|
||||
|
||||
final List<JsonNode> actualRecords = new ArrayList<>();
|
||||
try (final Connection connection = container.createConnection("")) {
|
||||
final PreparedStatement preparedStatement = connection.prepareStatement(
|
||||
"SELECT * from " + tableName + " WHERE " + cursorColumn + " > ?");
|
||||
sqlSourceOperations.setCursorField(preparedStatement, 1, MysqlType.DATE, DateTimeConverter.convertToDate(LocalDate.of(2019, 1, 1)));
|
||||
|
||||
try (final ResultSet resultSet = preparedStatement.executeQuery()) {
|
||||
while (resultSet.next()) {
|
||||
final ObjectNode jsonNode = (ObjectNode) Jsons.jsonNode(Collections.emptyMap());
|
||||
for (int i = 1; i <= resultSet.getMetaData().getColumnCount(); i++) {
|
||||
sqlSourceOperations.copyToJsonField(resultSet, i, jsonNode);
|
||||
}
|
||||
actualRecords.add(jsonNode);
|
||||
}
|
||||
}
|
||||
}
|
||||
assertThat(actualRecords, containsInAnyOrder(expectedRecords.toArray()));
|
||||
|
||||
// Test to check backward compatibility for connectors created before PR
|
||||
// https://github.com/airbytehq/airbyte/pull/15504
|
||||
actualRecords.clear();
|
||||
try (final Connection connection = container.createConnection("")) {
|
||||
final PreparedStatement preparedStatement = connection.prepareStatement(
|
||||
"SELECT * from " + tableName + " WHERE " + cursorColumn + " > ?");
|
||||
sqlSourceOperations.setCursorField(preparedStatement, 1, MysqlType.DATE, "2019-01-01T00:00:00Z");
|
||||
|
||||
try (final ResultSet resultSet = preparedStatement.executeQuery()) {
|
||||
while (resultSet.next()) {
|
||||
final ObjectNode jsonNode = (ObjectNode) Jsons.jsonNode(Collections.emptyMap());
|
||||
for (int i = 1; i <= resultSet.getMetaData().getColumnCount(); i++) {
|
||||
sqlSourceOperations.copyToJsonField(resultSet, i, jsonNode);
|
||||
}
|
||||
actualRecords.add(jsonNode);
|
||||
}
|
||||
}
|
||||
}
|
||||
assertThat(actualRecords, containsInAnyOrder(expectedRecords.toArray()));
|
||||
testImpl(
|
||||
"DATE",
|
||||
i -> LocalDate.of(2019, 1, i),
|
||||
DateTimeConverter::convertToDate,
|
||||
LocalDate::toString,
|
||||
MysqlType.DATE,
|
||||
DateTimeConverter.convertToDate(LocalDate.of(2019, 1, 1)),
|
||||
"2019-01-01T00:00:00Z");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void timeColumnAsCursor() throws SQLException {
|
||||
final String tableName = container.getDatabaseName() + ".table_with_time";
|
||||
final String cursorColumn = "cursor_column";
|
||||
executeQuery("CREATE TABLE " + tableName + "(id INTEGER PRIMARY KEY, " + cursorColumn + " TIME);");
|
||||
|
||||
final List<JsonNode> expectedRecords = new ArrayList<>();
|
||||
for (int i = 1; i <= 4; i++) {
|
||||
final ObjectNode jsonNode = (ObjectNode) Jsons.jsonNode(Collections.emptyMap());
|
||||
jsonNode.put("id", i);
|
||||
final LocalTime cursorValue = LocalTime.of(20, i, 0);
|
||||
jsonNode.put("cursor_column", DateTimeConverter.convertToTime(cursorValue));
|
||||
executeQuery("INSERT INTO " + tableName + " VALUES (" + i + ", '" + cursorValue + "');");
|
||||
if (i >= 2) {
|
||||
expectedRecords.add(jsonNode);
|
||||
}
|
||||
}
|
||||
|
||||
final List<JsonNode> actualRecords = new ArrayList<>();
|
||||
try (final Connection connection = container.createConnection("")) {
|
||||
final PreparedStatement preparedStatement = connection.prepareStatement(
|
||||
"SELECT * from " + tableName + " WHERE " + cursorColumn + " > ?");
|
||||
sqlSourceOperations.setCursorField(preparedStatement, 1, MysqlType.TIME, DateTimeConverter.convertToTime(LocalTime.of(20, 1, 0)));
|
||||
|
||||
try (final ResultSet resultSet = preparedStatement.executeQuery()) {
|
||||
while (resultSet.next()) {
|
||||
final ObjectNode jsonNode = (ObjectNode) Jsons.jsonNode(Collections.emptyMap());
|
||||
for (int i = 1; i <= resultSet.getMetaData().getColumnCount(); i++) {
|
||||
sqlSourceOperations.copyToJsonField(resultSet, i, jsonNode);
|
||||
}
|
||||
actualRecords.add(jsonNode);
|
||||
}
|
||||
}
|
||||
}
|
||||
assertThat(actualRecords, containsInAnyOrder(expectedRecords.toArray()));
|
||||
|
||||
// Test to check backward compatibility for connectors created before PR
|
||||
// https://github.com/airbytehq/airbyte/pull/15504
|
||||
actualRecords.clear();
|
||||
try (final Connection connection = container.createConnection("")) {
|
||||
final PreparedStatement preparedStatement = connection.prepareStatement(
|
||||
"SELECT * from " + tableName + " WHERE " + cursorColumn + " > ?");
|
||||
sqlSourceOperations.setCursorField(preparedStatement, 1, MysqlType.TIME, "1970-01-01T20:01:00Z");
|
||||
|
||||
try (final ResultSet resultSet = preparedStatement.executeQuery()) {
|
||||
while (resultSet.next()) {
|
||||
final ObjectNode jsonNode = (ObjectNode) Jsons.jsonNode(Collections.emptyMap());
|
||||
for (int i = 1; i <= resultSet.getMetaData().getColumnCount(); i++) {
|
||||
sqlSourceOperations.copyToJsonField(resultSet, i, jsonNode);
|
||||
}
|
||||
actualRecords.add(jsonNode);
|
||||
}
|
||||
}
|
||||
}
|
||||
testImpl(
|
||||
"TIME",
|
||||
i -> LocalTime.of(20, i, 0),
|
||||
DateTimeConverter::convertToTime,
|
||||
LocalTime::toString,
|
||||
MysqlType.TIME,
|
||||
DateTimeConverter.convertToTime(LocalTime.of(20, 1, 0)),
|
||||
"1970-01-01T20:01:00Z");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void dateTimeColumnAsCursor() throws SQLException {
|
||||
final String tableName = container.getDatabaseName() + ".table_with_datetime";
|
||||
final String cursorColumn = "cursor_column";
|
||||
executeQuery("CREATE TABLE " + tableName + "(id INTEGER PRIMARY KEY, " + cursorColumn + " DATETIME);");
|
||||
|
||||
final List<JsonNode> expectedRecords = new ArrayList<>();
|
||||
for (int i = 1; i <= 4; i++) {
|
||||
final ObjectNode jsonNode = (ObjectNode) Jsons.jsonNode(Collections.emptyMap());
|
||||
jsonNode.put("id", i);
|
||||
final LocalDateTime cursorValue = LocalDateTime.of(2019, i, 20, 3, 0, 0);
|
||||
jsonNode.put("cursor_column", DateTimeConverter.convertToTimestamp(cursorValue));
|
||||
executeQuery("INSERT INTO " + tableName + " VALUES (" + i + ", '" + cursorValue + "');");
|
||||
if (i >= 2) {
|
||||
expectedRecords.add(jsonNode);
|
||||
}
|
||||
}
|
||||
|
||||
final List<JsonNode> actualRecords = new ArrayList<>();
|
||||
try (final Connection connection = container.createConnection("")) {
|
||||
final PreparedStatement preparedStatement = connection.prepareStatement(
|
||||
"SELECT * from " + tableName + " WHERE " + cursorColumn + " > ?");
|
||||
sqlSourceOperations.setCursorField(preparedStatement, 1, MysqlType.DATETIME,
|
||||
DateTimeConverter.convertToTimestamp(LocalDateTime.of(2019, 1, 20, 3, 0, 0)));
|
||||
|
||||
try (final ResultSet resultSet = preparedStatement.executeQuery()) {
|
||||
while (resultSet.next()) {
|
||||
final ObjectNode jsonNode = (ObjectNode) Jsons.jsonNode(Collections.emptyMap());
|
||||
for (int i = 1; i <= resultSet.getMetaData().getColumnCount(); i++) {
|
||||
sqlSourceOperations.copyToJsonField(resultSet, i, jsonNode);
|
||||
}
|
||||
actualRecords.add(jsonNode);
|
||||
}
|
||||
}
|
||||
}
|
||||
assertThat(actualRecords, containsInAnyOrder(expectedRecords.toArray()));
|
||||
|
||||
// Test to check backward compatibility for connectors created before PR
|
||||
// https://github.com/airbytehq/airbyte/pull/15504
|
||||
actualRecords.clear();
|
||||
try (final Connection connection = container.createConnection("")) {
|
||||
final PreparedStatement preparedStatement = connection.prepareStatement(
|
||||
"SELECT * from " + tableName + " WHERE " + cursorColumn + " > ?");
|
||||
sqlSourceOperations.setCursorField(preparedStatement, 1, MysqlType.DATETIME, "2019-01-20T03:00:00.000000");
|
||||
|
||||
try (final ResultSet resultSet = preparedStatement.executeQuery()) {
|
||||
while (resultSet.next()) {
|
||||
final ObjectNode jsonNode = (ObjectNode) Jsons.jsonNode(Collections.emptyMap());
|
||||
for (int i = 1; i <= resultSet.getMetaData().getColumnCount(); i++) {
|
||||
sqlSourceOperations.copyToJsonField(resultSet, i, jsonNode);
|
||||
}
|
||||
actualRecords.add(jsonNode);
|
||||
}
|
||||
}
|
||||
}
|
||||
assertThat(actualRecords, containsInAnyOrder(expectedRecords.toArray()));
|
||||
testImpl(
|
||||
"DATETIME",
|
||||
i -> LocalDateTime.of(2019, i, 20, 3, 0, 0),
|
||||
DateTimeConverter::convertToTimestamp,
|
||||
LocalDateTime::toString,
|
||||
MysqlType.DATETIME,
|
||||
DateTimeConverter.convertToTimestamp(LocalDateTime.of(2019, 1, 20, 3, 0, 0)),
|
||||
"2019-01-20T03:00:00.000000");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void timestampColumnAsCursor() throws SQLException {
|
||||
final String tableName = container.getDatabaseName() + ".table_with_timestamp";
|
||||
final String cursorColumn = "cursor_column";
|
||||
executeQuery("CREATE TABLE " + tableName + "(id INTEGER PRIMARY KEY, " + cursorColumn + " timestamp);");
|
||||
|
||||
final List<JsonNode> expectedRecords = new ArrayList<>();
|
||||
for (int i = 1; i <= 4; i++) {
|
||||
final ObjectNode jsonNode = (ObjectNode) Jsons.jsonNode(Collections.emptyMap());
|
||||
jsonNode.put("id", i);
|
||||
final Instant cursorValue = Instant.ofEpochSecond(1660298508L).plusSeconds(i - 1);
|
||||
jsonNode.put("cursor_column", DateTimeConverter.convertToTimestampWithTimezone(cursorValue));
|
||||
executeQuery("INSERT INTO " + tableName + " VALUES (" + i + ", '" + Timestamp.from(cursorValue) + "');");
|
||||
if (i >= 2) {
|
||||
expectedRecords.add(jsonNode);
|
||||
}
|
||||
}
|
||||
|
||||
final List<JsonNode> actualRecords = new ArrayList<>();
|
||||
try (final Connection connection = container.createConnection("")) {
|
||||
final PreparedStatement preparedStatement = connection.prepareStatement(
|
||||
"SELECT * from " + tableName + " WHERE " + cursorColumn + " > ?");
|
||||
sqlSourceOperations.setCursorField(preparedStatement, 1, MysqlType.TIMESTAMP,
|
||||
DateTimeConverter.convertToTimestampWithTimezone(Instant.ofEpochSecond(1660298508L)));
|
||||
|
||||
try (final ResultSet resultSet = preparedStatement.executeQuery()) {
|
||||
while (resultSet.next()) {
|
||||
final ObjectNode jsonNode = (ObjectNode) Jsons.jsonNode(Collections.emptyMap());
|
||||
for (int i = 1; i <= resultSet.getMetaData().getColumnCount(); i++) {
|
||||
sqlSourceOperations.copyToJsonField(resultSet, i, jsonNode);
|
||||
}
|
||||
actualRecords.add(jsonNode);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Assertions.assertEquals(3, actualRecords.size());
|
||||
|
||||
// Test to check backward compatibility for connectors created before PR
|
||||
// https://github.com/airbytehq/airbyte/pull/15504
|
||||
actualRecords.clear();
|
||||
try (final Connection connection = container.createConnection("")) {
|
||||
final PreparedStatement preparedStatement = connection.prepareStatement(
|
||||
"SELECT * from " + tableName + " WHERE " + cursorColumn + " > ?");
|
||||
sqlSourceOperations.setCursorField(preparedStatement, 1, MysqlType.TIMESTAMP, Instant.ofEpochSecond(1660298508L).toString());
|
||||
|
||||
try (final ResultSet resultSet = preparedStatement.executeQuery()) {
|
||||
while (resultSet.next()) {
|
||||
final ObjectNode jsonNode = (ObjectNode) Jsons.jsonNode(Collections.emptyMap());
|
||||
for (int i = 1; i <= resultSet.getMetaData().getColumnCount(); i++) {
|
||||
sqlSourceOperations.copyToJsonField(resultSet, i, jsonNode);
|
||||
}
|
||||
actualRecords.add(jsonNode);
|
||||
}
|
||||
}
|
||||
}
|
||||
Assertions.assertEquals(3, actualRecords.size());
|
||||
testImpl(
|
||||
"TIMESTAMP",
|
||||
i -> Instant.ofEpochSecond(1660298508L).plusSeconds(i - 1),
|
||||
DateTimeConverter::convertToTimestampWithTimezone,
|
||||
r -> Timestamp.from(r).toString(),
|
||||
MysqlType.TIMESTAMP,
|
||||
DateTimeConverter.convertToTimestampWithTimezone(Instant.ofEpochSecond(1660298508L)),
|
||||
Instant.ofEpochSecond(1660298508L).toString());
|
||||
}
|
||||
|
||||
protected void executeQuery(final String query) {
|
||||
try {
|
||||
database.query(
|
||||
ctx -> ctx
|
||||
.execute(query));
|
||||
} catch (final SQLException e) {
|
||||
throw new RuntimeException(e);
|
||||
private <T> void testImpl(
|
||||
final String sqlType,
|
||||
IntFunction<T> recordBuilder,
|
||||
Function<T, String> airbyteRecordStringifier,
|
||||
Function<T, String> sqlRecordStringifier,
|
||||
MysqlType mysqlType,
|
||||
String initialCursorFieldValue,
|
||||
// Test to check backward compatibility for connectors created before PR
|
||||
// https://github.com/airbytehq/airbyte/pull/15504
|
||||
String backwardCompatibleInitialCursorFieldValue)
|
||||
throws SQLException {
|
||||
final var sqlSourceOperations = new MySqlSourceOperations();
|
||||
final String cursorColumn = "cursor_column";
|
||||
try (final var testdb = MySQLTestDatabase.in("mysql:8.0")
|
||||
.with("CREATE TABLE cursor_table (id INTEGER PRIMARY KEY, %s %s);", cursorColumn, sqlType)) {
|
||||
|
||||
final List<JsonNode> expectedRecords = new ArrayList<>();
|
||||
for (int i = 1; i <= 4; i++) {
|
||||
final ObjectNode jsonNode = (ObjectNode) Jsons.jsonNode(Collections.emptyMap());
|
||||
jsonNode.put("id", i);
|
||||
final T cursorValue = recordBuilder.apply(i);
|
||||
jsonNode.put("cursor_column", airbyteRecordStringifier.apply(cursorValue));
|
||||
testdb.with("INSERT INTO cursor_table VALUES (%d, '%s');", i, sqlRecordStringifier.apply(cursorValue));
|
||||
if (i >= 2) {
|
||||
expectedRecords.add(jsonNode);
|
||||
}
|
||||
}
|
||||
|
||||
try (final Connection connection = testdb.getContainer().createConnection("")) {
|
||||
final PreparedStatement preparedStatement = connection.prepareStatement(
|
||||
"SELECT * FROM " + testdb.getDatabaseName() + ".cursor_table WHERE " + cursorColumn + " > ?");
|
||||
for (final var initialValue : List.of(initialCursorFieldValue, backwardCompatibleInitialCursorFieldValue)) {
|
||||
sqlSourceOperations.setCursorField(preparedStatement, 1, mysqlType, initialValue);
|
||||
final List<JsonNode> actualRecords = new ArrayList<>();
|
||||
try (final ResultSet resultSet = preparedStatement.executeQuery()) {
|
||||
while (resultSet.next()) {
|
||||
final ObjectNode jsonNode = (ObjectNode) Jsons.jsonNode(Collections.emptyMap());
|
||||
for (int i = 1; i <= resultSet.getMetaData().getColumnCount(); i++) {
|
||||
sqlSourceOperations.copyToJsonField(resultSet, i, jsonNode);
|
||||
}
|
||||
actualRecords.add(jsonNode);
|
||||
}
|
||||
}
|
||||
assertThat(actualRecords, containsInAnyOrder(expectedRecords.toArray()));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -17,8 +17,9 @@ import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.source.jdbc.AbstractJdbcSource;
|
||||
import io.airbyte.cdk.integrations.source.jdbc.AbstractJdbcSource.PrimaryKeyAttributesFromDb;
|
||||
import io.airbyte.commons.exceptions.ConfigErrorException;
|
||||
import io.airbyte.commons.features.EnvVariableFeatureFlags;
|
||||
import io.airbyte.commons.features.FeatureFlagsWrapper;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.commons.string.Strings;
|
||||
import io.airbyte.commons.util.MoreIterators;
|
||||
import io.airbyte.protocol.models.Field;
|
||||
import io.airbyte.protocol.models.JsonSchemaType;
|
||||
@@ -28,71 +29,36 @@ import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog;
|
||||
import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream;
|
||||
import io.airbyte.protocol.models.v0.DestinationSyncMode;
|
||||
import io.airbyte.protocol.models.v0.SyncMode;
|
||||
import java.sql.Connection;
|
||||
import java.sql.DriverManager;
|
||||
import java.sql.SQLException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import org.junit.jupiter.api.Disabled;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
import org.testcontainers.containers.output.Slf4jLogConsumer;
|
||||
|
||||
public class MySqlSourceTests {
|
||||
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(MySqlSourceTests.class);
|
||||
|
||||
private static final String TEST_USER = "test";
|
||||
private static final String TEST_PASSWORD = "test";
|
||||
public MySqlSource source() {
|
||||
final var source = new MySqlSource();
|
||||
source.setFeatureFlags(FeatureFlagsWrapper.overridingUseStreamCapableState(new EnvVariableFeatureFlags(), true));
|
||||
return source;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSettingTimezones() throws Exception {
|
||||
// start DB
|
||||
try (final MySQLContainer<?> container = new MySQLContainer<>("mysql:8.0")
|
||||
.withUsername(TEST_USER)
|
||||
.withPassword(TEST_PASSWORD)
|
||||
.withEnv("MYSQL_ROOT_HOST", "%")
|
||||
.withEnv("MYSQL_ROOT_PASSWORD", TEST_PASSWORD)
|
||||
.withEnv("TZ", "Europe/Moscow")
|
||||
.withLogConsumer(new Slf4jLogConsumer(LOGGER))) {
|
||||
|
||||
container.start();
|
||||
|
||||
final Properties properties = new Properties();
|
||||
properties.putAll(ImmutableMap.of("user", "root", JdbcUtils.PASSWORD_KEY, TEST_PASSWORD, "serverTimezone", "Europe/Moscow"));
|
||||
DriverManager.getConnection(container.getJdbcUrl(), properties);
|
||||
final String dbName = Strings.addRandomSuffix("db", "_", 10);
|
||||
final JsonNode config = getConfig(container, dbName, "serverTimezone=Europe/Moscow");
|
||||
|
||||
try (final Connection connection = DriverManager.getConnection(container.getJdbcUrl(), properties)) {
|
||||
connection.createStatement().execute("GRANT ALL PRIVILEGES ON *.* TO '" + TEST_USER + "'@'%';\n");
|
||||
connection.createStatement().execute("CREATE DATABASE " + config.get(JdbcUtils.DATABASE_KEY).asText());
|
||||
}
|
||||
final AirbyteConnectionStatus check = new MySqlSource().check(config);
|
||||
assertEquals(AirbyteConnectionStatus.Status.SUCCEEDED, check.getStatus());
|
||||
try (final var testdb = MySQLTestDatabase.in("mysql:8.0", "withMoscowTimezone")) {
|
||||
final var config = testdb.testConfigBuilder()
|
||||
.with(JdbcUtils.JDBC_URL_PARAMS_KEY, "serverTimezone=Europe/Moscow")
|
||||
.withoutSsl()
|
||||
.build();
|
||||
final AirbyteConnectionStatus check = source().check(config);
|
||||
assertEquals(AirbyteConnectionStatus.Status.SUCCEEDED, check.getStatus(), check.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
private static JsonNode getConfig(final MySQLContainer dbContainer, final String dbName, final String jdbcParams) {
|
||||
return Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, dbContainer.getHost())
|
||||
.put(JdbcUtils.PORT_KEY, dbContainer.getFirstMappedPort())
|
||||
.put(JdbcUtils.DATABASE_KEY, dbName)
|
||||
.put(JdbcUtils.USERNAME_KEY, TEST_USER)
|
||||
.put(JdbcUtils.PASSWORD_KEY, TEST_PASSWORD)
|
||||
.put(JdbcUtils.JDBC_URL_PARAMS_KEY, jdbcParams)
|
||||
.build());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testJdbcUrlWithEscapedDatabaseName() {
|
||||
final JsonNode jdbcConfig = new MySqlSource().toDatabaseConfig(buildConfigEscapingNeeded());
|
||||
final JsonNode jdbcConfig = source().toDatabaseConfig(buildConfigEscapingNeeded());
|
||||
assertNotNull(jdbcConfig.get(JdbcUtils.JDBC_URL_KEY).asText());
|
||||
assertTrue(jdbcConfig.get(JdbcUtils.JDBC_URL_KEY).asText().startsWith(EXPECTED_JDBC_ESCAPED_URL));
|
||||
}
|
||||
@@ -109,95 +75,45 @@ public class MySqlSourceTests {
|
||||
|
||||
@Test
|
||||
@Disabled("See https://github.com/airbytehq/airbyte/pull/23908#issuecomment-1463753684, enable once communication is out")
|
||||
public void testTableWithNullCursorValueShouldThrowException() throws SQLException {
|
||||
try (final MySQLContainer<?> db = new MySQLContainer<>("mysql:8.0")
|
||||
.withUsername(TEST_USER)
|
||||
.withPassword(TEST_PASSWORD)
|
||||
.withEnv("MYSQL_ROOT_HOST", "%")
|
||||
.withEnv("MYSQL_ROOT_PASSWORD", TEST_PASSWORD)) {
|
||||
db.start();
|
||||
final JsonNode config = getConfig(db, "test", "");
|
||||
try (Connection connection = DriverManager.getConnection(db.getJdbcUrl(), "root", config.get(JdbcUtils.PASSWORD_KEY).asText())) {
|
||||
final ConfiguredAirbyteStream table = createTableWithNullValueCursor(connection);
|
||||
final ConfiguredAirbyteCatalog catalog = new ConfiguredAirbyteCatalog().withStreams(Collections.singletonList(table));
|
||||
public void testNullCursorValueShouldThrowException() {
|
||||
try (final var testdb = MySQLTestDatabase.in("mysql:8.0")
|
||||
.with("CREATE TABLE null_cursor_table(id INTEGER NULL);")
|
||||
.with("INSERT INTO null_cursor_table(id) VALUES (1), (2), (NULL);")
|
||||
.with("CREATE VIEW null_cursor_view(id) AS SELECT null_cursor_table.id FROM null_cursor_table;")) {
|
||||
final var config = testdb.testConfigBuilder().withoutSsl().build();
|
||||
|
||||
final Throwable throwable = catchThrowable(() -> MoreIterators.toSet(new MySqlSource().read(config, catalog, null)));
|
||||
assertThat(throwable).isInstanceOf(ConfigErrorException.class)
|
||||
.hasMessageContaining(
|
||||
"The following tables have invalid columns selected as cursor, please select a column with a well-defined ordering with no null values as a cursor. {tableName='test.null_cursor_table', cursorColumnName='id', cursorSqlType=INT, cause=Cursor column contains NULL value}");
|
||||
final var tableStream = new ConfiguredAirbyteStream()
|
||||
.withCursorField(Lists.newArrayList("id"))
|
||||
.withDestinationSyncMode(DestinationSyncMode.APPEND)
|
||||
.withSyncMode(SyncMode.INCREMENTAL)
|
||||
.withStream(CatalogHelpers.createAirbyteStream(
|
||||
"null_cursor_table",
|
||||
testdb.getDatabaseName(),
|
||||
Field.of("id", JsonSchemaType.STRING))
|
||||
.withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL))
|
||||
.withSourceDefinedPrimaryKey(List.of(List.of("id"))));
|
||||
final var tableCatalog = new ConfiguredAirbyteCatalog().withStreams(List.of(tableStream));
|
||||
final var tableThrowable = catchThrowable(() -> MoreIterators.toSet(source().read(config, tableCatalog, null)));
|
||||
assertThat(tableThrowable).isInstanceOf(ConfigErrorException.class).hasMessageContaining(NULL_CURSOR_EXCEPTION_MESSAGE_CONTAINS);
|
||||
|
||||
} finally {
|
||||
db.stop();
|
||||
}
|
||||
final var viewStream = new ConfiguredAirbyteStream()
|
||||
.withCursorField(Lists.newArrayList("id"))
|
||||
.withDestinationSyncMode(DestinationSyncMode.APPEND)
|
||||
.withSyncMode(SyncMode.INCREMENTAL)
|
||||
.withStream(CatalogHelpers.createAirbyteStream(
|
||||
"null_cursor_view",
|
||||
testdb.getDatabaseName(),
|
||||
Field.of("id", JsonSchemaType.STRING))
|
||||
.withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL))
|
||||
.withSourceDefinedPrimaryKey(List.of(List.of("id"))));
|
||||
final var viewCatalog = new ConfiguredAirbyteCatalog().withStreams(List.of(viewStream));
|
||||
final var viewThrowable = catchThrowable(() -> MoreIterators.toSet(source().read(config, viewCatalog, null)));
|
||||
assertThat(viewThrowable).isInstanceOf(ConfigErrorException.class).hasMessageContaining(NULL_CURSOR_EXCEPTION_MESSAGE_CONTAINS);
|
||||
}
|
||||
}
|
||||
|
||||
private ConfiguredAirbyteStream createTableWithNullValueCursor(final Connection connection) throws SQLException {
|
||||
connection.createStatement().execute("GRANT ALL PRIVILEGES ON *.* TO '" + TEST_USER + "'@'%';\n");
|
||||
connection.createStatement().execute("CREATE TABLE IF NOT EXISTS test.null_cursor_table(id INTEGER NULL)");
|
||||
connection.createStatement().execute("INSERT INTO test.null_cursor_table(id) VALUES (1), (2), (NULL)");
|
||||
|
||||
return new ConfiguredAirbyteStream().withSyncMode(SyncMode.INCREMENTAL)
|
||||
.withCursorField(Lists.newArrayList("id"))
|
||||
.withDestinationSyncMode(DestinationSyncMode.APPEND)
|
||||
.withSyncMode(SyncMode.INCREMENTAL)
|
||||
.withStream(CatalogHelpers.createAirbyteStream(
|
||||
"null_cursor_table",
|
||||
"test",
|
||||
Field.of("id", JsonSchemaType.STRING))
|
||||
.withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL))
|
||||
.withSourceDefinedPrimaryKey(List.of(List.of("id"))));
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
@Disabled("See https://github.com/airbytehq/airbyte/pull/23908#issuecomment-1463753684, enable once communication is out")
|
||||
public void viewWithNullValueCursorShouldThrowException() throws SQLException {
|
||||
try (final MySQLContainer<?> db = new MySQLContainer<>("mysql:8.0")
|
||||
.withUsername(TEST_USER)
|
||||
.withPassword(TEST_PASSWORD)
|
||||
.withEnv("MYSQL_ROOT_HOST", "%")
|
||||
.withEnv("MYSQL_ROOT_PASSWORD", TEST_PASSWORD)) {
|
||||
db.start();
|
||||
final JsonNode config = getConfig(db, "test", "");
|
||||
try (Connection connection = DriverManager.getConnection(db.getJdbcUrl(), "root", config.get(JdbcUtils.PASSWORD_KEY).asText())) {
|
||||
final ConfiguredAirbyteStream table = createViewWithNullValueCursor(connection);
|
||||
final ConfiguredAirbyteCatalog catalog = new ConfiguredAirbyteCatalog().withStreams(Collections.singletonList(table));
|
||||
|
||||
final Throwable throwable = catchThrowable(() -> MoreIterators.toSet(new MySqlSource().read(config, catalog, null)));
|
||||
assertThat(throwable).isInstanceOf(ConfigErrorException.class)
|
||||
.hasMessageContaining(
|
||||
"The following tables have invalid columns selected as cursor, please select a column with a well-defined ordering with no null values as a cursor. {tableName='test.test_view_null_cursor', cursorColumnName='id', cursorSqlType=INT, cause=Cursor column contains NULL value}");
|
||||
|
||||
} finally {
|
||||
db.stop();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private ConfiguredAirbyteStream createViewWithNullValueCursor(final Connection connection) throws SQLException {
|
||||
|
||||
connection.createStatement().execute("GRANT ALL PRIVILEGES ON *.* TO '" + TEST_USER + "'@'%';\n");
|
||||
connection.createStatement().execute("CREATE TABLE IF NOT EXISTS test.test_table_null_cursor(id INTEGER NULL)");
|
||||
connection.createStatement().execute("""
|
||||
CREATE VIEW test_view_null_cursor(id) as
|
||||
SELECT test_table_null_cursor.id
|
||||
FROM test_table_null_cursor
|
||||
""");
|
||||
connection.createStatement().execute("INSERT INTO test.test_table_null_cursor(id) VALUES (1), (2), (NULL)");
|
||||
|
||||
return new ConfiguredAirbyteStream().withSyncMode(SyncMode.INCREMENTAL)
|
||||
.withCursorField(Lists.newArrayList("id"))
|
||||
.withDestinationSyncMode(DestinationSyncMode.APPEND)
|
||||
.withSyncMode(SyncMode.INCREMENTAL)
|
||||
.withStream(CatalogHelpers.createAirbyteStream(
|
||||
"test_view_null_cursor",
|
||||
"test",
|
||||
Field.of("id", JsonSchemaType.STRING))
|
||||
.withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL))
|
||||
.withSourceDefinedPrimaryKey(List.of(List.of("id"))));
|
||||
|
||||
}
|
||||
static private final String NULL_CURSOR_EXCEPTION_MESSAGE_CONTAINS = "The following tables have invalid columns " +
|
||||
"selected as cursor, please select a column with a well-defined ordering with no null values as a cursor.";
|
||||
|
||||
@Test
|
||||
void testParseJdbcParameters() {
|
||||
@@ -210,26 +126,12 @@ public class MySqlSourceTests {
|
||||
|
||||
@Test
|
||||
public void testJDBCSessionVariable() throws Exception {
|
||||
// start DB
|
||||
try (final MySQLContainer<?> container = new MySQLContainer<>("mysql:8.0")
|
||||
.withUsername(TEST_USER)
|
||||
.withPassword(TEST_PASSWORD)
|
||||
.withEnv("MYSQL_ROOT_HOST", "%")
|
||||
.withEnv("MYSQL_ROOT_PASSWORD", TEST_PASSWORD)
|
||||
.withLogConsumer(new Slf4jLogConsumer(LOGGER))) {
|
||||
|
||||
container.start();
|
||||
final Properties properties = new Properties();
|
||||
properties.putAll(ImmutableMap.of("user", "root", JdbcUtils.PASSWORD_KEY, TEST_PASSWORD));
|
||||
DriverManager.getConnection(container.getJdbcUrl(), properties);
|
||||
final String dbName = Strings.addRandomSuffix("db", "_", 10);
|
||||
final JsonNode config = getConfig(container, dbName, "sessionVariables=MAX_EXECUTION_TIME=28800000");
|
||||
|
||||
try (final Connection connection = DriverManager.getConnection(container.getJdbcUrl(), properties)) {
|
||||
connection.createStatement().execute("GRANT ALL PRIVILEGES ON *.* TO '" + TEST_USER + "'@'%';\n");
|
||||
connection.createStatement().execute("CREATE DATABASE " + config.get(JdbcUtils.DATABASE_KEY).asText());
|
||||
}
|
||||
final AirbyteConnectionStatus check = new MySqlSource().check(config);
|
||||
try (final var testdb = MySQLTestDatabase.in("mysql:8.0")) {
|
||||
final var config = testdb.testConfigBuilder()
|
||||
.with(JdbcUtils.JDBC_URL_PARAMS_KEY, "sessionVariables=MAX_EXECUTION_TIME=28800000")
|
||||
.withoutSsl()
|
||||
.build();
|
||||
final AirbyteConnectionStatus check = source().check(config);
|
||||
assertEquals(AirbyteConnectionStatus.Status.SUCCEEDED, check.getStatus());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -4,49 +4,27 @@
|
||||
|
||||
package io.airbyte.integrations.source.mysql;
|
||||
|
||||
import static io.airbyte.integrations.source.mysql.MySqlSource.SSL_PARAMETERS;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.airbyte.cdk.db.Database;
|
||||
import io.airbyte.cdk.db.factory.DSLContextFactory;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.commons.string.Strings;
|
||||
import org.jooq.SQLDialect;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Order;
|
||||
|
||||
@Order(3)
|
||||
class MySqlSslJdbcSourceAcceptanceTest extends MySqlJdbcSourceAcceptanceTest {
|
||||
|
||||
@BeforeEach
|
||||
public void setup() throws Exception {
|
||||
config = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put(JdbcUtils.HOST_KEY, container.getHost())
|
||||
.put(JdbcUtils.PORT_KEY, container.getFirstMappedPort())
|
||||
.put(JdbcUtils.DATABASE_KEY, Strings.addRandomSuffix("db", "_", 10))
|
||||
.put(JdbcUtils.USERNAME_KEY, TEST_USER)
|
||||
.put(JdbcUtils.PASSWORD_KEY, TEST_PASSWORD.call())
|
||||
.put(JdbcUtils.SSL_KEY, true)
|
||||
.build());
|
||||
@Override
|
||||
protected JsonNode config() {
|
||||
return testdb.testConfigBuilder()
|
||||
.with(JdbcUtils.SSL_KEY, true)
|
||||
.build();
|
||||
}
|
||||
|
||||
dslContext = DSLContextFactory.create(
|
||||
config.get(JdbcUtils.USERNAME_KEY).asText(),
|
||||
config.get(JdbcUtils.PASSWORD_KEY).asText(),
|
||||
DatabaseDriver.MYSQL.getDriverClassName(),
|
||||
String.format("jdbc:mysql://%s:%s?%s",
|
||||
config.get(JdbcUtils.HOST_KEY).asText(),
|
||||
config.get(JdbcUtils.PORT_KEY).asText(),
|
||||
String.join("&", SSL_PARAMETERS)),
|
||||
SQLDialect.MYSQL);
|
||||
database = new Database(dslContext);
|
||||
|
||||
database.query(ctx -> {
|
||||
ctx.fetch("CREATE DATABASE " + config.get(JdbcUtils.DATABASE_KEY).asText());
|
||||
ctx.fetch("SHOW STATUS LIKE 'Ssl_cipher'");
|
||||
return null;
|
||||
});
|
||||
|
||||
super.setup();
|
||||
@Override
|
||||
protected MySQLTestDatabase createTestDatabase() {
|
||||
return new MySQLTestDatabase(new MySQLContainerFactory().shared("mysql:8.0"))
|
||||
.withConnectionProperty("useSSL", "true")
|
||||
.withConnectionProperty("requireSSL", "true")
|
||||
.initialized()
|
||||
.with("SHOW STATUS LIKE 'Ssl_cipher'");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -0,0 +1,73 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.source.mysql;
|
||||
|
||||
import io.airbyte.cdk.testutils.ContainerFactory;
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
import org.testcontainers.containers.Network;
|
||||
import org.testcontainers.utility.DockerImageName;
|
||||
|
||||
public class MySQLContainerFactory implements ContainerFactory<MySQLContainer<?>> {
|
||||
|
||||
@Override
|
||||
public MySQLContainer<?> createNewContainer(DockerImageName imageName) {
|
||||
return new MySQLContainer<>(imageName.asCompatibleSubstituteFor("mysql"));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<?> getContainerClass() {
|
||||
return MySQLContainer.class;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new network and bind it to the container.
|
||||
*/
|
||||
public void withNetwork(MySQLContainer<?> container) {
|
||||
container.withNetwork(Network.newNetwork());
|
||||
}
|
||||
|
||||
private static final String INVALID_TIMEZONE_CEST = "CEST";
|
||||
|
||||
public void withInvalidTimezoneCEST(MySQLContainer<?> container) {
|
||||
container.withEnv("TZ", INVALID_TIMEZONE_CEST);
|
||||
}
|
||||
|
||||
public void withMoscowTimezone(MySQLContainer<?> container) {
|
||||
container.withEnv("TZ", "Europe/Moscow");
|
||||
}
|
||||
|
||||
public void withRootAndServerCertificates(MySQLContainer<?> container) {
|
||||
execInContainer(container,
|
||||
"sed -i '31 a ssl' /etc/my.cnf",
|
||||
"sed -i '32 a ssl-ca=/var/lib/mysql/ca.pem' /etc/my.cnf",
|
||||
"sed -i '33 a ssl-cert=/var/lib/mysql/server-cert.pem' /etc/my.cnf",
|
||||
"sed -i '34 a ssl-key=/var/lib/mysql/server-key.pem' /etc/my.cnf",
|
||||
"sed -i '35 a require_secure_transport=ON' /etc/my.cnf");
|
||||
}
|
||||
|
||||
public void withClientCertificate(MySQLContainer<?> container) {
|
||||
execInContainer(container,
|
||||
"sed -i '39 a [client]' /etc/mysql/my.cnf",
|
||||
"sed -i '40 a ssl-ca=/var/lib/mysql/ca.pem' /etc/my.cnf",
|
||||
"sed -i '41 a ssl-cert=/var/lib/mysql/client-cert.pem' /etc/my.cnf",
|
||||
"sed -i '42 a ssl-key=/var/lib/mysql/client-key.pem' /etc/my.cnf");
|
||||
}
|
||||
|
||||
static private void execInContainer(MySQLContainer<?> container, String... commands) {
|
||||
container.start();
|
||||
try {
|
||||
for (String command : commands) {
|
||||
container.execInContainer("sh", "-c", command);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new UncheckedIOException(e);
|
||||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,136 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.source.mysql;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.airbyte.cdk.db.factory.DatabaseDriver;
|
||||
import io.airbyte.cdk.testutils.TestDatabase;
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.jooq.SQLDialect;
|
||||
import org.testcontainers.containers.MySQLContainer;
|
||||
|
||||
public class MySQLTestDatabase extends
|
||||
TestDatabase<MySQLContainer<?>, MySQLTestDatabase, MySQLTestDatabase.MySQLConfigBuilder> {
|
||||
|
||||
static public MySQLTestDatabase in(String imageName, String... methods) {
|
||||
final var container = new MySQLContainerFactory().shared(imageName, methods);
|
||||
return new MySQLTestDatabase(container).initialized();
|
||||
}
|
||||
|
||||
public MySQLTestDatabase(MySQLContainer<?> container) {
|
||||
super(container);
|
||||
}
|
||||
|
||||
public MySQLTestDatabase withCdcPermissions() {
|
||||
return this
|
||||
.with("REVOKE ALL PRIVILEGES, GRANT OPTION FROM '%s';", getUserName())
|
||||
.with("GRANT SELECT, RELOAD, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT ON *.* TO '%s';", getUserName());
|
||||
}
|
||||
|
||||
public MySQLTestDatabase withoutStrictMode() {
|
||||
// This disables strict mode in the DB and allows to insert specific values.
|
||||
// For example, it's possible to insert date with zero values "2021-00-00"
|
||||
return with("SET @@sql_mode=''");
|
||||
}
|
||||
|
||||
static private final int MAX_CONNECTIONS = 1000;
|
||||
|
||||
@Override
|
||||
protected Stream<Stream<String>> inContainerBootstrapCmd() {
|
||||
return Stream.of(mysqlCmd(Stream.of(
|
||||
String.format("SET GLOBAL max_connections=%d", MAX_CONNECTIONS),
|
||||
String.format("CREATE DATABASE %s", getDatabaseName()),
|
||||
String.format("CREATE USER '%s' IDENTIFIED BY '%s'", getUserName(), getPassword()),
|
||||
// Grant privileges also to the container's user, which is not root.
|
||||
String.format("GRANT ALL PRIVILEGES ON *.* TO '%s', '%s' WITH GRANT OPTION", getUserName(),
|
||||
getContainer().getUsername()))));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Stream<String> inContainerUndoBootstrapCmd() {
|
||||
return mysqlCmd(Stream.of(
|
||||
String.format("DROP USER '%s'", getUserName()),
|
||||
String.format("DROP DATABASE %s", getDatabaseName())));
|
||||
}
|
||||
|
||||
@Override
|
||||
public DatabaseDriver getDatabaseDriver() {
|
||||
return DatabaseDriver.MYSQL;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SQLDialect getSqlDialect() {
|
||||
return SQLDialect.MYSQL;
|
||||
}
|
||||
|
||||
@Override
|
||||
public MySQLConfigBuilder configBuilder() {
|
||||
return new MySQLConfigBuilder(this);
|
||||
}
|
||||
|
||||
public Stream<String> mysqlCmd(Stream<String> sql) {
|
||||
return Stream.of("bash", "-c", String.format(
|
||||
"set -o errexit -o pipefail; echo \"%s\" | mysql -v -v -v --user=root --password=test",
|
||||
sql.collect(Collectors.joining("; "))));
|
||||
}
|
||||
|
||||
static public class MySQLConfigBuilder extends ConfigBuilder<MySQLTestDatabase, MySQLConfigBuilder> {
|
||||
|
||||
protected MySQLConfigBuilder(MySQLTestDatabase testDatabase) {
|
||||
super(testDatabase);
|
||||
}
|
||||
|
||||
public MySQLConfigBuilder withStandardReplication() {
|
||||
return with("replication_method", ImmutableMap.builder().put("method", "STANDARD").build());
|
||||
}
|
||||
|
||||
public MySQLConfigBuilder withCdcReplication() {
|
||||
return this
|
||||
.with("is_test", true)
|
||||
.with("replication_method", ImmutableMap.builder()
|
||||
.put("method", "CDC")
|
||||
.put("initial_waiting_seconds", 5)
|
||||
.put("server_time_zone", "America/Los_Angeles")
|
||||
.build());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private String cachedCaCertificate;
|
||||
private Certificates cachedCertificates;
|
||||
|
||||
public synchronized String getCaCertificate() {
|
||||
if (cachedCaCertificate == null) {
|
||||
cachedCaCertificate = catFileInContainer("/var/lib/mysql/ca.pem");
|
||||
}
|
||||
return cachedCaCertificate;
|
||||
}
|
||||
|
||||
public synchronized Certificates getCertificates() {
|
||||
if (cachedCertificates == null) {
|
||||
cachedCertificates = new Certificates(
|
||||
catFileInContainer("/var/lib/mysql/ca.pem"),
|
||||
catFileInContainer("/var/lib/mysql/client-cert.pem"),
|
||||
catFileInContainer("/var/lib/mysql/client-key.pem"));
|
||||
}
|
||||
return cachedCertificates;
|
||||
}
|
||||
|
||||
public record Certificates(String caCertificate, String clientCertificate, String clientKey) {}
|
||||
|
||||
private String catFileInContainer(String filePath) {
|
||||
try {
|
||||
return getContainer().execInContainer("sh", "-c", "cat " + filePath).getStdout().trim();
|
||||
} catch (IOException e) {
|
||||
throw new UncheckedIOException(e);
|
||||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -13,12 +13,12 @@ java {
|
||||
}
|
||||
|
||||
airbyteJavaConnector {
|
||||
cdkVersionRequired = '0.4.8'
|
||||
cdkVersionRequired = '0.5.0'
|
||||
features = ['db-sources']
|
||||
useLocalCdk = false
|
||||
useLocalCdk = true
|
||||
}
|
||||
|
||||
airbyteJavaConnector.addCdkDependencies()
|
||||
|
||||
|
||||
application {
|
||||
mainClass = 'io.airbyte.integrations.source.postgres.PostgresSource'
|
||||
@@ -52,6 +52,8 @@ dependencies {
|
||||
implementation libs.bundles.datadog
|
||||
|
||||
testImplementation 'org.hamcrest:hamcrest-all:1.3'
|
||||
testFixturesImplementation libs.testcontainers.jdbc
|
||||
testFixturesImplementation libs.testcontainers.postgresql
|
||||
testImplementation libs.testcontainers.jdbc
|
||||
testImplementation libs.testcontainers.postgresql
|
||||
testImplementation libs.junit.jupiter.system.stubs
|
||||
|
||||
@@ -9,7 +9,7 @@ data:
|
||||
connectorSubtype: database
|
||||
connectorType: source
|
||||
definitionId: decd338e-5647-4c0b-adf4-da0e75f5a750
|
||||
dockerImageTag: 3.2.21
|
||||
dockerImageTag: 3.2.22
|
||||
dockerRepository: airbyte/source-postgres
|
||||
documentationUrl: https://docs.airbyte.com/integrations/sources/postgres
|
||||
githubIssueLabel: source-postgres
|
||||
|
||||
@@ -45,6 +45,8 @@ public class PostgresUtils {
|
||||
public static final Duration MIN_FIRST_RECORD_WAIT_TIME = Duration.ofMinutes(2);
|
||||
public static final Duration MAX_FIRST_RECORD_WAIT_TIME = Duration.ofMinutes(20);
|
||||
public static final Duration DEFAULT_FIRST_RECORD_WAIT_TIME = Duration.ofMinutes(5);
|
||||
public static final Duration DEFAULT_SUBSEQUENT_RECORD_WAIT_TIME = Duration.ofMinutes(1);
|
||||
|
||||
private static final int MIN_QUEUE_SIZE = 1000;
|
||||
private static final int MAX_QUEUE_SIZE = 10000;
|
||||
|
||||
@@ -157,6 +159,18 @@ public class PostgresUtils {
|
||||
return firstRecordWaitTime;
|
||||
}
|
||||
|
||||
public static Duration getSubsequentRecordWaitTime(final JsonNode config) {
|
||||
Duration subsequentRecordWaitTime = DEFAULT_SUBSEQUENT_RECORD_WAIT_TIME;
|
||||
final boolean isTest = config.has("is_test") && config.get("is_test").asBoolean();
|
||||
final Optional<Integer> firstRecordWaitSeconds = getFirstRecordWaitSeconds(config);
|
||||
if (isTest && firstRecordWaitSeconds.isPresent()) {
|
||||
// In tests, reuse the initial_waiting_seconds property to speed things up.
|
||||
subsequentRecordWaitTime = Duration.ofSeconds(firstRecordWaitSeconds.get());
|
||||
}
|
||||
LOGGER.info("Subsequent record waiting time: {} seconds", subsequentRecordWaitTime.getSeconds());
|
||||
return subsequentRecordWaitTime;
|
||||
}
|
||||
|
||||
public static boolean isXmin(final JsonNode config) {
|
||||
final boolean isXmin = config.hasNonNull("replication_method")
|
||||
&& config.get("replication_method").get("method").asText().equals("Xmin");
|
||||
|
||||
@@ -68,6 +68,7 @@ public class PostgresCdcCtidInitializer {
|
||||
try {
|
||||
final JsonNode sourceConfig = database.getSourceConfig();
|
||||
final Duration firstRecordWaitTime = PostgresUtils.getFirstRecordWaitTime(sourceConfig);
|
||||
final Duration subsequentRecordWaitTime = PostgresUtils.getSubsequentRecordWaitTime(sourceConfig);
|
||||
final OptionalInt queueSize = OptionalInt.of(PostgresUtils.getQueueSize(sourceConfig));
|
||||
LOGGER.info("First record waiting time: {} seconds", firstRecordWaitTime.getSeconds());
|
||||
LOGGER.info("Queue size: {}", queueSize.getAsInt());
|
||||
@@ -163,20 +164,9 @@ public class PostgresCdcCtidInitializer {
|
||||
|
||||
final var targetPosition = PostgresCdcTargetPosition.targetPosition(database);
|
||||
final AirbyteDebeziumHandler<Long> handler = new AirbyteDebeziumHandler<>(sourceConfig,
|
||||
targetPosition, false, firstRecordWaitTime, queueSize);
|
||||
targetPosition, false, firstRecordWaitTime, subsequentRecordWaitTime, queueSize);
|
||||
final PostgresCdcStateHandler postgresCdcStateHandler = new PostgresCdcStateHandler(stateManager);
|
||||
|
||||
final boolean canShortCircuitDebeziumEngine = savedOffset.isPresent() &&
|
||||
// Until the need presents itself in production, short-circuiting should only be done in tests.
|
||||
sourceConfig.has("is_test") && sourceConfig.get("is_test").asBoolean() &&
|
||||
!postgresDebeziumStateUtil.maybeReplicationStreamIntervalHasRecords(
|
||||
database.getDatabaseConfig(),
|
||||
sourceConfig.get("replication_method").get("replication_slot").asText(),
|
||||
sourceConfig.get("replication_method").get("publication").asText(),
|
||||
PostgresUtils.getPluginValue(sourceConfig.get("replication_method")),
|
||||
savedOffset.getAsLong(),
|
||||
targetPosition.targetLsn.asLong());
|
||||
|
||||
final Supplier<AutoCloseableIterator<AirbyteMessage>> incrementalIteratorSupplier = () -> handler.getIncrementalIterators(
|
||||
catalog,
|
||||
new PostgresCdcSavedInfoFetcher(stateToBeUsed),
|
||||
@@ -185,8 +175,7 @@ public class PostgresCdcCtidInitializer {
|
||||
PostgresCdcProperties.getDebeziumDefaultProperties(database),
|
||||
DebeziumPropertiesManager.DebeziumConnectorType.RELATIONALDB,
|
||||
emittedAt,
|
||||
false,
|
||||
canShortCircuitDebeziumEngine);
|
||||
false);
|
||||
|
||||
if (initialSyncCtidIterators.isEmpty()) {
|
||||
return Collections.singletonList(incrementalIteratorSupplier.get());
|
||||
|
||||
@@ -5,52 +5,36 @@
|
||||
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.PostgresUtils;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
|
||||
import io.airbyte.cdk.testutils.PostgresTestDatabase;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import java.util.List;
|
||||
import io.airbyte.integrations.source.postgres.PostgresTestDatabase;
|
||||
import java.util.Map;
|
||||
|
||||
public abstract class AbstractCdcPostgresSourceSslAcceptanceTest extends CdcPostgresSourceAcceptanceTest {
|
||||
|
||||
protected static final String PASSWORD = "Passw0rd";
|
||||
protected static PostgresUtils.Certificate certs;
|
||||
|
||||
@Override
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) throws Exception {
|
||||
testdb = PostgresTestDatabase.make(getServerImageName(), "withWalLevelLogical", "withCert");
|
||||
certs = testdb.getCertificate();
|
||||
slotName = testdb.withSuffix("debezium_slot");
|
||||
publication = testdb.withSuffix("publication");
|
||||
final JsonNode replicationMethod = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("method", "CDC")
|
||||
.put("replication_slot", slotName)
|
||||
.put("publication", publication)
|
||||
.put("initial_waiting_seconds", INITIAL_WAITING_SECONDS)
|
||||
.build());
|
||||
config = Jsons.jsonNode(testdb.makeConfigBuilder()
|
||||
.put(JdbcUtils.SCHEMAS_KEY, List.of(NAMESPACE))
|
||||
.put("replication_method", replicationMethod)
|
||||
.put(JdbcUtils.SSL_KEY, true)
|
||||
.put("ssl_mode", getCertificateConfiguration())
|
||||
.put("is_test", true)
|
||||
.build());
|
||||
testdb = PostgresTestDatabase.in(getServerImageName(), "withWalLevelLogical", "withCert")
|
||||
.with("CREATE TABLE id_and_name(id INTEGER primary key, name VARCHAR(200));")
|
||||
.with("INSERT INTO id_and_name (id, name) VALUES (1,'picard'), (2, 'crusher'), (3, 'vash');")
|
||||
.with("CREATE TABLE starships(id INTEGER primary key, name VARCHAR(200));")
|
||||
.with("INSERT INTO starships (id, name) VALUES (1,'enterprise-d'), (2, 'defiant'), (3, 'yamato');")
|
||||
.withReplicationSlot()
|
||||
.withPublicationForAllTables();
|
||||
}
|
||||
|
||||
testdb.database.query(ctx -> {
|
||||
ctx.execute("CREATE TABLE id_and_name(id INTEGER primary key, name VARCHAR(200));");
|
||||
ctx.execute("INSERT INTO id_and_name (id, name) VALUES (1,'picard'), (2, 'crusher'), (3, 'vash');");
|
||||
ctx.execute("CREATE TABLE starships(id INTEGER primary key, name VARCHAR(200));");
|
||||
ctx.execute("INSERT INTO starships (id, name) VALUES (1,'enterprise-d'), (2, 'defiant'), (3, 'yamato');");
|
||||
ctx.execute("SELECT pg_create_logical_replication_slot('" + slotName + "', 'pgoutput');");
|
||||
ctx.execute("CREATE PUBLICATION " + publication + " FOR ALL TABLES;");
|
||||
return null;
|
||||
});
|
||||
@Override
|
||||
protected JsonNode getConfig() {
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.withSchemas(NAMESPACE)
|
||||
.withSsl(getCertificateConfiguration())
|
||||
.withCdcReplication()
|
||||
.build();
|
||||
}
|
||||
|
||||
protected abstract String getServerImageName();
|
||||
|
||||
public abstract ImmutableMap getCertificateConfiguration();
|
||||
public abstract Map<Object, Object> getCertificateConfiguration();
|
||||
|
||||
}
|
||||
|
||||
@@ -10,20 +10,17 @@ import static io.airbyte.protocol.models.JsonSchemaType.STRING_TIMESTAMP_WITH_TI
|
||||
import static io.airbyte.protocol.models.JsonSchemaType.STRING_TIME_WITHOUT_TIMEZONE;
|
||||
import static io.airbyte.protocol.models.JsonSchemaType.STRING_TIME_WITH_TIMEZONE;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
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.cdk.testutils.PostgresTestDatabase;
|
||||
import io.airbyte.integrations.source.postgres.PostgresTestDatabase;
|
||||
import io.airbyte.protocol.models.JsonSchemaPrimitiveUtil.JsonSchemaPrimitive;
|
||||
import io.airbyte.protocol.models.JsonSchemaType;
|
||||
import java.sql.SQLException;
|
||||
import java.util.Set;
|
||||
|
||||
public abstract class AbstractPostgresSourceDatatypeTest extends AbstractSourceDatabaseTypeTest {
|
||||
|
||||
protected PostgresTestDatabase testdb;
|
||||
protected JsonNode config;
|
||||
|
||||
protected static final String SCHEMA_NAME = "test";
|
||||
|
||||
@@ -38,12 +35,7 @@ public abstract class AbstractPostgresSourceDatatypeTest extends AbstractSourceD
|
||||
}
|
||||
|
||||
@Override
|
||||
protected JsonNode getConfig() {
|
||||
return config;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void tearDown(final TestDestinationEnv testEnv) throws SQLException {
|
||||
protected void tearDown(final TestDestinationEnv testEnv) {
|
||||
testdb.close();
|
||||
}
|
||||
|
||||
|
||||
@@ -5,14 +5,12 @@
|
||||
package io.airbyte.integrations.io.airbyte.integration_tests.sources;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.airbyte.cdk.db.PostgresUtils;
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
|
||||
import io.airbyte.cdk.testutils.PostgresTestDatabase;
|
||||
import io.airbyte.commons.features.FeatureFlags;
|
||||
import io.airbyte.commons.features.FeatureFlagsWrapper;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.integrations.source.postgres.PostgresTestDatabase;
|
||||
import io.airbyte.protocol.models.Field;
|
||||
import io.airbyte.protocol.models.JsonSchemaType;
|
||||
import io.airbyte.protocol.models.v0.CatalogHelpers;
|
||||
@@ -22,6 +20,7 @@ import io.airbyte.protocol.models.v0.DestinationSyncMode;
|
||||
import io.airbyte.protocol.models.v0.SyncMode;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public abstract class AbstractPostgresSourceSSLCertificateAcceptanceTest extends AbstractPostgresSourceAcceptanceTest {
|
||||
|
||||
@@ -29,11 +28,9 @@ public abstract class AbstractPostgresSourceSSLCertificateAcceptanceTest extends
|
||||
private static final String STREAM_NAME2 = "starships";
|
||||
private static final String STREAM_NAME_MATERIALIZED_VIEW = "testview";
|
||||
private static final String SCHEMA_NAME = "public";
|
||||
|
||||
private PostgresTestDatabase testdb;
|
||||
private JsonNode config;
|
||||
protected static final String PASSWORD = "Passw0rd";
|
||||
protected static PostgresUtils.Certificate certs;
|
||||
|
||||
protected PostgresTestDatabase testdb;
|
||||
|
||||
@Override
|
||||
protected FeatureFlags featureFlags() {
|
||||
@@ -42,29 +39,15 @@ public abstract class AbstractPostgresSourceSSLCertificateAcceptanceTest extends
|
||||
|
||||
@Override
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) throws Exception {
|
||||
testdb = PostgresTestDatabase.make("postgres:16-bullseye", "withCert");
|
||||
certs = testdb.getCertificate();
|
||||
final JsonNode replicationMethod = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("method", "Standard")
|
||||
.build());
|
||||
|
||||
config = Jsons.jsonNode(testdb.makeConfigBuilder()
|
||||
.put("schemas", Jsons.jsonNode(List.of("public")))
|
||||
.put("ssl", true)
|
||||
.put("replication_method", replicationMethod)
|
||||
.put("ssl_mode", getCertificateConfiguration())
|
||||
.build());
|
||||
testdb.database.query(ctx -> {
|
||||
ctx.fetch("CREATE TABLE id_and_name(id INTEGER, name VARCHAR(200));");
|
||||
ctx.fetch("INSERT INTO id_and_name (id, name) VALUES (1,'picard'), (2, 'crusher'), (3, 'vash');");
|
||||
ctx.fetch("CREATE TABLE starships(id INTEGER, name VARCHAR(200));");
|
||||
ctx.fetch("INSERT INTO starships (id, name) VALUES (1,'enterprise-d'), (2, 'defiant'), (3, 'yamato');");
|
||||
ctx.fetch("CREATE MATERIALIZED VIEW testview AS select * from id_and_name where id = '2';");
|
||||
return null;
|
||||
});
|
||||
testdb = PostgresTestDatabase.in("postgres:16-bullseye", "withCert")
|
||||
.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 MATERIALIZED VIEW testview AS select * from id_and_name where id = '2';");
|
||||
}
|
||||
|
||||
public abstract ImmutableMap getCertificateConfiguration();
|
||||
public abstract Map<Object, Object> getCertificateConfiguration();
|
||||
|
||||
@Override
|
||||
protected void tearDown(final TestDestinationEnv testEnv) {
|
||||
@@ -73,7 +56,11 @@ public abstract class AbstractPostgresSourceSSLCertificateAcceptanceTest extends
|
||||
|
||||
@Override
|
||||
protected JsonNode getConfig() {
|
||||
return config;
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.withSchemas("public")
|
||||
.withStandardReplication()
|
||||
.withSsl(getCertificateConfiguration())
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -13,11 +13,11 @@ import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.base.ssh.SshBastionContainer;
|
||||
import io.airbyte.cdk.integrations.base.ssh.SshTunnel;
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
|
||||
import io.airbyte.cdk.testutils.PostgresTestDatabase;
|
||||
import io.airbyte.commons.features.FeatureFlags;
|
||||
import io.airbyte.commons.features.FeatureFlagsWrapper;
|
||||
import io.airbyte.commons.functional.CheckedFunction;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.integrations.source.postgres.PostgresTestDatabase;
|
||||
import io.airbyte.protocol.models.Field;
|
||||
import io.airbyte.protocol.models.JsonSchemaType;
|
||||
import io.airbyte.protocol.models.v0.CatalogHelpers;
|
||||
@@ -25,6 +25,8 @@ import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog;
|
||||
import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream;
|
||||
import io.airbyte.protocol.models.v0.DestinationSyncMode;
|
||||
import io.airbyte.protocol.models.v0.SyncMode;
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import org.jooq.SQLDialect;
|
||||
@@ -37,13 +39,13 @@ public abstract class AbstractSshPostgresSourceAcceptanceTest extends AbstractPo
|
||||
|
||||
private final SshBastionContainer bastion = new SshBastionContainer();
|
||||
private PostgresTestDatabase testdb;
|
||||
private JsonNode config;
|
||||
|
||||
private void populateDatabaseTestData() throws Exception {
|
||||
final var builder = testdb.makeConfigBuilder()
|
||||
.put("schemas", List.of("public"))
|
||||
.put("ssl", false);
|
||||
final var outerConfig = bastion.getTunnelConfig(getTunnelMethod(), builder, false);
|
||||
final var outerConfig = testdb.integrationTestConfigBuilder()
|
||||
.withSchemas("public")
|
||||
.withoutSsl()
|
||||
.with("tunnel_method", bastion.getTunnelMethod(getTunnelMethod(), false))
|
||||
.build();
|
||||
SshTunnel.sshWrap(
|
||||
outerConfig,
|
||||
JdbcUtils.HOST_LIST_KEY,
|
||||
@@ -82,12 +84,8 @@ public abstract class AbstractSshPostgresSourceAcceptanceTest extends AbstractPo
|
||||
// requiring data to already be in place.
|
||||
@Override
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) throws Exception {
|
||||
testdb = PostgresTestDatabase.make("postgres:16-bullseye", "withNetwork");
|
||||
bastion.initAndStartBastion(testdb.container.getNetwork());
|
||||
final var builder = testdb.makeConfigBuilder()
|
||||
.put("schemas", List.of("public"))
|
||||
.put("ssl", false);
|
||||
config = bastion.getTunnelConfig(getTunnelMethod(), builder, true);
|
||||
testdb = PostgresTestDatabase.in("postgres:16-bullseye", "withNetwork");
|
||||
bastion.initAndStartBastion(testdb.getContainer().getNetwork());
|
||||
populateDatabaseTestData();
|
||||
}
|
||||
|
||||
@@ -98,7 +96,17 @@ public abstract class AbstractSshPostgresSourceAcceptanceTest extends AbstractPo
|
||||
|
||||
@Override
|
||||
protected JsonNode getConfig() {
|
||||
return config;
|
||||
try {
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.withSchemas("public")
|
||||
.withoutSsl()
|
||||
.with("tunnel_method", bastion.getTunnelMethod(getTunnelMethod(), true))
|
||||
.build();
|
||||
} catch (IOException e) {
|
||||
throw new UncheckedIOException(e);
|
||||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -5,13 +5,14 @@
|
||||
package io.airbyte.integrations.io.airbyte.integration_tests.sources;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class CDCPostgresSourceCaCertificateSslAcceptanceTest extends AbstractCdcPostgresSourceSslAcceptanceTest {
|
||||
|
||||
public ImmutableMap getCertificateConfiguration() {
|
||||
public Map<Object, Object> getCertificateConfiguration() {
|
||||
return ImmutableMap.builder()
|
||||
.put("mode", "verify-ca")
|
||||
.put("ca_certificate", certs.getCaCertificate())
|
||||
.put("ca_certificate", testdb.getCertificates().caCertificate())
|
||||
.put("client_key_password", PASSWORD)
|
||||
.build();
|
||||
}
|
||||
|
||||
@@ -5,16 +5,18 @@
|
||||
package io.airbyte.integrations.io.airbyte.integration_tests.sources;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class CDCPostgresSourceFullCertificateSslAcceptanceTest extends AbstractCdcPostgresSourceSslAcceptanceTest {
|
||||
|
||||
@Override
|
||||
public ImmutableMap getCertificateConfiguration() {
|
||||
public Map<Object, Object> getCertificateConfiguration() {
|
||||
final var certs = testdb.getCertificates();
|
||||
return ImmutableMap.builder()
|
||||
.put("mode", "verify-ca")
|
||||
.put("ca_certificate", certs.getCaCertificate())
|
||||
.put("client_certificate", certs.getClientCertificate())
|
||||
.put("client_key", certs.getClientKey())
|
||||
.put("ca_certificate", certs.caCertificate())
|
||||
.put("client_certificate", certs.clientCertificate())
|
||||
.put("client_key", certs.clientKey())
|
||||
.put("client_key_password", PASSWORD)
|
||||
.build();
|
||||
}
|
||||
|
||||
@@ -5,26 +5,16 @@
|
||||
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.Database;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDataHolder;
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
|
||||
import io.airbyte.cdk.testutils.PostgresTestDatabase;
|
||||
import io.airbyte.commons.features.FeatureFlags;
|
||||
import io.airbyte.commons.features.FeatureFlagsWrapper;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.integrations.source.postgres.PostgresTestDatabase;
|
||||
import io.airbyte.protocol.models.JsonSchemaType;
|
||||
import java.sql.SQLException;
|
||||
import java.util.List;
|
||||
|
||||
public class CdcInitialSnapshotPostgresSourceDatatypeTest extends AbstractPostgresSourceDatatypeTest {
|
||||
|
||||
private static final String SCHEMA_NAME = "test";
|
||||
private static final int INITIAL_WAITING_SECONDS = 30;
|
||||
|
||||
private String slotName;
|
||||
private String publication;
|
||||
|
||||
@Override
|
||||
protected FeatureFlags featureFlags() {
|
||||
@@ -33,60 +23,28 @@ public class CdcInitialSnapshotPostgresSourceDatatypeTest extends AbstractPostgr
|
||||
|
||||
@Override
|
||||
protected Database setupDatabase() throws Exception {
|
||||
testdb = PostgresTestDatabase.make("postgres:16-bullseye", "withConf");
|
||||
slotName = testdb.withSuffix("debezium_slot");
|
||||
publication = testdb.withSuffix("publication");
|
||||
|
||||
/**
|
||||
* The publication is not being set as part of the config and because of it
|
||||
* {@link io.airbyte.integrations.source.postgres.PostgresSource#isCdc(JsonNode)} returns false, as
|
||||
* a result no test in this class runs through the cdc path.
|
||||
*/
|
||||
final JsonNode replicationMethod = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("method", "CDC")
|
||||
.put("replication_slot", slotName)
|
||||
.put("publication", publication)
|
||||
.put("initial_waiting_seconds", INITIAL_WAITING_SECONDS)
|
||||
.build());
|
||||
config = Jsons.jsonNode(testdb.makeConfigBuilder()
|
||||
.put(JdbcUtils.SCHEMAS_KEY, List.of(SCHEMA_NAME))
|
||||
.put("replication_method", replicationMethod)
|
||||
.put("is_test", true)
|
||||
.put(JdbcUtils.SSL_KEY, false)
|
||||
.build());
|
||||
|
||||
testdb.database.query(ctx -> {
|
||||
ctx.execute(
|
||||
"SELECT pg_create_logical_replication_slot('" + slotName + "', 'pgoutput');");
|
||||
ctx.execute("CREATE PUBLICATION " + publication + " FOR ALL TABLES;");
|
||||
ctx.execute("CREATE EXTENSION hstore;");
|
||||
return null;
|
||||
});
|
||||
|
||||
testdb.database.query(ctx -> ctx.fetch("CREATE SCHEMA TEST;"));
|
||||
testdb.database.query(ctx -> ctx.fetch("CREATE TYPE mood AS ENUM ('sad', 'ok', 'happy');"));
|
||||
testdb.database.query(ctx -> ctx.fetch("CREATE TYPE inventory_item AS (\n"
|
||||
+ " name text,\n"
|
||||
+ " supplier_id integer,\n"
|
||||
+ " price numeric\n"
|
||||
+ ");"));
|
||||
|
||||
testdb.database.query(ctx -> ctx.fetch("SET TIMEZONE TO 'MST'"));
|
||||
return testdb.database;
|
||||
testdb = PostgresTestDatabase.in("postgres:16-bullseye", "withConf")
|
||||
.with("CREATE EXTENSION hstore;")
|
||||
.with("CREATE SCHEMA TEST;")
|
||||
.with("CREATE TYPE mood AS ENUM ('sad', 'ok', 'happy');")
|
||||
.with("CREATE TYPE inventory_item AS (\n"
|
||||
+ " name text,\n"
|
||||
+ " supplier_id integer,\n"
|
||||
+ " price numeric\n"
|
||||
+ ");")
|
||||
.with("SET TIMEZONE TO 'MST'")
|
||||
.withReplicationSlot()
|
||||
.withPublicationForAllTables();
|
||||
return testdb.getDatabase();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void tearDown(TestDestinationEnv testEnv) throws SQLException {
|
||||
testdb.database.query(ctx -> {
|
||||
ctx.execute("SELECT pg_drop_replication_slot('" + slotName + "');");
|
||||
ctx.execute("DROP PUBLICATION " + publication + " CASCADE;");
|
||||
return null;
|
||||
});
|
||||
super.tearDown(testEnv);
|
||||
}
|
||||
|
||||
public boolean testCatalog() {
|
||||
return true;
|
||||
protected JsonNode getConfig() {
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.withSchemas(SCHEMA_NAME)
|
||||
.withoutSsl()
|
||||
.withCdcReplication()
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -8,14 +8,12 @@ import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
|
||||
import io.airbyte.cdk.testutils.PostgresTestDatabase;
|
||||
import io.airbyte.commons.features.FeatureFlags;
|
||||
import io.airbyte.commons.features.FeatureFlagsWrapper;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.integrations.source.postgres.PostgresTestDatabase;
|
||||
import io.airbyte.protocol.models.Field;
|
||||
import io.airbyte.protocol.models.JsonSchemaType;
|
||||
import io.airbyte.protocol.models.v0.AirbyteMessage;
|
||||
@@ -39,12 +37,8 @@ public class CdcPostgresSourceAcceptanceTest extends AbstractPostgresSourceAccep
|
||||
protected static final String NAMESPACE = "public";
|
||||
private static final String STREAM_NAME = "id_and_name";
|
||||
private static final String STREAM_NAME2 = "starships";
|
||||
protected static final int INITIAL_WAITING_SECONDS = 30;
|
||||
|
||||
protected PostgresTestDatabase testdb;
|
||||
protected JsonNode config;
|
||||
protected String slotName;
|
||||
protected String publication;
|
||||
|
||||
@Override
|
||||
protected FeatureFlags featureFlags() {
|
||||
@@ -53,47 +47,27 @@ public class CdcPostgresSourceAcceptanceTest extends AbstractPostgresSourceAccep
|
||||
|
||||
@Override
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) throws Exception {
|
||||
testdb = PostgresTestDatabase.make(getServerImageName(), "withConf");
|
||||
slotName = testdb.withSuffix("debezium_slot");
|
||||
publication = testdb.withSuffix("publication");
|
||||
final JsonNode replicationMethod = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("method", "CDC")
|
||||
.put("replication_slot", slotName)
|
||||
.put("publication", publication)
|
||||
.put("initial_waiting_seconds", INITIAL_WAITING_SECONDS)
|
||||
.build());
|
||||
|
||||
config = Jsons.jsonNode(testdb.makeConfigBuilder()
|
||||
.put(JdbcUtils.SCHEMAS_KEY, List.of(NAMESPACE))
|
||||
.put("replication_method", replicationMethod)
|
||||
.put(JdbcUtils.SSL_KEY, false)
|
||||
.put("is_test", true)
|
||||
.build());
|
||||
|
||||
testdb.database.query(ctx -> {
|
||||
ctx.execute("CREATE TABLE id_and_name(id INTEGER primary key, name VARCHAR(200));");
|
||||
ctx.execute("INSERT INTO id_and_name (id, name) VALUES (1,'picard'), (2, 'crusher'), (3, 'vash');");
|
||||
ctx.execute("CREATE TABLE starships(id INTEGER primary key, name VARCHAR(200));");
|
||||
ctx.execute("INSERT INTO starships (id, name) VALUES (1,'enterprise-d'), (2, 'defiant'), (3, 'yamato');");
|
||||
ctx.execute("SELECT pg_create_logical_replication_slot('" + slotName + "', 'pgoutput');");
|
||||
ctx.execute("CREATE PUBLICATION " + publication + " FOR ALL TABLES;");
|
||||
return null;
|
||||
});
|
||||
testdb = PostgresTestDatabase.in(getServerImageName(), "withConf")
|
||||
.with("CREATE TABLE id_and_name(id INTEGER primary key, name VARCHAR(200));")
|
||||
.with("INSERT INTO id_and_name (id, name) VALUES (1,'picard'), (2, 'crusher'), (3, 'vash');")
|
||||
.with("CREATE TABLE starships(id INTEGER primary key, name VARCHAR(200));")
|
||||
.with("INSERT INTO starships (id, name) VALUES (1,'enterprise-d'), (2, 'defiant'), (3, 'yamato');")
|
||||
.withReplicationSlot()
|
||||
.withPublicationForAllTables();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void tearDown(final TestDestinationEnv testEnv) throws SQLException {
|
||||
testdb.database.query(ctx -> {
|
||||
ctx.execute("SELECT pg_drop_replication_slot('" + slotName + "');");
|
||||
ctx.execute("DROP PUBLICATION " + publication + " CASCADE;");
|
||||
return null;
|
||||
});
|
||||
testdb.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected JsonNode getConfig() {
|
||||
return config;
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.withSchemas(NAMESPACE)
|
||||
.withoutSsl()
|
||||
.withCdcReplication()
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -5,21 +5,17 @@
|
||||
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.Database;
|
||||
import io.airbyte.cdk.db.jdbc.JdbcUtils;
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDataHolder;
|
||||
import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv;
|
||||
import io.airbyte.cdk.testutils.PostgresTestDatabase;
|
||||
import io.airbyte.commons.features.FeatureFlags;
|
||||
import io.airbyte.commons.features.FeatureFlagsWrapper;
|
||||
import io.airbyte.commons.json.Jsons;
|
||||
import io.airbyte.integrations.source.postgres.PostgresTestDatabase;
|
||||
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.sql.SQLException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
@@ -27,10 +23,7 @@ import java.util.Set;
|
||||
public class CdcWalLogsPostgresSourceDatatypeTest extends AbstractPostgresSourceDatatypeTest {
|
||||
|
||||
private static final String SCHEMA_NAME = "test";
|
||||
private static final int INITIAL_WAITING_SECONDS = 30;
|
||||
private JsonNode stateAfterFirstSync;
|
||||
private String slotName;
|
||||
private String publication;
|
||||
|
||||
@Override
|
||||
protected List<AirbyteMessage> runRead(final ConfiguredAirbyteCatalog configuredCatalog) throws Exception {
|
||||
@@ -43,7 +36,6 @@ public class CdcWalLogsPostgresSourceDatatypeTest extends AbstractPostgresSource
|
||||
@Override
|
||||
protected void postSetup() throws Exception {
|
||||
final Database database = setupDatabase();
|
||||
initTests();
|
||||
for (final TestDataHolder test : testDataHolders) {
|
||||
database.query(ctx -> {
|
||||
ctx.fetch(test.getCreateSqlQuery());
|
||||
@@ -78,61 +70,29 @@ public class CdcWalLogsPostgresSourceDatatypeTest extends AbstractPostgresSource
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Database setupDatabase() throws Exception {
|
||||
testdb = PostgresTestDatabase.make("postgres:16-bullseye", "withConf");
|
||||
slotName = testdb.withSuffix("debezium_slot");
|
||||
publication = testdb.withSuffix("publication");
|
||||
|
||||
/**
|
||||
* The publication is not being set as part of the config and because of it
|
||||
* {@link io.airbyte.integrations.source.postgres.PostgresSource#isCdc(JsonNode)} returns false, as
|
||||
* a result no test in this class runs through the cdc path.
|
||||
*/
|
||||
final JsonNode replicationMethod = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("method", "CDC")
|
||||
.put("replication_slot", slotName)
|
||||
.put("publication", publication)
|
||||
.put("initial_waiting_seconds", INITIAL_WAITING_SECONDS)
|
||||
.build());
|
||||
config = Jsons.jsonNode(testdb.makeConfigBuilder()
|
||||
.put(JdbcUtils.SCHEMAS_KEY, List.of(SCHEMA_NAME))
|
||||
.put("replication_method", replicationMethod)
|
||||
.put("is_test", true)
|
||||
.put(JdbcUtils.SSL_KEY, false)
|
||||
.build());
|
||||
|
||||
testdb.database.query(ctx -> {
|
||||
ctx.execute(
|
||||
"SELECT pg_create_logical_replication_slot('" + slotName + "', 'pgoutput');");
|
||||
ctx.execute("CREATE PUBLICATION " + publication + " FOR ALL TABLES;");
|
||||
ctx.execute("CREATE EXTENSION hstore;");
|
||||
return null;
|
||||
});
|
||||
|
||||
testdb.database.query(ctx -> ctx.fetch("CREATE SCHEMA TEST;"));
|
||||
testdb.database.query(ctx -> ctx.fetch("CREATE TYPE mood AS ENUM ('sad', 'ok', 'happy');"));
|
||||
testdb.database.query(ctx -> ctx.fetch("CREATE TYPE inventory_item AS (\n"
|
||||
+ " name text,\n"
|
||||
+ " supplier_id integer,\n"
|
||||
+ " price numeric\n"
|
||||
+ ");"));
|
||||
|
||||
testdb.database.query(ctx -> ctx.fetch("SET TIMEZONE TO 'MST'"));
|
||||
return testdb.database;
|
||||
protected Database setupDatabase() {
|
||||
testdb = PostgresTestDatabase.in("postgres:16-bullseye", "withConf")
|
||||
.with("CREATE EXTENSION hstore;")
|
||||
.with("CREATE SCHEMA TEST;")
|
||||
.with("CREATE TYPE mood AS ENUM ('sad', 'ok', 'happy');")
|
||||
.with("CREATE TYPE inventory_item AS (\n"
|
||||
+ " name text,\n"
|
||||
+ " supplier_id integer,\n"
|
||||
+ " price numeric\n"
|
||||
+ ");")
|
||||
.with("SET TIMEZONE TO 'MST'")
|
||||
.withReplicationSlot()
|
||||
.withPublicationForAllTables();
|
||||
return testdb.getDatabase();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void tearDown(TestDestinationEnv testEnv) throws SQLException {
|
||||
testdb.database.query(ctx -> {
|
||||
ctx.execute("SELECT pg_drop_replication_slot('" + slotName + "');");
|
||||
ctx.execute("DROP PUBLICATION " + publication + " CASCADE;");
|
||||
return null;
|
||||
});
|
||||
super.tearDown(testEnv);
|
||||
}
|
||||
|
||||
public boolean testCatalog() {
|
||||
return true;
|
||||
protected JsonNode getConfig() throws Exception {
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.withSchemas(SCHEMA_NAME)
|
||||
.withoutSsl()
|
||||
.withCdcReplication()
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -7,16 +7,15 @@ package io.airbyte.integrations.io.airbyte.integration_tests.sources;
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.airbyte.cdk.db.PostgresUtils;
|
||||
import io.airbyte.cdk.integrations.base.adaptive.AdaptiveSourceRunner;
|
||||
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.cdk.testutils.PostgresTestDatabase;
|
||||
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.postgres.PostgresTestDatabase;
|
||||
import io.airbyte.protocol.models.Field;
|
||||
import io.airbyte.protocol.models.JsonSchemaType;
|
||||
import io.airbyte.protocol.models.v0.CatalogHelpers;
|
||||
@@ -36,10 +35,8 @@ public class CloudDeploymentPostgresSourceAcceptanceTest extends SourceAcceptanc
|
||||
private static final String SCHEMA_NAME = "public";
|
||||
|
||||
private PostgresTestDatabase testdb;
|
||||
private JsonNode config;
|
||||
|
||||
protected static final String PASSWORD = "Passw0rd";
|
||||
protected static PostgresUtils.Certificate certs;
|
||||
|
||||
@Override
|
||||
protected FeatureFlags featureFlags() {
|
||||
@@ -52,23 +49,8 @@ public class CloudDeploymentPostgresSourceAcceptanceTest extends SourceAcceptanc
|
||||
|
||||
@Override
|
||||
protected void setupEnvironment(final TestDestinationEnv environment) throws Exception {
|
||||
testdb = PostgresTestDatabase.make("postgres:16-bullseye", "withCert");
|
||||
certs = testdb.getCertificate();
|
||||
final JsonNode replicationMethod = Jsons.jsonNode(ImmutableMap.builder()
|
||||
.put("method", "Standard")
|
||||
.build());
|
||||
config = Jsons.jsonNode(testdb.makeConfigBuilder()
|
||||
.put("replication_method", replicationMethod)
|
||||
.put("ssl_mode", ImmutableMap.builder()
|
||||
.put("mode", "verify-ca")
|
||||
.put("ca_certificate", certs.getCaCertificate())
|
||||
.put("client_certificate", certs.getClientCertificate())
|
||||
.put("client_key", certs.getClientKey())
|
||||
.put("client_key_password", PASSWORD)
|
||||
.build())
|
||||
.build());
|
||||
|
||||
testdb.database.query(ctx -> {
|
||||
testdb = PostgresTestDatabase.in("postgres:16-bullseye", "withCert");
|
||||
testdb.query(ctx -> {
|
||||
ctx.fetch("CREATE TABLE id_and_name(id INTEGER, name VARCHAR(200));");
|
||||
ctx.fetch("INSERT INTO id_and_name (id, name) VALUES (1,'picard'), (2, 'crusher'), (3, 'vash');");
|
||||
ctx.fetch("CREATE TABLE starships(id INTEGER, name VARCHAR(200));");
|
||||
@@ -96,7 +78,17 @@ public class CloudDeploymentPostgresSourceAcceptanceTest extends SourceAcceptanc
|
||||
|
||||
@Override
|
||||
protected JsonNode getConfig() {
|
||||
return config;
|
||||
final var certs = testdb.getCertificates();
|
||||
return testdb.integrationTestConfigBuilder()
|
||||
.withStandardReplication()
|
||||
.withSsl(ImmutableMap.builder()
|
||||
.put("mode", "verify-ca")
|
||||
.put("ca_certificate", certs.caCertificate())
|
||||
.put("client_certificate", certs.clientCertificate())
|
||||
.put("client_key", certs.clientKey())
|
||||
.put("client_key_password", PASSWORD)
|
||||
.build())
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user