DBZ-4726: Remove SQL Server single-partition mode

This commit is contained in:
Sergei Morozov 2022-07-06 15:29:15 -07:00 committed by Jiri Pechanec
parent 383333a50c
commit 77f853f936
31 changed files with 507 additions and 686 deletions

View File

@ -103,7 +103,6 @@ public class SqlServerConnection extends JdbcConnection {
private static final String URL_PATTERN = "jdbc:sqlserver://${" + JdbcConfiguration.HOSTNAME + "}:${" + JdbcConfiguration.PORT + "}";
private final boolean multiPartitionMode;
private final String getAllChangesForTable;
private final int queryFetchSize;
@ -122,8 +121,8 @@ public class SqlServerConnection extends JdbcConnection {
*/
public SqlServerConnection(JdbcConfiguration config, SourceTimestampMode sourceTimestampMode,
SqlServerValueConverters valueConverters, Supplier<ClassLoader> classLoaderSupplier,
Set<Envelope.Operation> skippedOperations, boolean multiPartitionMode) {
super(config, createConnectionFactory(multiPartitionMode), classLoaderSupplier, OPENING_QUOTING_CHARACTER, CLOSING_QUOTING_CHARACTER);
Set<Envelope.Operation> skippedOperations) {
super(config, createConnectionFactory(), classLoaderSupplier, OPENING_QUOTING_CHARACTER, CLOSING_QUOTING_CHARACTER);
defaultValueConverter = new SqlServerDefaultValueConverter(this::connection, valueConverters);
this.queryFetchSize = config().getInteger(CommonConnectorConfig.QUERY_FETCH_SIZE);
@ -159,7 +158,6 @@ public SqlServerConnection(JdbcConfiguration config, SourceTimestampMode sourceT
getAllChangesForTable = get_all_changes_for_table.replaceFirst(STATEMENTS_PLACEHOLDER,
Matcher.quoteReplacement(sourceTimestampMode.lsnTimestampSelectStatement()));
this.multiPartitionMode = multiPartitionMode;
this.optionRecompile = false;
}
@ -176,23 +174,14 @@ public SqlServerConnection(JdbcConfiguration config, SourceTimestampMode sourceT
*/
public SqlServerConnection(JdbcConfiguration config, SourceTimestampMode sourceTimestampMode,
SqlServerValueConverters valueConverters, Supplier<ClassLoader> classLoaderSupplier,
Set<Envelope.Operation> skippedOperations, boolean multiPartitionMode, boolean optionRecompile) {
this(config, sourceTimestampMode, valueConverters, classLoaderSupplier, skippedOperations, multiPartitionMode);
Set<Envelope.Operation> skippedOperations, boolean optionRecompile) {
this(config, sourceTimestampMode, valueConverters, classLoaderSupplier, skippedOperations);
this.optionRecompile = optionRecompile;
}
private static String createUrlPattern(boolean multiPartitionMode) {
String pattern = URL_PATTERN;
if (!multiPartitionMode) {
pattern += ";databaseName=${" + JdbcConfiguration.DATABASE + "}";
}
return pattern;
}
private static ConnectionFactory createConnectionFactory(boolean multiPartitionMode) {
return JdbcConnection.patternBasedFactory(createUrlPattern(multiPartitionMode),
private static ConnectionFactory createConnectionFactory() {
return JdbcConnection.patternBasedFactory(URL_PATTERN,
SQLServerDriver.class.getName(),
SqlServerConnection.class.getClassLoader(),
JdbcConfiguration.PORT.withDefault(SqlServerConnectorConfig.PORT.defaultValueAsString()));
@ -204,7 +193,7 @@ private static ConnectionFactory createConnectionFactory(boolean multiPartitionM
* @return a {@code String} where the variables in {@code urlPattern} are replaced with values from the configuration
*/
public String connectionString() {
return connectionString(createUrlPattern(multiPartitionMode));
return connectionString(URL_PATTERN);
}
@Override
@ -349,11 +338,11 @@ public Lsn incrementLsn(String databaseName, Lsn lsn) throws SQLException {
* access to CDC table.
*
* @return boolean indicating the presence/absence of access
* @throws SQLException
*/
public boolean checkIfConnectedUserHasAccessToCDCTable() throws SQLException {
public boolean checkIfConnectedUserHasAccessToCDCTable(String databaseName) throws SQLException {
final AtomicBoolean userHasAccess = new AtomicBoolean();
this.query("EXEC sys.sp_cdc_help_change_data_capture", rs -> userHasAccess.set(rs.next()));
final String query = replaceDatabaseNamePlaceholder("EXEC [#db].sys.sp_cdc_help_change_data_capture", databaseName);
this.query(query, rs -> userHasAccess.set(rs.next()));
return userHasAccess.get();
}

View File

@ -6,7 +6,6 @@
package io.debezium.connector.sqlserver;
import static io.debezium.config.CommonConnectorConfig.TASK_ID;
import static io.debezium.connector.sqlserver.SqlServerConnectorConfig.DATABASE_NAME;
import static io.debezium.connector.sqlserver.SqlServerConnectorConfig.DATABASE_NAMES;
import java.sql.SQLException;
@ -71,7 +70,6 @@ public List<Map<String, String>> taskConfigs(int maxTasks) {
private List<Map<String, String>> buildTaskConfigs(SqlServerConnection connection, SqlServerConnectorConfig config,
int maxTasks) {
final boolean multiPartitionMode = config.isMultiPartitionModeEnabled();
List<String> databaseNames = config.getDatabaseNames();
// Initialize the database list for each task
@ -92,13 +90,8 @@ private List<Map<String, String>> buildTaskConfigs(SqlServerConnection connectio
for (int taskIndex = 0; taskIndex < maxTasks; taskIndex++) {
String taskDatabases = String.join(",", databasesByTask.get(taskIndex));
Map<String, String> taskProperties = new HashMap<>(properties);
if (multiPartitionMode) {
taskProperties.put(SqlServerConnectorConfig.DATABASE_NAMES.name(), taskDatabases);
taskProperties.put(TASK_ID, String.valueOf(taskIndex));
}
else {
taskProperties.put(SqlServerConnectorConfig.DATABASE_NAME.name(), taskDatabases);
}
taskConfigs.add(Collections.unmodifiableMap(taskProperties));
}
@ -116,8 +109,7 @@ public ConfigDef config() {
@Override
protected void validateConnection(Map<String, ConfigValue> configValues, Configuration config) {
if (!configValues.get(DATABASE_NAME.name()).errorMessages().isEmpty()
|| !configValues.get(DATABASE_NAMES.name()).errorMessages().isEmpty()) {
if (!configValues.get(DATABASE_NAMES.name()).errorMessages().isEmpty()) {
return;
}
@ -130,14 +122,22 @@ protected void validateConnection(Map<String, ConfigValue> configValues, Configu
LOGGER.debug("Successfully tested connection for {} with user '{}'", connection.connectionString(),
connection.username());
LOGGER.info("Checking if user has access to CDC table");
boolean userHasAccessToCDCTable = connection.checkIfConnectedUserHasAccessToCDCTable();
if (!userHasAccessToCDCTable
&& sqlServerConfig.getSnapshotMode() != SqlServerConnectorConfig.SnapshotMode.INITIAL_ONLY) {
String errorMessage = "User " + userValue.value() + " does not have access to CDC table and can only be used in initial_only snapshot mode";
if (sqlServerConfig.getSnapshotMode() != SqlServerConnectorConfig.SnapshotMode.INITIAL_ONLY) {
final List<String> noAccessDatabaseNames = new ArrayList<>();
for (String databaseName : sqlServerConfig.getDatabaseNames()) {
if (!connection.checkIfConnectedUserHasAccessToCDCTable(databaseName)) {
noAccessDatabaseNames.add(databaseName);
}
}
if (!noAccessDatabaseNames.isEmpty()) {
String errorMessage = String.format(
"User %s does not have access to CDC schema in the following databases: %s. This user can only be used in initial_only snapshot mode",
userValue.value(), String.join(", ", noAccessDatabaseNames));
LOGGER.error(errorMessage);
userValue.addErrorMessage(errorMessage);
}
}
}
catch (Exception e) {
LOGGER.error("Failed testing connection for {} with user '{}'", config.withMaskedPasswords(),
userValue, e);
@ -155,7 +155,6 @@ private SqlServerConnection connect(SqlServerConnectorConfig sqlServerConfig) {
return new SqlServerConnection(sqlServerConfig.getJdbcConfig(),
sqlServerConfig.getSourceTimestampMode(), null,
() -> getClass().getClassLoader(),
Collections.emptySet(),
sqlServerConfig.isMultiPartitionModeEnabled());
Collections.emptySet());
}
}

View File

@ -233,10 +233,6 @@ public static SnapshotIsolationMode parse(String value, String defaultValue) {
.withValidation(Field::isOptional)
.withDescription("The SQL Server instance name");
public static final Field DATABASE_NAME = RelationalDatabaseConnectorConfig.DATABASE_NAME
.withNoValidation()
.withValidation(SqlServerConnectorConfig::validateDatabaseName);
public static final Field DATABASE_NAMES = Field.create(DATABASE_CONFIG_PREFIX + "names")
.withDisplayName("Databases")
.withType(Type.LIST)
@ -316,7 +312,6 @@ public static SnapshotIsolationMode parse(String value, String defaultValue) {
private static final ConfigDefinition CONFIG_DEFINITION = HistorizedRelationalDatabaseConnectorConfig.CONFIG_DEFINITION.edit()
.name("SQL Server")
.type(
DATABASE_NAME,
DATABASE_NAMES,
HOSTNAME,
PORT,
@ -354,27 +349,18 @@ public static ConfigDef configDef() {
private final SourceTimestampMode sourceTimestampMode;
private final boolean readOnlyDatabaseConnection;
private final int maxTransactionsPerIteration;
private final boolean multiPartitionMode;
private final boolean optionRecompile;
public SqlServerConnectorConfig(Configuration config) {
super(SqlServerConnector.class, config, config.getString(SERVER_NAME), new SystemTablesPredicate(), x -> x.schema() + "." + x.table(), true,
ColumnFilterMode.SCHEMA, config.hasKey(DATABASE_NAMES));
ColumnFilterMode.SCHEMA, true);
final String databaseName = config.getString(DATABASE_NAME.name());
final String databaseNames = config.getString(DATABASE_NAMES.name());
if (databaseName != null) {
multiPartitionMode = false;
this.databaseNames = Collections.singletonList(databaseName);
}
else if (databaseNames != null) {
multiPartitionMode = true;
if (databaseNames != null) {
this.databaseNames = Arrays.asList(databaseNames.split(","));
LOGGER.info("Multi-partition mode is enabled");
}
else {
multiPartitionMode = false;
this.databaseNames = Collections.emptyList();
}
@ -408,10 +394,6 @@ public String getInstanceName() {
return instanceName;
}
public boolean isMultiPartitionModeEnabled() {
return multiPartitionMode;
}
public SnapshotIsolationMode getSnapshotIsolationMode() {
return this.snapshotIsolationMode;
}
@ -501,28 +483,13 @@ public Map<TableId, String> getSnapshotSelectOverridesByTable() {
return Collections.unmodifiableMap(snapshotSelectOverridesByTable);
}
private static int validateDatabaseName(Configuration config, Field field, Field.ValidationOutput problems) {
if (!config.hasKey(field) && !config.hasKey(DATABASE_NAMES)) {
problems.accept(field, null, "Either " + DATABASE_NAME + " or " + DATABASE_NAMES + " must be specified");
return 1;
}
return 0;
}
private static int validateDatabaseNames(Configuration config, Field field, Field.ValidationOutput problems) {
String databaseNames = config.getString(field);
int count = 0;
if (databaseNames != null) {
if (config.hasKey(DATABASE_NAME)) {
problems.accept(field, null, "Cannot be specified alongside " + DATABASE_NAME);
++count;
}
if (databaseNames.split(",").length == 0) {
if (databaseNames == null || databaseNames.split(",").length == 0) {
problems.accept(field, databaseNames, "Cannot be empty");
++count;
}
}
return count;
}

View File

@ -23,8 +23,6 @@
import io.debezium.pipeline.DataChangeEvent;
import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.metrics.DefaultChangeEventSourceMetricsFactory;
import io.debezium.pipeline.metrics.spi.ChangeEventSourceMetricsFactory;
import io.debezium.pipeline.spi.Offsets;
import io.debezium.relational.TableId;
import io.debezium.spi.topic.TopicNamingStrategy;
@ -66,18 +64,18 @@ public ChangeEventSourceCoordinator<SqlServerPartition, SqlServerOffsetContext>
.build();
final SqlServerConnectorConfig connectorConfig = new SqlServerConnectorConfig(config);
final boolean multiPartitionMode = connectorConfig.isMultiPartitionModeEnabled();
final TopicNamingStrategy topicNamingStrategy = connectorConfig.getTopicNamingStrategy(CommonConnectorConfig.TOPIC_NAMING_STRATEGY, multiPartitionMode);
final TopicNamingStrategy topicNamingStrategy = connectorConfig.getTopicNamingStrategy(
CommonConnectorConfig.TOPIC_NAMING_STRATEGY, true);
final SchemaNameAdjuster schemaNameAdjuster = connectorConfig.schemaNameAdjustmentMode().createAdjuster();
final SqlServerValueConverters valueConverters = new SqlServerValueConverters(connectorConfig.getDecimalMode(),
connectorConfig.getTemporalPrecisionMode(), connectorConfig.binaryHandlingMode());
dataConnection = new SqlServerConnection(connectorConfig.getJdbcConfig(), connectorConfig.getSourceTimestampMode(), valueConverters,
() -> getClass().getClassLoader(),
connectorConfig.getSkippedOperations(), multiPartitionMode, connectorConfig.getOptionRecompile());
connectorConfig.getSkippedOperations(), connectorConfig.getOptionRecompile());
metadataConnection = new SqlServerConnection(connectorConfig.getJdbcConfig(), connectorConfig.getSourceTimestampMode(), valueConverters,
() -> getClass().getClassLoader(),
connectorConfig.getSkippedOperations(), multiPartitionMode);
connectorConfig.getSkippedOperations());
this.schema = new SqlServerDatabaseSchema(connectorConfig, metadataConnection.getDefaultValueConverter(), valueConverters, topicNamingStrategy,
schemaNameAdjuster);
@ -120,7 +118,7 @@ public ChangeEventSourceCoordinator<SqlServerPartition, SqlServerOffsetContext>
SqlServerConnector.class,
connectorConfig,
new SqlServerChangeEventSourceFactory(connectorConfig, dataConnection, metadataConnection, errorHandler, dispatcher, clock, schema),
createChangeEventSourceMetricsFactory(multiPartitionMode, offsets),
new SqlServerMetricsFactory(offsets.getPartitions()),
dispatcher,
schema,
clock);
@ -170,14 +168,4 @@ protected void doStop() {
protected Iterable<Field> getAllConfigurationFields() {
return SqlServerConnectorConfig.ALL_FIELDS;
}
private ChangeEventSourceMetricsFactory<SqlServerPartition> createChangeEventSourceMetricsFactory(boolean multiPartitionMode,
Offsets<SqlServerPartition, SqlServerOffsetContext> offsets) {
if (multiPartitionMode) {
return new SqlServerMetricsFactory(offsets.getPartitions());
}
else {
return new DefaultChangeEventSourceMetricsFactory<>();
}
}
}

View File

@ -40,7 +40,7 @@ public SqlServerDatabaseSchema(SqlServerConnectorConfig connectorConfig, SqlServ
connectorConfig.customConverterRegistry(),
connectorConfig.getSourceInfoStructMaker().schema(),
connectorConfig.getSanitizeFieldNames(),
connectorConfig.isMultiPartitionModeEnabled()),
true),
false, connectorConfig.getKeyMapper());
}

View File

@ -5,12 +5,9 @@
*/
package io.debezium.connector.sqlserver;
import static io.debezium.connector.sqlserver.SqlServerConnectorConfig.DATABASE_NAME;
import static io.debezium.connector.sqlserver.SqlServerConnectorConfig.DATABASE_NAMES;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
@ -29,14 +26,11 @@ public class SqlServerPartition extends AbstractPartition implements Partition {
private final Map<String, String> sourcePartition;
private final int hashCode;
public SqlServerPartition(String serverName, String databaseName, boolean multiPartitionMode) {
public SqlServerPartition(String serverName, String databaseName) {
super(databaseName);
this.serverName = serverName;
this.sourcePartition = Collect.hashMapOf(SERVER_PARTITION_KEY, serverName);
if (multiPartitionMode) {
this.sourcePartition.put(DATABASE_PARTITION_KEY, databaseName);
}
this.sourcePartition = Collect.hashMapOf(SERVER_PARTITION_KEY, serverName, DATABASE_PARTITION_KEY, databaseName);
this.hashCode = Objects.hash(serverName, databaseName);
}
@ -87,19 +81,9 @@ static class Provider implements Partition.Provider<SqlServerPartition> {
@Override
public Set<SqlServerPartition> getPartitions() {
String serverName = connectorConfig.getLogicalName();
boolean multiPartitionMode = connectorConfig.isMultiPartitionModeEnabled();
List<String> databaseNames;
if (multiPartitionMode) {
databaseNames = Arrays.asList(taskConfig.getString(DATABASE_NAMES.name()).split(","));
}
else {
databaseNames = Collections.singletonList(taskConfig.getString(DATABASE_NAME.name()));
}
return databaseNames.stream()
.map(databaseName -> new SqlServerPartition(serverName, databaseName, multiPartitionMode))
return Arrays.stream(taskConfig.getString(DATABASE_NAMES.name()).split(","))
.map(databaseName -> new SqlServerPartition(serverName, databaseName))
.collect(Collectors.toSet());
}
}

View File

@ -7,7 +7,6 @@
import io.debezium.relational.TableId;
import io.debezium.schema.TopicSelector;
import io.debezium.schema.TopicSelector.DataCollectionTopicNamer;
/**
* The topic naming strategy based on connector configuration and table name
@ -18,14 +17,9 @@
public class SqlServerTopicSelector {
public static TopicSelector<TableId> defaultSelector(SqlServerConnectorConfig connectorConfig) {
DataCollectionTopicNamer<TableId> topicNamer;
if (connectorConfig.isMultiPartitionModeEnabled()) {
topicNamer = (tableId, prefix, delimiter) -> String.join(delimiter, prefix, tableId.catalog(), tableId.schema(), tableId.table());
}
else {
topicNamer = (tableId, prefix, delimiter) -> String.join(delimiter, prefix, tableId.schema(), tableId.table());
}
return TopicSelector.defaultSelector(connectorConfig, topicNamer);
return TopicSelector.defaultSelector(
connectorConfig,
(tableId, prefix, delimiter) -> String.join(delimiter, prefix, tableId.catalog(), tableId.schema(), tableId.table()));
}
}

View File

@ -213,7 +213,7 @@ public void intTypes() throws Exception {
final SourceRecords records = consumeRecordsByTopic(EXPECTED_RECORD_COUNT);
List<SourceRecord> testTableRecords = records.recordsForTopic("server1.dbo.type_int");
List<SourceRecord> testTableRecords = records.recordsForTopic("server1.testDB1.dbo.type_int");
assertThat(testTableRecords).hasSize(1);
// insert
@ -228,7 +228,7 @@ public void fpTypes() throws Exception {
final SourceRecords records = consumeRecordsByTopic(EXPECTED_RECORD_COUNT);
List<SourceRecord> testTableRecords = records.recordsForTopic("server1.dbo.type_fp");
List<SourceRecord> testTableRecords = records.recordsForTopic("server1.testDB1.dbo.type_fp");
assertThat(testTableRecords).hasSize(1);
// insert
@ -243,7 +243,7 @@ public void stringTypes() throws Exception {
final SourceRecords records = consumeRecordsByTopic(EXPECTED_RECORD_COUNT);
List<SourceRecord> testTableRecords = records.recordsForTopic("server1.dbo.type_string");
List<SourceRecord> testTableRecords = records.recordsForTopic("server1.testDB1.dbo.type_string");
assertThat(testTableRecords).hasSize(1);
// insert
@ -258,7 +258,7 @@ public void dateTimeTypes() throws Exception {
final SourceRecords records = consumeRecordsByTopic(EXPECTED_RECORD_COUNT);
List<SourceRecord> testTableRecords = records.recordsForTopic("server1.dbo.type_time");
List<SourceRecord> testTableRecords = records.recordsForTopic("server1.testDB1.dbo.type_time");
assertThat(testTableRecords).hasSize(1);
// insert
@ -276,7 +276,7 @@ public void dateTimeTypesAsConnect() throws Exception {
final SourceRecords records = consumeRecordsByTopic(EXPECTED_RECORD_COUNT);
List<SourceRecord> testTableRecords = records.recordsForTopic("server1.dbo.type_time");
List<SourceRecord> testTableRecords = records.recordsForTopic("server1.testDB1.dbo.type_time");
assertThat(testTableRecords).hasSize(1);
// insert
@ -291,7 +291,7 @@ public void otherTypes() throws Exception {
final SourceRecords records = consumeRecordsByTopic(EXPECTED_RECORD_COUNT);
List<SourceRecord> testTableRecords = records.recordsForTopic("server1.dbo.type_xml");
List<SourceRecord> testTableRecords = records.recordsForTopic("server1.testDB1.dbo.type_xml");
assertThat(testTableRecords).hasSize(1);
// insert

View File

@ -62,7 +62,7 @@ public void caseInsensitiveDatabase() throws Exception {
@FixFor("DBZ-1051")
public void caseSensitiveDatabase() throws Exception {
connection.execute(
"ALTER DATABASE testDB COLLATE Latin1_General_BIN",
"ALTER DATABASE testDB1 COLLATE Latin1_General_BIN",
"CREATE TABLE MyTableOne (Id int primary key, ColA varchar(30))",
"INSERT INTO MyTableOne VALUES(1, 'a')");
TestHelper.enableTableCdc(connection, "MyTableOne");
@ -78,40 +78,40 @@ private void testDatabase() throws Exception {
assertConnectorIsRunning();
SourceRecords records = consumeRecordsByTopic(1);
Assertions.assertThat(records.recordsForTopic("server1.dbo.MyTableOne")).hasSize(1);
SourceRecord record = records.recordsForTopic("server1.dbo.MyTableOne").get(0);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.MyTableOne")).hasSize(1);
SourceRecord record = records.recordsForTopic("server1.testDB1.dbo.MyTableOne").get(0);
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.MyTableOne.Value")
.name("server1.testDB1.dbo.MyTableOne.Value")
.field("Id", Schema.INT32_SCHEMA)
.field("ColA", Schema.OPTIONAL_STRING_SCHEMA)
.build());
assertSchemaMatchesStruct(
(Struct) record.key(),
SchemaBuilder.struct()
.name("server1.dbo.MyTableOne.Key")
.name("server1.testDB1.dbo.MyTableOne.Key")
.field("Id", Schema.INT32_SCHEMA)
.build());
Assertions.assertThat(((Struct) ((Struct) record.value()).get("after")).getInt32("Id")).isEqualTo(1);
connection.execute("INSERT INTO MyTableOne VALUES(2, 'b')");
records = consumeRecordsByTopic(1);
Assertions.assertThat(records.recordsForTopic("server1.dbo.MyTableOne")).hasSize(1);
record = records.recordsForTopic("server1.dbo.MyTableOne").get(0);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.MyTableOne")).hasSize(1);
record = records.recordsForTopic("server1.testDB1.dbo.MyTableOne").get(0);
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.MyTableOne.Value")
.name("server1.testDB1.dbo.MyTableOne.Value")
.field("Id", Schema.INT32_SCHEMA)
.field("ColA", Schema.OPTIONAL_STRING_SCHEMA)
.build());
assertSchemaMatchesStruct(
(Struct) record.key(),
SchemaBuilder.struct()
.name("server1.dbo.MyTableOne.Key")
.name("server1.testDB1.dbo.MyTableOne.Key")
.field("Id", Schema.INT32_SCHEMA)
.build());
Assertions.assertThat(((Struct) ((Struct) record.value()).get("after")).getInt32("Id")).isEqualTo(2);
@ -121,20 +121,20 @@ record = records.recordsForTopic("server1.dbo.MyTableOne").get(0);
TestHelper.enableTableCdc(connection, "MyTableTwo");
connection.execute("INSERT INTO MyTableTwo VALUES(3, 'b')");
records = consumeRecordsByTopic(1);
Assertions.assertThat(records.recordsForTopic("server1.dbo.MyTableTwo")).hasSize(1);
record = records.recordsForTopic("server1.dbo.MyTableTwo").get(0);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.MyTableTwo")).hasSize(1);
record = records.recordsForTopic("server1.testDB1.dbo.MyTableTwo").get(0);
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.MyTableTwo.Value")
.name("server1.testDB1.dbo.MyTableTwo.Value")
.field("Id", Schema.INT32_SCHEMA)
.field("ColB", Schema.OPTIONAL_STRING_SCHEMA)
.build());
assertSchemaMatchesStruct(
(Struct) record.key(),
SchemaBuilder.struct()
.name("server1.dbo.MyTableTwo.Key")
.name("server1.testDB1.dbo.MyTableTwo.Key")
.field("Id", Schema.INT32_SCHEMA)
.build());
Assertions.assertThat(((Struct) ((Struct) record.value()).get("after")).getInt32("Id")).isEqualTo(3);

View File

@ -84,8 +84,8 @@ public void warn() throws Exception {
}
SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).isNull();
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).isNull();
Awaitility.await()
.alias("Found warning message in logs")
@ -120,8 +120,8 @@ public void ignore() throws Exception {
}
SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).isNull();
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).isNull();
}
@Test

View File

@ -75,27 +75,27 @@ protected JdbcConnection databaseConnection() {
@Override
protected String topicName() {
return "server1.dbo.a";
return "server1.testDB1.dbo.a";
}
@Override
protected List<String> topicNames() {
return List.of("server1.dbo.a", "server1.dbo.b");
return List.of("server1.testDB1.dbo.a", "server1.testDB1.dbo.b");
}
@Override
protected String tableName() {
return "testDB.dbo.a";
return "testDB1.dbo.a";
}
@Override
protected List<String> tableNames() {
return List.of("testDB.dbo.a", "testDB.dbo.b");
return List.of("testDB1.dbo.a", "testDB1.dbo.b");
}
@Override
protected String tableName(String table) {
return "testDB.dbo." + table;
return "testDB1.dbo." + table;
}
@Override
@ -149,7 +149,7 @@ protected String createTableStatement(String newTable, String copyTable) {
protected Builder config() {
return TestHelper.defaultConfig()
.with(SqlServerConnectorConfig.SNAPSHOT_MODE, SnapshotMode.SCHEMA_ONLY)
.with(SqlServerConnectorConfig.SIGNAL_DATA_COLLECTION, "testDB.dbo.debezium_signal")
.with(SqlServerConnectorConfig.SIGNAL_DATA_COLLECTION, "testDB1.dbo.debezium_signal")
.with(SqlServerConnectorConfig.INCREMENTAL_SNAPSHOT_CHUNK_SIZE, 250)
.with(SqlServerConnectorConfig.INCREMENTAL_SNAPSHOT_ALLOW_SCHEMA_CHANGES, true);
}
@ -165,7 +165,7 @@ protected Builder mutableConfig(boolean signalTableOnly, boolean storeOnlyCaptur
}
return TestHelper.defaultConfig()
.with(SqlServerConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL)
.with(SqlServerConnectorConfig.SIGNAL_DATA_COLLECTION, "testDB.dbo.debezium_signal")
.with(SqlServerConnectorConfig.SIGNAL_DATA_COLLECTION, "testDB1.dbo.debezium_signal")
.with(SqlServerConnectorConfig.TABLE_INCLUDE_LIST, tableIncludeList)
.with(SqlServerConnectorConfig.INCREMENTAL_SNAPSHOT_CHUNK_SIZE, 250)
.with(SqlServerConnectorConfig.INCREMENTAL_SNAPSHOT_ALLOW_SCHEMA_CHANGES, true)

View File

@ -74,22 +74,22 @@ protected JdbcConnection databaseConnection() {
@Override
protected String topicName() {
return "server1.dbo.a";
return "server1.testDB1.dbo.a";
}
@Override
public List<String> topicNames() {
return List.of("server1.dbo.a", "server1.dbo.b");
return List.of("server1.testDB1.dbo.a", "server1.testDB1.dbo.b");
}
@Override
protected String tableName() {
return "testDB.dbo.a";
return "testDB1.dbo.a";
}
@Override
protected List<String> tableNames() {
return List.of("testDB.dbo.a", "testDB.dbo.b");
return List.of("testDB1.dbo.a", "testDB1.dbo.b");
}
@Override
@ -101,7 +101,7 @@ protected String signalTableName() {
protected Builder config() {
return TestHelper.defaultConfig()
.with(SqlServerConnectorConfig.SNAPSHOT_MODE, SnapshotMode.SCHEMA_ONLY)
.with(SqlServerConnectorConfig.SIGNAL_DATA_COLLECTION, "testDB.dbo.debezium_signal")
.with(SqlServerConnectorConfig.SIGNAL_DATA_COLLECTION, "testDB1.dbo.debezium_signal")
.with(SqlServerConnectorConfig.INCREMENTAL_SNAPSHOT_OPTION_RECOMPILE, true);
}
@ -116,7 +116,7 @@ protected Builder mutableConfig(boolean signalTableOnly, boolean storeOnlyCaptur
}
return TestHelper.defaultConfig()
.with(SqlServerConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL)
.with(SqlServerConnectorConfig.SIGNAL_DATA_COLLECTION, "testDB.dbo.debezium_signal")
.with(SqlServerConnectorConfig.SIGNAL_DATA_COLLECTION, "testDB1.dbo.debezium_signal")
.with(SqlServerConnectorConfig.INCREMENTAL_SNAPSHOT_OPTION_RECOMPILE, true)
.with(SqlServerConnectorConfig.TABLE_INCLUDE_LIST, tableIncludeList)
.with(DatabaseHistory.STORE_ONLY_CAPTURED_TABLES_DDL, storeOnlyCapturedDdl);

View File

@ -88,7 +88,7 @@ public void decimalModeConfigString() throws Exception {
connection.execute("INSERT INTO tablenuma VALUES (111.1111, 1111111, 1111111.1, 1111111 );");
final SourceRecords records = consumeRecordsByTopic(1);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.tablenuma");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.tablenuma");
Assertions.assertThat(tableA).hasSize(1);
final Struct valueA = (Struct) tableA.get(0).value();
assertSchema(valueA, Schema.OPTIONAL_STRING_SCHEMA);
@ -120,7 +120,7 @@ public void decimalModeConfigDouble() throws Exception {
connection.execute("INSERT INTO tablenumb VALUES (222.2222, 22222, 22222.2, 2222222 );");
final SourceRecords records = consumeRecordsByTopic(1);
final List<SourceRecord> results = records.recordsForTopic("server1.dbo.tablenumb");
final List<SourceRecord> results = records.recordsForTopic("server1.testDB1.dbo.tablenumb");
Assertions.assertThat(results).hasSize(1);
final Struct valueA = (Struct) results.get(0).value();
assertSchema(valueA, Schema.OPTIONAL_FLOAT64_SCHEMA);
@ -151,7 +151,7 @@ public void decimalModeConfigPrecise() throws Exception {
connection.execute("INSERT INTO tablenumc VALUES (333.3333, 3333, 3333.3, 33333333 );");
final SourceRecords records = consumeRecordsByTopic(1);
final List<SourceRecord> results = records.recordsForTopic("server1.dbo.tablenumc");
final List<SourceRecord> results = records.recordsForTopic("server1.testDB1.dbo.tablenumc");
Assertions.assertThat(results).hasSize(1);
final Struct valueA = (Struct) results.get(0).value();
Assertions.assertThat(valueA.schema().field("after").schema().field("cola").schema())

View File

@ -8,9 +8,7 @@
import static org.fest.assertions.Assertions.assertThat;
import java.sql.SQLException;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.kafka.connect.data.Schema;
@ -98,7 +96,7 @@ public void streamingSchemaChanges() throws Exception {
Assertions.assertThat(schemaRecords).hasSize(3);
schemaRecords.forEach(record -> {
Assertions.assertThat(record.topic()).isEqualTo("server1");
Assertions.assertThat(((Struct) record.key()).getString("databaseName")).isEqualTo("testDB");
Assertions.assertThat(((Struct) record.key()).getString("databaseName")).isEqualTo("testDB1");
Assertions.assertThat(record.sourceOffset().get("snapshot")).isEqualTo(true);
});
Assertions.assertThat(((Struct) schemaRecords.get(0).value()).getStruct("source").getString("snapshot")).isEqualTo("true");
@ -112,8 +110,8 @@ public void streamingSchemaChanges() throws Exception {
waitForAvailableRecords(TestHelper.waitTimeForRecords(), TimeUnit.SECONDS);
records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES, 24);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
final List<SourceRecord> tablebRecords = records.recordsForTopic("server1.dbo.tableb");
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
final List<SourceRecord> tablebRecords = records.recordsForTopic("server1.testDB1.dbo.tableb");
// Additional schema change record was emitted
if (tablebRecords.size() == RECORDS_PER_TABLE - 1) {
tablebRecords.add(consumeRecord());
@ -124,7 +122,7 @@ public void streamingSchemaChanges() throws Exception {
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tableb.Value")
.name("server1.testDB1.dbo.tableb.Value")
.field("id", Schema.INT32_SCHEMA)
.field("colb", Schema.OPTIONAL_STRING_SCHEMA)
.build());
@ -152,7 +150,7 @@ public void streamingSchemaChanges() throws Exception {
Assertions.assertThat(records.allRecordsInOrder()).hasSize(1);
final SourceRecord schemaRecord = records.allRecordsInOrder().get(0);
Assertions.assertThat(schemaRecord.topic()).isEqualTo("server1");
Assertions.assertThat(((Struct) schemaRecord.key()).getString("databaseName")).isEqualTo("testDB");
Assertions.assertThat(((Struct) schemaRecord.key()).getString("databaseName")).isEqualTo("testDB1");
Assertions.assertThat(schemaRecord.sourceOffset().get("snapshot")).isNull();
Assertions.assertThat(((Struct) schemaRecord.value()).getStruct("source").getString("snapshot")).isNull();
@ -163,15 +161,15 @@ public void streamingSchemaChanges() throws Exception {
Assertions.assertThat(lastUpdate.sourceOffset()).isEqualTo(schemaRecord.sourceOffset());
records = consumeRecordsByTopic(RECORDS_PER_TABLE * 2);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.dbo.tableb").forEach(record -> {
records.recordsForTopic("server1.testDB1.dbo.tableb").forEach(record -> {
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tableb.Value")
.name("server1.testDB1.dbo.tableb.Value")
.field("id", Schema.INT32_SCHEMA)
.field("newcolb", Schema.OPTIONAL_STRING_SCHEMA)
.build());
@ -185,14 +183,14 @@ public void streamingSchemaChanges() throws Exception {
"INSERT INTO tableb VALUES(" + id + ", 'b3')");
}
records = consumeRecordsByTopic(RECORDS_PER_TABLE * 2);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.dbo.tableb").forEach(record -> {
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.testDB1.dbo.tableb").forEach(record -> {
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tableb.Value")
.name("server1.testDB1.dbo.tableb.Value")
.field("id", Schema.INT32_SCHEMA)
.field("newcolb", Schema.OPTIONAL_STRING_SCHEMA)
.build());
@ -230,7 +228,7 @@ public void snapshotSchemaChanges() throws Exception {
Assertions.assertThat(schemaRecords).hasSize(3);
schemaRecords.forEach(record -> {
Assertions.assertThat(record.topic()).isEqualTo("server1");
Assertions.assertThat(((Struct) record.key()).getString("databaseName")).isEqualTo("testDB");
Assertions.assertThat(((Struct) record.key()).getString("databaseName")).isEqualTo("testDB1");
Assertions.assertThat(record.sourceOffset().get("snapshot")).isEqualTo(true);
});
Assertions.assertThat(((Struct) schemaRecords.get(0).value()).getStruct("source").getString("snapshot")).isEqualTo("true");
@ -242,14 +240,14 @@ public void snapshotSchemaChanges() throws Exception {
Assertions.assertThat(tableChanges.get(0).get("type")).isEqualTo("CREATE");
records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.dbo.tableb").forEach(record -> {
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.testDB1.dbo.tableb").forEach(record -> {
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tableb.Value")
.name("server1.testDB1.dbo.tableb.Value")
.field("id", Schema.INT32_SCHEMA)
.field("colb", Schema.OPTIONAL_STRING_SCHEMA)
.build());
@ -282,7 +280,7 @@ public void schemaChangeAfterSnapshot() throws Exception {
// 1 schema event + 1 data event
Testing.Print.enable();
SourceRecords records = consumeRecordsByTopic(1 + 1);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablec")).hasSize(1);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablec")).hasSize(1);
stopConnector();
assertConnectorNotRunning();
@ -307,7 +305,7 @@ public void schemaChangeAfterSnapshot() throws Exception {
// 1-2 schema events + 1 data event
records = consumeRecordsByTopic(2 + 1);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tabled")).hasSize(1);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tabled")).hasSize(1);
final List<SourceRecord> schemaEvents = records.recordsForTopic("server1");
@ -322,30 +320,13 @@ public void schemaChangeAfterSnapshot() throws Exception {
Assertions.assertThat(((Struct) schemaEventD.value()).getStruct("source").getString("table")).isEqualTo("tabled");
}
@Test
@FixFor("DBZ-3347")
public void shouldContainPartitionInSchemaChangeEventInSinglePartitionMode() throws Exception {
shouldContainPartitionInSchemaChangeEvent(TestHelper.defaultConfig(),
TestHelper::waitForStreamingStarted,
Collections.singletonMap("server", "server1"));
}
@Test
@FixFor({ "DBZ-3347", "DBZ-2975" })
public void shouldContainPartitionInSchemaChangeEventInMultiPartitionMode() throws Exception {
shouldContainPartitionInSchemaChangeEvent(TestHelper.defaultMultiPartitionConfig(),
TestHelper::waitForTaskStreamingStarted,
Collect.hashMapOf("server", "server1", "database", "testDB"));
}
private void shouldContainPartitionInSchemaChangeEvent(Configuration.Builder configBuilder,
Runnable waitForStreamingStarted,
Map<String, String> expectedPartition)
throws Exception {
public void shouldContainPartitionInSchemaChangeEvent() throws Exception {
connection.execute("create table dbz3347 (id int primary key, data varchar(50))");
TestHelper.enableTableCdc(connection, "dbz3347");
Configuration config = configBuilder
Configuration config = TestHelper.defaultConfig()
.with(SqlServerConnectorConfig.TABLE_INCLUDE_LIST, "dbo\\.dbz3347")
.with(SqlServerConnectorConfig.INCLUDE_SCHEMA_CHANGES, true)
.build();
@ -353,10 +334,10 @@ private void shouldContainPartitionInSchemaChangeEvent(Configuration.Builder con
start(SqlServerConnector.class, config);
assertConnectorIsRunning();
waitForStreamingStarted.run();
TestHelper.waitForStreamingStarted();
SourceRecords schemaChanges = consumeRecordsByTopic(1);
SourceRecord change = schemaChanges.recordsForTopic("server1").get(0);
assertThat(change.sourcePartition()).isEqualTo(expectedPartition);
assertThat(change.sourcePartition()).isEqualTo(Collect.hashMapOf("server", "server1", "database", "testDB1"));
}
}

View File

@ -119,7 +119,7 @@ private void takeSnapshot(SnapshotIsolationMode lockingMode) throws Exception {
assertConnectorIsRunning();
final SourceRecords records = consumeRecordsByTopic(INITIAL_RECORDS_PER_TABLE);
final List<SourceRecord> table1 = records.recordsForTopic("server1.dbo.table1");
final List<SourceRecord> table1 = records.recordsForTopic("server1.testDB1.dbo.table1");
assertThat(table1).hasSize(INITIAL_RECORDS_PER_TABLE);
@ -153,7 +153,7 @@ public void takeSnapshotAndStartStreaming() throws Exception {
// Ignore initial records
final SourceRecords records = consumeRecordsByTopic(INITIAL_RECORDS_PER_TABLE);
final List<SourceRecord> table1 = records.recordsForTopic("server1.dbo.table1");
final List<SourceRecord> table1 = records.recordsForTopic("server1.testDB1.dbo.table1");
assertThat(((Struct) table1.get(0).value()).getStruct("source").getString("snapshot")).isEqualTo("first");
table1.subList(1, INITIAL_RECORDS_PER_TABLE - 1).forEach(record -> {
assertThat(((Struct) record.value()).getStruct("source").getString("snapshot")).isEqualTo("true");
@ -193,7 +193,7 @@ private void testStreaming() throws SQLException, InterruptedException {
TestHelper.waitForCdcRecord(connection, "table1", rs -> rs.getInt("id") == lastId);
final SourceRecords records = consumeRecordsByTopic(STREAMING_RECORDS_PER_TABLE);
final List<SourceRecord> table1 = records.recordsForTopic("server1.dbo.table1");
final List<SourceRecord> table1 = records.recordsForTopic("server1.testDB1.dbo.table1");
assertThat(table1).hasSize(INITIAL_RECORDS_PER_TABLE);
@ -256,7 +256,7 @@ public void takeSnapshotFromTableWithReservedName() throws Exception {
assertConnectorIsRunning();
final SourceRecords records = consumeRecordsByTopic(INITIAL_RECORDS_PER_TABLE);
final List<SourceRecord> user = records.recordsForTopic("server1.dbo.User");
final List<SourceRecord> user = records.recordsForTopic("server1.testDB1.dbo.User");
assertThat(user).hasSize(INITIAL_RECORDS_PER_TABLE);
@ -317,8 +317,8 @@ public void shouldSelectivelySnapshotTables() throws SQLException, InterruptedEx
assertConnectorIsRunning();
SourceRecords records = consumeRecordsByTopic(1);
List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.table_a");
List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.table_b");
List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.table_a");
List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.table_b");
Assertions.assertThat(tableA).hasSize(1);
Assertions.assertThat(tableB).isNull();
@ -327,8 +327,8 @@ public void shouldSelectivelySnapshotTables() throws SQLException, InterruptedEx
connection.execute("INSERT INTO table_b VALUES(24, 'some_name', 558)");
records = consumeRecordsByTopic(2);
tableA = records.recordsForTopic("server1.dbo.table_a");
tableB = records.recordsForTopic("server1.dbo.table_b");
tableA = records.recordsForTopic("server1.testDB1.dbo.table_a");
tableB = records.recordsForTopic("server1.testDB1.dbo.table_b");
Assertions.assertThat(tableA).hasSize(1);
Assertions.assertThat(tableB).hasSize(1);
@ -357,12 +357,12 @@ public void testColumnExcludeList() throws Exception {
assertConnectorIsRunning();
final SourceRecords records = consumeRecordsByTopic(2);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.blacklist_column_table_a");
final List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.blacklist_column_table_b");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.blacklist_column_table_a");
final List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.blacklist_column_table_b");
Schema expectedSchemaA = SchemaBuilder.struct()
.optional()
.name("server1.dbo.blacklist_column_table_a.Value")
.name("server1.testDB1.dbo.blacklist_column_table_a.Value")
.field("id", Schema.INT32_SCHEMA)
.field("name", Schema.OPTIONAL_STRING_SCHEMA)
.build();
@ -372,7 +372,7 @@ public void testColumnExcludeList() throws Exception {
Schema expectedSchemaB = SchemaBuilder.struct()
.optional()
.name("server1.dbo.blacklist_column_table_b.Value")
.name("server1.testDB1.dbo.blacklist_column_table_b.Value")
.field("id", Schema.INT32_SCHEMA)
.field("name", Schema.OPTIONAL_STRING_SCHEMA)
.field("amount", Schema.OPTIONAL_INT32_SCHEMA)
@ -414,14 +414,14 @@ public void reoderCapturedTables() throws Exception {
assertConnectorIsRunning();
SourceRecords records = consumeRecordsByTopic(1);
List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.table_a");
List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.table_b");
List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.table_a");
List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.table_b");
Assertions.assertThat(tableB).hasSize(1);
Assertions.assertThat(tableA).isNull();
records = consumeRecordsByTopic(1);
tableA = records.recordsForTopic("server1.dbo.table_a");
tableA = records.recordsForTopic("server1.testDB1.dbo.table_a");
Assertions.assertThat(tableA).hasSize(1);
stopConnector();
@ -449,21 +449,21 @@ public void reoderCapturedTablesWithOverlappingTableWhitelist() throws Exception
assertConnectorIsRunning();
SourceRecords records = consumeRecordsByTopic(1);
List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.table_a");
List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.table_ab");
List<SourceRecord> tableC = records.recordsForTopic("server1.dbo.table_ac");
List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.table_a");
List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.table_ab");
List<SourceRecord> tableC = records.recordsForTopic("server1.testDB1.dbo.table_ac");
Assertions.assertThat(tableB).hasSize(1);
Assertions.assertThat(tableA).isNull();
Assertions.assertThat(tableC).isNull();
records = consumeRecordsByTopic(1);
tableA = records.recordsForTopic("server1.dbo.table_a");
tableA = records.recordsForTopic("server1.testDB1.dbo.table_a");
Assertions.assertThat(tableA).hasSize(1);
Assertions.assertThat(tableC).isNull();
records = consumeRecordsByTopic(1);
tableC = records.recordsForTopic("server1.dbo.table_ac");
tableC = records.recordsForTopic("server1.testDB1.dbo.table_ac");
Assertions.assertThat(tableC).hasSize(1);
stopConnector();
@ -492,21 +492,21 @@ public void reoderCapturedTablesWithoutTableWhitelist() throws Exception {
assertConnectorIsRunning();
SourceRecords records = consumeRecordsByTopic(1);
List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.table_a");
List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.table_ab");
List<SourceRecord> tableC = records.recordsForTopic("server1.dbo.table_ac");
List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.table_a");
List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.table_ab");
List<SourceRecord> tableC = records.recordsForTopic("server1.testDB1.dbo.table_ac");
Assertions.assertThat(tableA).hasSize(1);
Assertions.assertThat(tableB).isNull();
Assertions.assertThat(tableC).isNull();
records = consumeRecordsByTopic(1);
tableB = records.recordsForTopic("server1.dbo.table_ab");
tableB = records.recordsForTopic("server1.testDB1.dbo.table_ab");
Assertions.assertThat(tableB).hasSize(1);
Assertions.assertThat(tableC).isNull();
records = consumeRecordsByTopic(1);
tableC = records.recordsForTopic("server1.dbo.table_ac");
tableC = records.recordsForTopic("server1.testDB1.dbo.table_ac");
Assertions.assertThat(tableC).hasSize(1);
stopConnector();
@ -521,7 +521,7 @@ public void shouldOutputRecordsInCloudEventsFormat() throws Exception {
assertConnectorIsRunning();
final SourceRecords snapshotRecords = consumeRecordsByTopic(INITIAL_RECORDS_PER_TABLE);
final List<SourceRecord> snapshotTable1 = snapshotRecords.recordsForTopic("server1.dbo.table1");
final List<SourceRecord> snapshotTable1 = snapshotRecords.recordsForTopic("server1.testDB1.dbo.table1");
assertThat(snapshotTable1).hasSize(INITIAL_RECORDS_PER_TABLE);
@ -539,7 +539,7 @@ public void shouldOutputRecordsInCloudEventsFormat() throws Exception {
}
final SourceRecords streamingRecords = consumeRecordsByTopic(STREAMING_RECORDS_PER_TABLE);
final List<SourceRecord> streamingTable1 = streamingRecords.recordsForTopic("server1.dbo.table1");
final List<SourceRecord> streamingTable1 = streamingRecords.recordsForTopic("server1.testDB1.dbo.table1");
assertThat(streamingTable1).hasSize(INITIAL_RECORDS_PER_TABLE);
@ -572,7 +572,7 @@ public void shouldHandleBracketsInSnapshotSelect() throws InterruptedException,
assertConnectorIsRunning();
SourceRecords records = consumeRecordsByTopic(1);
List<SourceRecord> recordsForTopic = records.recordsForTopic("server1.dbo.user_detail");
List<SourceRecord> recordsForTopic = records.recordsForTopic("server1.testDB1.dbo.user_detail");
assertThat(recordsForTopic.get(0).key()).isNotNull();
Struct value = (Struct) ((Struct) recordsForTopic.get(0).value()).get("after");
System.out.println("DATA: " + value);

View File

@ -91,37 +91,20 @@ public void after() throws SQLException {
}
@Test
@FixFor("DBZ-1224")
public void takeSnapshotWithOverridesInSinglePartitionMode() throws Exception {
@FixFor({ "DBZ-1224", "DBZ-2975" })
public void takeSnapshotWithOverridesInMultiPartitionMode() throws Exception {
final Configuration config = TestHelper.defaultConfig()
.with(
RelationalDatabaseConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE,
"dbo.table1,dbo.table3")
.with(
RelationalDatabaseConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE + ".dbo.table1",
"SELECT * FROM [dbo].[table1] where soft_deleted = 0 order by id desc")
"SELECT * FROM [" + TestHelper.TEST_DATABASE_1 + "].[dbo].[table1] where soft_deleted = 0 order by id desc")
.with(
RelationalDatabaseConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE + ".dbo.table3",
"SELECT * FROM [dbo].[table3] where soft_deleted = 0")
"SELECT * FROM [" + TestHelper.TEST_DATABASE_1 + "].[dbo].[table3] where soft_deleted = 0")
.build();
takeSnapshotWithOverrides(config, "server1.dbo.");
}
@Test
@FixFor({ "DBZ-1224", "DBZ-2975" })
public void takeSnapshotWithOverridesInMultiPartitionMode() throws Exception {
final Configuration config = TestHelper.defaultMultiPartitionConfig()
.with(
RelationalDatabaseConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE,
"dbo.table1,dbo.table3")
.with(
RelationalDatabaseConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE + ".dbo.table1",
"SELECT * FROM [" + TestHelper.TEST_DATABASE + "].[dbo].[table1] where soft_deleted = 0 order by id desc")
.with(
RelationalDatabaseConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE + ".dbo.table3",
"SELECT * FROM [" + TestHelper.TEST_DATABASE + "].[dbo].[table3] where soft_deleted = 0")
.build();
takeSnapshotWithOverrides(config, "server1.testDB.dbo.");
takeSnapshotWithOverrides(config, "server1.testDB1.dbo.");
}
private void takeSnapshotWithOverrides(Configuration config, String topicPrefix) throws Exception {
@ -157,37 +140,20 @@ private void takeSnapshotWithOverrides(Configuration config, String topicPrefix)
}
@Test
@FixFor("DBZ-3429")
public void takeSnapshotWithOverridesWithAdditionalWhitespaceInSinglePartitionMode() throws Exception {
@FixFor({ "DBZ-3429", "DBZ-2975" })
public void takeSnapshotWithOverridesWithAdditionalWhitespaceInMultiPartitionMode() throws Exception {
final Configuration config = TestHelper.defaultConfig()
.with(
RelationalDatabaseConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE,
" dbo.table1 , dbo.table3 ")
.with(
RelationalDatabaseConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE + ".dbo.table1",
"SELECT * FROM [dbo].[table1] where soft_deleted = 0 order by id desc")
"SELECT * FROM [" + TestHelper.TEST_DATABASE_1 + "].[dbo].[table1] where soft_deleted = 0 order by id desc")
.with(
RelationalDatabaseConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE + ".dbo.table3",
"SELECT * FROM [dbo].[table3] where soft_deleted = 0")
"SELECT * FROM [" + TestHelper.TEST_DATABASE_1 + "].[dbo].[table3] where soft_deleted = 0")
.build();
takeSnapshotWithOverridesWithAdditionalWhitespace(config, "server1.dbo.");
}
@Test
@FixFor({ "DBZ-3429", "DBZ-2975" })
public void takeSnapshotWithOverridesWithAdditionalWhitespaceInMultiPartitionMode() throws Exception {
final Configuration config = TestHelper.defaultMultiPartitionConfig()
.with(
RelationalDatabaseConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE,
" dbo.table1 , dbo.table3 ")
.with(
RelationalDatabaseConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE + ".dbo.table1",
"SELECT * FROM [" + TestHelper.TEST_DATABASE + "].[dbo].[table1] where soft_deleted = 0 order by id desc")
.with(
RelationalDatabaseConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE + ".dbo.table3",
"SELECT * FROM [" + TestHelper.TEST_DATABASE + "].[dbo].[table3] where soft_deleted = 0")
.build();
takeSnapshotWithOverridesWithAdditionalWhitespace(config, "server1.testDB.dbo.");
takeSnapshotWithOverridesWithAdditionalWhitespace(config, "server1.testDB1.dbo.");
}
private void takeSnapshotWithOverridesWithAdditionalWhitespace(Configuration config, String topicPrefix) throws Exception {

View File

@ -66,10 +66,10 @@ public void shouldParseWhitespaceChars() throws Exception {
assertConnectorIsRunning();
SourceRecords actualRecords = consumeRecordsByTopic(2, false);
assertThat(actualRecords.recordsForTopic("server1.dbo.UAT_WAG_CZ_Fixed_Asset")).hasSize(1);
assertThat(actualRecords.recordsForTopic("server1.dbo.UAT_WAG_CZ_Fixed_Prop")).hasSize(1);
assertThat(actualRecords.recordsForTopic("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset")).hasSize(1);
assertThat(actualRecords.recordsForTopic("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Prop")).hasSize(1);
List<SourceRecord> carRecords = actualRecords.recordsForTopic("server1.dbo.UAT_WAG_CZ_Fixed_Asset");
List<SourceRecord> carRecords = actualRecords.recordsForTopic("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset");
assertThat(carRecords.size()).isEqualTo(1);
SourceRecord carRecord = carRecords.get(0);
@ -77,19 +77,19 @@ public void shouldParseWhitespaceChars() throws Exception {
(Struct) ((Struct) carRecord.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.UAT_WAG_CZ_Fixed_Asset.Value")
.name("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset.Value")
.field("id", Schema.INT32_SCHEMA)
.field("my col$a", Schema.OPTIONAL_STRING_SCHEMA)
.build());
assertSchemaMatchesStruct(
(Struct) carRecord.key(),
SchemaBuilder.struct()
.name("server1.dbo.UAT_WAG_CZ_Fixed_Asset.Key")
.name("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset.Key")
.field("id", Schema.INT32_SCHEMA)
.build());
assertThat(((Struct) carRecord.value()).getStruct("after").getString("my col$a")).isEqualTo("asset");
List<SourceRecord> personRecords = actualRecords.recordsForTopic("server1.dbo.UAT_WAG_CZ_Fixed_Prop");
List<SourceRecord> personRecords = actualRecords.recordsForTopic("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Prop");
assertThat(personRecords.size()).isEqualTo(1);
SourceRecord personRecord = personRecords.get(0);
@ -97,14 +97,14 @@ public void shouldParseWhitespaceChars() throws Exception {
(Struct) ((Struct) personRecord.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.UAT_WAG_CZ_Fixed_Prop.Value")
.name("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Prop.Value")
.field("id", Schema.INT32_SCHEMA)
.field("my col$a", Schema.OPTIONAL_STRING_SCHEMA)
.build());
assertSchemaMatchesStruct(
(Struct) personRecord.key(),
SchemaBuilder.struct()
.name("server1.dbo.UAT_WAG_CZ_Fixed_Prop.Key")
.name("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Prop.Key")
.field("id", Schema.INT32_SCHEMA)
.build());
assertThat(((Struct) personRecord.value()).getStruct("after").getString("my col$a")).isEqualTo("prop");
@ -133,41 +133,41 @@ public void shouldParseSpecialChars() throws Exception {
assertConnectorIsRunning();
SourceRecords records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic("server1.dbo.UAT_WAG_CZ_Fixed_Asset")).hasSize(1);
assertThat(records.recordsForTopic("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset")).hasSize(1);
SourceRecord record = records.recordsForTopic("server1.dbo.UAT_WAG_CZ_Fixed_Asset").get(0);
SourceRecord record = records.recordsForTopic("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset").get(0);
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.UAT_WAG_CZ_Fixed_Asset.Value")
.name("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset.Value")
.field("id", Schema.INT32_SCHEMA)
.field("my_col_a", Schema.OPTIONAL_STRING_SCHEMA)
.build());
assertSchemaMatchesStruct(
(Struct) record.key(),
SchemaBuilder.struct()
.name("server1.dbo.UAT_WAG_CZ_Fixed_Asset.Key")
.name("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset.Key")
.field("id", Schema.INT32_SCHEMA)
.build());
assertThat(((Struct) record.value()).getStruct("after").getInt32("id")).isEqualTo(1);
connection.execute("INSERT INTO [UAT WAG CZ$Fixed Asset] VALUES(2, 'b')");
records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic("server1.dbo.UAT_WAG_CZ_Fixed_Asset")).hasSize(1);
record = records.recordsForTopic("server1.dbo.UAT_WAG_CZ_Fixed_Asset").get(0);
assertThat(records.recordsForTopic("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset")).hasSize(1);
record = records.recordsForTopic("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset").get(0);
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.UAT_WAG_CZ_Fixed_Asset.Value")
.name("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset.Value")
.field("id", Schema.INT32_SCHEMA)
.field("my_col_a", Schema.OPTIONAL_STRING_SCHEMA)
.build());
assertSchemaMatchesStruct(
(Struct) record.key(),
SchemaBuilder.struct()
.name("server1.dbo.UAT_WAG_CZ_Fixed_Asset.Key")
.name("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset.Key")
.field("id", Schema.INT32_SCHEMA)
.build());
assertThat(((Struct) record.value()).getStruct("after").getInt32("id")).isEqualTo(2);
@ -177,13 +177,13 @@ record = records.recordsForTopic("server1.dbo.UAT_WAG_CZ_Fixed_Asset").get(0);
TestHelper.enableTableCdc(connection, "UAT WAG CZ$Fixed Asset Two");
connection.execute("INSERT INTO [UAT WAG CZ$Fixed Asset Two] VALUES(3, 'b', 'empty')");
records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic("server1.dbo.UAT_WAG_CZ_Fixed_Asset_Two")).hasSize(1);
record = records.recordsForTopic("server1.dbo.UAT_WAG_CZ_Fixed_Asset_Two").get(0);
assertThat(records.recordsForTopic("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset_Two")).hasSize(1);
record = records.recordsForTopic("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset_Two").get(0);
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.UAT_WAG_CZ_Fixed_Asset_Two.Value")
.name("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset_Two.Value")
.field("id", Schema.INT32_SCHEMA)
.field("my_col_", Schema.OPTIONAL_STRING_SCHEMA)
.field("Description", Schema.STRING_SCHEMA)
@ -191,20 +191,20 @@ record = records.recordsForTopic("server1.dbo.UAT_WAG_CZ_Fixed_Asset_Two").get(0
assertSchemaMatchesStruct(
(Struct) record.key(),
SchemaBuilder.struct()
.name("server1.dbo.UAT_WAG_CZ_Fixed_Asset_Two.Key")
.name("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset_Two.Key")
.field("id", Schema.INT32_SCHEMA)
.build());
assertThat(((Struct) record.value()).getStruct("after").getInt32("id")).isEqualTo(3);
connection.execute("UPDATE [UAT WAG CZ$Fixed Asset Two] SET Description='c1' WHERE id=3");
records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic("server1.dbo.UAT_WAG_CZ_Fixed_Asset_Two")).hasSize(1);
record = records.recordsForTopic("server1.dbo.UAT_WAG_CZ_Fixed_Asset_Two").get(0);
assertThat(records.recordsForTopic("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset_Two")).hasSize(1);
record = records.recordsForTopic("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset_Two").get(0);
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.UAT_WAG_CZ_Fixed_Asset_Two.Value")
.name("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset_Two.Value")
.field("id", Schema.INT32_SCHEMA)
.field("my_col_", Schema.OPTIONAL_STRING_SCHEMA)
.field("Description", Schema.STRING_SCHEMA)
@ -213,7 +213,7 @@ record = records.recordsForTopic("server1.dbo.UAT_WAG_CZ_Fixed_Asset_Two").get(0
(Struct) ((Struct) record.value()).get("before"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.UAT_WAG_CZ_Fixed_Asset_Two.Value")
.name("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset_Two.Value")
.field("id", Schema.INT32_SCHEMA)
.field("my_col_", Schema.OPTIONAL_STRING_SCHEMA)
.field("Description", Schema.STRING_SCHEMA)
@ -228,20 +228,20 @@ record = records.recordsForTopic("server1.dbo.UAT_WAG_CZ_Fixed_Asset_Two").get(0
connection.execute("INSERT INTO [UAT WAG CZ$Fixed Asset] VALUES(4, 'b')");
records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic("server1.dbo.UAT_WAG_CZ_Fixed_Asset")).hasSize(1);
record = records.recordsForTopic("server1.dbo.UAT_WAG_CZ_Fixed_Asset").get(0);
assertThat(records.recordsForTopic("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset")).hasSize(1);
record = records.recordsForTopic("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset").get(0);
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.UAT_WAG_CZ_Fixed_Asset.Value")
.name("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset.Value")
.field("id", Schema.INT32_SCHEMA)
.field("my_col_a", Schema.OPTIONAL_STRING_SCHEMA)
.build());
assertSchemaMatchesStruct(
(Struct) record.key(),
SchemaBuilder.struct()
.name("server1.dbo.UAT_WAG_CZ_Fixed_Asset.Key")
.name("server1.testDB1.dbo.UAT_WAG_CZ_Fixed_Asset.Key")
.field("id", Schema.INT32_SCHEMA)
.build());
assertThat(((Struct) record.value()).getStruct("after").getInt32("id")).isEqualTo(4);
@ -257,9 +257,8 @@ public void shouldHandleSpecialCharactersInDatabaseNames() throws Exception {
initializeConnectorTestFramework();
Testing.Files.delete(TestHelper.DB_HISTORY_PATH);
final Configuration config = TestHelper.defaultConfig()
final Configuration config = TestHelper.defaultConfig(databaseName)
.with(SqlServerConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL)
.with(SqlServerConnectorConfig.DATABASE_NAME, databaseName)
.with(SqlServerConnectorConfig.SANITIZE_FIELD_NAMES, false)
.build();
connection.execute(
@ -271,7 +270,7 @@ public void shouldHandleSpecialCharactersInDatabaseNames() throws Exception {
// Wait for snapshot completion
SourceRecords recordsByTopic = consumeRecordsByTopic(1);
List<SourceRecord> records = recordsByTopic.recordsForTopic("server1.dbo.tablea");
List<SourceRecord> records = recordsByTopic.recordsForTopic("server1.test-db.dbo.tablea");
assertThat(records).hasSize(1);
Struct source = (Struct) ((Struct) records.get(0).value()).get("source");
assertThat(source.get("db")).isEqualTo("test-db");

View File

@ -90,7 +90,7 @@ private Struct consume(BinaryHandlingMode binaryMode) throws InterruptedExceptio
TestHelper.waitForSnapshotToBeCompleted();
SourceRecords records = consumeRecordsByTopic(1);
final List<SourceRecord> results = records.recordsForTopic("server1.dbo.binary_mode_test");
final List<SourceRecord> results = records.recordsForTopic("server1.testDB1.dbo.binary_mode_test");
Assertions.assertThat(results).hasSize(1);
return (Struct) ((Struct) results.get(0).value()).get("after");

View File

@ -87,8 +87,8 @@ public void addTable() throws Exception {
}
SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
// Enable CDC for already existing table
TestHelper.enableTableCdc(connection, "tablec");
@ -106,24 +106,24 @@ public void addTable() throws Exception {
"INSERT INTO tabled VALUES(" + id + ", 'd')");
}
records = consumeRecordsByTopic(RECORDS_PER_TABLE * 2);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablec")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tabled")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.dbo.tablec").forEach(record -> {
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablec")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tabled")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.testDB1.dbo.tablec").forEach(record -> {
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tablec.Value")
.name("server1.testDB1.dbo.tablec.Value")
.field("id", Schema.INT32_SCHEMA)
.field("colc", Schema.OPTIONAL_STRING_SCHEMA)
.build());
});
records.recordsForTopic("server1.dbo.tabled").forEach(record -> {
records.recordsForTopic("server1.testDB1.dbo.tabled").forEach(record -> {
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tabled.Value")
.name("server1.testDB1.dbo.tabled.Value")
.field("id", Schema.INT32_SCHEMA)
.field("cold", Schema.OPTIONAL_STRING_SCHEMA)
.build());
@ -153,8 +153,8 @@ public void removeTable() throws Exception {
}
SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
// Disable CDC for a table
TestHelper.disableTableCdc(connection, "tableb");
@ -167,8 +167,8 @@ public void removeTable() throws Exception {
"INSERT INTO tableb VALUES(" + id + ", 'b2')");
}
records = consumeRecordsByTopic(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).isNullOrEmpty();
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).isNullOrEmpty();
}
@Test
@ -228,14 +228,14 @@ private void addColumnToTable(Configuration config, boolean pauseAfterCaptureCha
}
SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.dbo.tableb").forEach(record -> {
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.testDB1.dbo.tableb").forEach(record -> {
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tableb.Value")
.name("server1.testDB1.dbo.tableb.Value")
.field("id", Schema.INT32_SCHEMA)
.field("colb", Schema.OPTIONAL_STRING_SCHEMA)
.build());
@ -252,15 +252,15 @@ private void addColumnToTable(Configuration config, boolean pauseAfterCaptureCha
"INSERT INTO tableb VALUES(" + id + ", 'b2', 2)");
}
records = consumeRecordsByTopic(RECORDS_PER_TABLE * 2);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.dbo.tableb").forEach(record -> {
records.recordsForTopic("server1.testDB1.dbo.tableb").forEach(record -> {
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tableb.Value")
.name("server1.testDB1.dbo.tableb.Value")
.field("id", Schema.INT32_SCHEMA)
.field("colb", Schema.OPTIONAL_STRING_SCHEMA)
.build());
@ -279,15 +279,15 @@ private void addColumnToTable(Configuration config, boolean pauseAfterCaptureCha
"INSERT INTO tableb VALUES(" + id + ", 'b3', 3)");
}
records = consumeRecordsByTopic(RECORDS_PER_TABLE * 2);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.dbo.tableb").forEach(record -> {
records.recordsForTopic("server1.testDB1.dbo.tableb").forEach(record -> {
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tableb.Value")
.name("server1.testDB1.dbo.tableb.Value")
.field("id", Schema.INT32_SCHEMA)
.field("colb", Schema.OPTIONAL_STRING_SCHEMA)
.field("newcol", SchemaBuilder.int32().defaultValue(0).build())
@ -302,14 +302,14 @@ private void addColumnToTable(Configuration config, boolean pauseAfterCaptureCha
"INSERT INTO tableb VALUES(" + id + ", 'b4', 4)");
}
records = consumeRecordsByTopic(RECORDS_PER_TABLE * 2);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.dbo.tableb").forEach(record -> {
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.testDB1.dbo.tableb").forEach(record -> {
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tableb.Value")
.name("server1.testDB1.dbo.tableb.Value")
.field("id", Schema.INT32_SCHEMA)
.field("colb", Schema.OPTIONAL_STRING_SCHEMA)
.field("newcol", SchemaBuilder.int32().defaultValue(0).build())
@ -341,14 +341,14 @@ public void removeColumnFromTable() throws Exception {
}
SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.dbo.tableb").forEach(record -> {
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.testDB1.dbo.tableb").forEach(record -> {
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tableb.Value")
.name("server1.testDB1.dbo.tableb.Value")
.field("id", Schema.INT32_SCHEMA)
.field("colb", Schema.OPTIONAL_STRING_SCHEMA)
.build());
@ -366,15 +366,15 @@ public void removeColumnFromTable() throws Exception {
"INSERT INTO tableb VALUES(" + id + ")");
}
records = consumeRecordsByTopic(RECORDS_PER_TABLE * 2);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.dbo.tableb").forEach(record -> {
records.recordsForTopic("server1.testDB1.dbo.tableb").forEach(record -> {
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tableb.Value")
.name("server1.testDB1.dbo.tableb.Value")
.field("id", Schema.INT32_SCHEMA)
.build());
});
@ -387,14 +387,14 @@ public void removeColumnFromTable() throws Exception {
"INSERT INTO tableb VALUES(" + id + ")");
}
records = consumeRecordsByTopic(RECORDS_PER_TABLE * 2);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.dbo.tableb").forEach(record -> {
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.testDB1.dbo.tableb").forEach(record -> {
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tableb.Value")
.name("server1.testDB1.dbo.tableb.Value")
.field("id", Schema.INT32_SCHEMA)
.build());
});
@ -427,13 +427,13 @@ public void removeColumnFromTableWithoutChangingCapture() throws Exception {
}
SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb2")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.dbo.tableb2").forEach(record -> {
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb2")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.testDB1.dbo.tableb2").forEach(record -> {
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tableb2.Value")
.name("server1.testDB1.dbo.tableb2.Value")
.field("id", Schema.INT32_SCHEMA)
.build());
});
@ -463,14 +463,14 @@ public void addColumnToTableWithParallelWrites() throws Exception {
}
SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.dbo.tableb").forEach(record -> {
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.testDB1.dbo.tableb").forEach(record -> {
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tableb.Value")
.name("server1.testDB1.dbo.tableb.Value")
.field("id", Schema.INT32_SCHEMA)
.field("colb", Schema.OPTIONAL_STRING_SCHEMA)
.build());
@ -498,12 +498,12 @@ public void addColumnToTableWithParallelWrites() throws Exception {
TestHelper.enableTableCdc(connection, "tableb", "after_change");
records = consumeRecordsByTopic(RECORDS_PER_TABLE * 2);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
final AtomicInteger beforeChangeCount = new AtomicInteger();
final AtomicInteger afterChangeCount = new AtomicInteger();
records.recordsForTopic("server1.dbo.tableb").forEach(record -> {
records.recordsForTopic("server1.testDB1.dbo.tableb").forEach(record -> {
if (((Struct) record.value()).getStruct("after").schema().field("colb2") != null) {
afterChangeCount.incrementAndGet();
}
@ -523,14 +523,14 @@ public void addColumnToTableWithParallelWrites() throws Exception {
"INSERT INTO tableb VALUES(" + id + ", 'b1', 'b2')");
}
records = consumeRecordsByTopic(RECORDS_PER_TABLE * 2);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.dbo.tableb").forEach(record -> {
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.testDB1.dbo.tableb").forEach(record -> {
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tableb.Value")
.name("server1.testDB1.dbo.tableb.Value")
.field("id", Schema.INT32_SCHEMA)
.field("colb", Schema.OPTIONAL_STRING_SCHEMA)
.field("colb2", Schema.OPTIONAL_STRING_SCHEMA)
@ -562,8 +562,8 @@ public void readHistoryAfterRestart() throws Exception {
}
SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
// Enable a second capture instance
connection.execute("ALTER TABLE dbo.tableb DROP COLUMN colb");
@ -577,8 +577,8 @@ public void readHistoryAfterRestart() throws Exception {
"INSERT INTO tableb VALUES(" + id + ")");
}
records = consumeRecordsByTopic(RECORDS_PER_TABLE * 2);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
stopConnector();
start(SqlServerConnector.class, config);
@ -592,14 +592,14 @@ public void readHistoryAfterRestart() throws Exception {
"INSERT INTO tableb VALUES(" + id + ")");
}
records = consumeRecordsByTopic(RECORDS_PER_TABLE * 2);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.dbo.tableb").forEach(record -> {
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.testDB1.dbo.tableb").forEach(record -> {
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tableb.Value")
.name("server1.testDB1.dbo.tableb.Value")
.field("id", Schema.INT32_SCHEMA)
.build());
});
@ -628,7 +628,7 @@ public void readHistoryAfterRestart() throws Exception {
final String type = changeArray.get(0).asDocument().getString("type");
final String tableIid = changeArray.get(0).asDocument().getString("id");
Assertions.assertThat(type).isEqualTo("ALTER");
Assertions.assertThat(tableIid).isEqualTo("\"testDB\".\"dbo\".\"tableb\"");
Assertions.assertThat(tableIid).isEqualTo("\"testDB1\".\"dbo\".\"tableb\"");
}
@Test
@ -655,14 +655,14 @@ public void renameColumn() throws Exception {
}
SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.dbo.tableb").forEach(record -> {
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.testDB1.dbo.tableb").forEach(record -> {
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tableb.Value")
.name("server1.testDB1.dbo.tableb.Value")
.field("id", Schema.INT32_SCHEMA)
.field("colb", Schema.OPTIONAL_STRING_SCHEMA)
.build());
@ -682,15 +682,15 @@ public void renameColumn() throws Exception {
"INSERT INTO tableb(id,newcolb) VALUES(" + id + ", 'b2')");
}
records = consumeRecordsByTopic(RECORDS_PER_TABLE * 2);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.dbo.tableb").forEach(record -> {
records.recordsForTopic("server1.testDB1.dbo.tableb").forEach(record -> {
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tableb.Value")
.name("server1.testDB1.dbo.tableb.Value")
.field("id", Schema.INT32_SCHEMA)
.field("newcolb", Schema.OPTIONAL_STRING_SCHEMA)
.build());
@ -704,14 +704,14 @@ public void renameColumn() throws Exception {
"INSERT INTO tableb VALUES(" + id + ", 'b3')");
}
records = consumeRecordsByTopic(RECORDS_PER_TABLE * 2);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.dbo.tableb").forEach(record -> {
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.testDB1.dbo.tableb").forEach(record -> {
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tableb.Value")
.name("server1.testDB1.dbo.tableb.Value")
.field("id", Schema.INT32_SCHEMA)
.field("newcolb", Schema.OPTIONAL_STRING_SCHEMA)
.build());
@ -742,14 +742,14 @@ public void changeColumn() throws Exception {
}
SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.dbo.tableb").forEach(record -> {
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.testDB1.dbo.tableb").forEach(record -> {
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tableb.Value")
.name("server1.testDB1.dbo.tableb.Value")
.field("id", Schema.INT32_SCHEMA)
.field("colb", Schema.OPTIONAL_STRING_SCHEMA)
.build());
@ -771,15 +771,15 @@ public void changeColumn() throws Exception {
"INSERT INTO tableb VALUES(" + id + ", '" + id + " ')");
}
records = consumeRecordsByTopic(RECORDS_PER_TABLE * 2);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.dbo.tableb").forEach(record -> {
records.recordsForTopic("server1.testDB1.dbo.tableb").forEach(record -> {
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tableb.Value")
.name("server1.testDB1.dbo.tableb.Value")
.field("id", Schema.INT32_SCHEMA)
.field("colb", Schema.OPTIONAL_INT32_SCHEMA)
.build());
@ -797,14 +797,14 @@ public void changeColumn() throws Exception {
"INSERT INTO tableb VALUES(" + id + ", '" + id + " ')");
}
records = consumeRecordsByTopic(RECORDS_PER_TABLE * 2);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.dbo.tableb").forEach(record -> {
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tablea")).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.tableb")).hasSize(RECORDS_PER_TABLE);
records.recordsForTopic("server1.testDB1.dbo.tableb").forEach(record -> {
assertSchemaMatchesStruct(
(Struct) ((Struct) record.value()).get("after"),
SchemaBuilder.struct()
.optional()
.name("server1.dbo.tableb.Value")
.name("server1.testDB1.dbo.tableb.Value")
.field("id", Schema.INT32_SCHEMA)
.field("colb", Schema.OPTIONAL_INT32_SCHEMA)
.build());
@ -835,7 +835,7 @@ public void addDefaultValue() throws Exception {
connection.execute("INSERT INTO tableb VALUES('1', 'some_value')");
TestHelper.waitForCdcRecord(connection, "tableb", "after_change", rs -> rs.getInt("id") == 1);
List<SourceRecord> records = consumeRecordsByTopic(1).recordsForTopic("server1.dbo.tableb");
List<SourceRecord> records = consumeRecordsByTopic(1).recordsForTopic("server1.testDB1.dbo.tableb");
Assertions.assertThat(records).hasSize(1);
Testing.debug("Records: " + records);
Testing.debug("Value Schema: " + records.get(0).valueSchema());
@ -874,7 +874,7 @@ public void alterDefaultValue() throws Exception {
TestHelper.enableTableCdc(connection, "table_dv", "after_change");
connection.execute("INSERT INTO table_dv VALUES('2', 'some_value2')");
List<SourceRecord> records = consumeRecordsByTopic(1).recordsForTopic("server1.dbo.table_dv");
List<SourceRecord> records = consumeRecordsByTopic(1).recordsForTopic("server1.testDB1.dbo.table_dv");
Assertions.assertThat(records).hasSize(1);
Schema colbSchema = records.get(0).valueSchema().field("after").schema().field("colb").schema();

View File

@ -69,10 +69,10 @@ private ZoneOffset getDatabaseZoneOffset() throws SQLException {
public void shouldEnableCdcForDatabase() throws Exception {
try (SqlServerConnection connection = TestHelper.adminConnection()) {
connection.connect();
connection.execute("CREATE DATABASE testDB");
connection.execute("USE testDB");
connection.execute("CREATE DATABASE testDB1");
connection.execute("USE testDB1");
// NOTE: you cannot enable CDC on master
TestHelper.enableDbCdc(connection, "testDB");
TestHelper.enableDbCdc(connection, "testDB1");
}
}
@ -80,10 +80,10 @@ public void shouldEnableCdcForDatabase() throws Exception {
public void shouldEnableCdcWithWrapperFunctionsForTable() throws Exception {
try (SqlServerConnection connection = TestHelper.adminConnection()) {
connection.connect();
connection.execute("CREATE DATABASE testDB");
connection.execute("USE testDB");
connection.execute("CREATE DATABASE testDB1");
connection.execute("USE testDB1");
// NOTE: you cannot enable CDC on master
TestHelper.enableDbCdc(connection, "testDB");
TestHelper.enableDbCdc(connection, "testDB1");
// create table if exists
String sql = "IF EXISTS (select 1 from sys.objects where name = 'testTable' and type = 'u')\n"
@ -124,19 +124,19 @@ public void shouldEnableCdcWithWrapperFunctionsForTable() throws Exception {
public void shouldProperlyGetDefaultColumnValues() throws Exception {
try (SqlServerConnection connection = TestHelper.adminConnection()) {
connection.connect();
connection.execute("CREATE DATABASE testDB");
connection.execute("USE testDB");
connection.execute("CREATE DATABASE testDB1");
connection.execute("USE testDB1");
}
try (SqlServerConnection connection = TestHelper.testConnection()) {
connection.connect();
// NOTE: you cannot enable CDC on master
TestHelper.enableDbCdc(connection, "testDB");
TestHelper.enableDbCdc(connection, "testDB1");
// create table if exists
String sql = "IF EXISTS (select 1 from sys.objects where name = 'table_with_defaults' and type = 'u')\n"
+ "DROP TABLE testTable\n"
+ "CREATE TABLE testDB.dbo.table_with_defaults ("
+ "CREATE TABLE testDB1.dbo.table_with_defaults ("
+ " int_no_default_not_null int not null,"
+ " int_no_default int,"
+ " bigint_column bigint default (3147483648),"
@ -204,14 +204,14 @@ public void shouldProperlyGetDefaultColumnValues() throws Exception {
"time_7_column", "char_column", "varchar_column", "text_column", "nchar_column", "nvarchar_column", "ntext_column", "binary_column",
"varbinary_column", "image_column");
SqlServerChangeTable changeTable = new SqlServerChangeTable(new TableId("testDB", "dbo", "table_with_defaults"),
SqlServerChangeTable changeTable = new SqlServerChangeTable(new TableId("testDB1", "dbo", "table_with_defaults"),
null, 0, null, capturedColumns);
Table table = connection.getTableSchemaFromTable(TestHelper.TEST_DATABASE, changeTable);
Table table = connection.getTableSchemaFromTable(TestHelper.TEST_DATABASE_1, changeTable);
TableSchemaBuilder tableSchemaBuilder = new TableSchemaBuilder(
new SqlServerValueConverters(JdbcValueConverters.DecimalMode.PRECISE, TemporalPrecisionMode.ADAPTIVE, null),
connection.getDefaultValueConverter(),
SchemaNameAdjuster.NO_OP, new CustomConverterRegistry(null), SchemaBuilder.struct().build(), false, false);
SchemaNameAdjuster.NO_OP, new CustomConverterRegistry(null), SchemaBuilder.struct().build(), false, true);
assertColumnHasNotDefaultValue(table, "int_no_default_not_null");
assertColumnHasDefaultValue(table, "int_no_default", null, tableSchemaBuilder);
@ -288,19 +288,19 @@ public void shouldProperlyGetDefaultColumnValues() throws Exception {
public void shouldProperlyGetDefaultColumnNullValues() throws Exception {
try (SqlServerConnection connection = TestHelper.adminConnection()) {
connection.connect();
connection.execute("CREATE DATABASE testDB");
connection.execute("USE testDB");
connection.execute("CREATE DATABASE testDB1");
connection.execute("USE testDB1");
}
try (SqlServerConnection connection = TestHelper.testConnection()) {
connection.connect();
// NOTE: you cannot enable CDC on master
TestHelper.enableDbCdc(connection, "testDB");
TestHelper.enableDbCdc(connection, "testDB1");
// create table if exists
String sql = "IF EXISTS (select 1 from sys.objects where name = 'table_with_defaults' and type = 'u')\n"
+ "DROP TABLE testTable\n"
+ "CREATE TABLE testDB.dbo.table_with_defaults ("
+ "CREATE TABLE testDB1.dbo.table_with_defaults ("
+ " int_no_default_not_null int not null,"
+ " int_no_default int,"
+ " int_default_null int default null,"
@ -341,7 +341,7 @@ public void shouldProperlyGetDefaultColumnNullValues() throws Exception {
// and issue a test call to a CDC wrapper function
Awaitility.await()
.atMost(5, TimeUnit.SECONDS)
.until(() -> connection.getMinLsn(TestHelper.TEST_DATABASE, "table_with_defaults").isAvailable()); // Need to wait to make sure the min_lsn is available
.until(() -> connection.getMinLsn(TestHelper.TEST_DATABASE_1, "table_with_defaults").isAvailable()); // Need to wait to make sure the min_lsn is available
List<String> capturedColumns = Arrays
.asList(
"int_no_default_not_null",
@ -374,14 +374,14 @@ public void shouldProperlyGetDefaultColumnNullValues() throws Exception {
"real_default_null",
"real_column");
SqlServerChangeTable changeTable = new SqlServerChangeTable(new TableId("testDB", "dbo", "table_with_defaults"),
SqlServerChangeTable changeTable = new SqlServerChangeTable(new TableId("testDB1", "dbo", "table_with_defaults"),
null, 0, null, capturedColumns);
Table table = connection.getTableSchemaFromTable(TestHelper.TEST_DATABASE, changeTable);
Table table = connection.getTableSchemaFromTable(TestHelper.TEST_DATABASE_1, changeTable);
TableSchemaBuilder tableSchemaBuilder = new TableSchemaBuilder(
new SqlServerValueConverters(JdbcValueConverters.DecimalMode.PRECISE, TemporalPrecisionMode.ADAPTIVE, null),
connection.getDefaultValueConverter(),
SchemaNameAdjuster.NO_OP, new CustomConverterRegistry(null), SchemaBuilder.struct().build(), false, false);
SchemaNameAdjuster.NO_OP, new CustomConverterRegistry(null), SchemaBuilder.struct().build(), false, true);
assertColumnHasNotDefaultValue(table, "int_no_default_not_null");
assertColumnHasDefaultValue(table, "int_no_default", null, tableSchemaBuilder);
@ -421,8 +421,8 @@ public void testAccessToCDCTableBasedOnUserRoleAccess() throws Exception {
// Setup a user with only read-only access
try (SqlServerConnection connection = TestHelper.adminConnection()) {
connection.connect();
connection.execute("CREATE DATABASE testDB");
connection.execute("USE testDB");
connection.execute("CREATE DATABASE testDB1");
connection.execute("USE testDB1");
String testUserCreateSql = "IF EXISTS (select 1 from sys.server_principals where name = 'test_user')\n"
+ "DROP LOGIN test_user\n"
@ -433,7 +433,7 @@ public void testAccessToCDCTableBasedOnUserRoleAccess() throws Exception {
connection.execute(testUserCreateSql);
// NOTE: you cannot enable CDC on master
TestHelper.enableDbCdc(connection, "testDB");
TestHelper.enableDbCdc(connection, "testDB1");
// create table if exists
String sql = "IF EXISTS (select 1 from sys.objects w" +
@ -445,15 +445,15 @@ public void testAccessToCDCTableBasedOnUserRoleAccess() throws Exception {
TestHelper.enableTableCdc(connection, "testTable");
// sa user should have access to CDC table
Assertions.assertThat(connection.checkIfConnectedUserHasAccessToCDCTable()).isTrue();
Assertions.assertThat(connection.checkIfConnectedUserHasAccessToCDCTable(TestHelper.TEST_DATABASE_1)).isTrue();
}
// Re-connect with the newly created user
try (SqlServerConnection connection = TestHelper.testConnection(
TestHelper.jdbcConfig("test_user", "Password!"))) {
// This user shouldn't have access to CDC table
connection.execute("USE testDB");
Assertions.assertThat(connection.checkIfConnectedUserHasAccessToCDCTable()).isFalse();
connection.execute("USE testDB1");
Assertions.assertThat(connection.checkIfConnectedUserHasAccessToCDCTable(TestHelper.TEST_DATABASE_1)).isFalse();
}
}

View File

@ -20,40 +20,21 @@ public class SqlServerConnectorConfigTest {
private static final Logger LOGGER = LoggerFactory.getLogger(SqlServerConnectorConfigTest.class);
@Test
public void noDatabaseName() {
public void emptyDatabaseNames() {
final SqlServerConnectorConfig connectorConfig = new SqlServerConnectorConfig(
defaultConfig().build());
assertFalse(connectorConfig.validateAndRecord(SqlServerConnectorConfig.ALL_FIELDS, LOGGER::error));
}
@Test
public void onlyDatabaseName() {
public void nonEmptyDatabaseNames() {
final SqlServerConnectorConfig connectorConfig = new SqlServerConnectorConfig(
defaultConfig()
.with(SqlServerConnectorConfig.DATABASE_NAME, "testDB")
.with(SqlServerConnectorConfig.DATABASE_NAMES, "testDB1")
.build());
assertTrue(connectorConfig.validateAndRecord(SqlServerConnectorConfig.ALL_FIELDS, LOGGER::error));
}
@Test
public void onlyDatabaseNames() {
final SqlServerConnectorConfig connectorConfig = new SqlServerConnectorConfig(
defaultConfig()
.with(SqlServerConnectorConfig.DATABASE_NAMES, "testDB")
.build());
assertTrue(connectorConfig.validateAndRecord(SqlServerConnectorConfig.ALL_FIELDS, LOGGER::error));
}
@Test
public void databaseNameAndDatabaseNames() {
final SqlServerConnectorConfig connectorConfig = new SqlServerConnectorConfig(
defaultConfig()
.with(SqlServerConnectorConfig.DATABASE_NAME, "testDB")
.with(SqlServerConnectorConfig.DATABASE_NAMES, "testDB")
.build());
assertFalse(connectorConfig.validateAndRecord(SqlServerConnectorConfig.ALL_FIELDS, LOGGER::error));
}
private Configuration.Builder defaultConfig() {
return Configuration.create()
.with(SqlServerConnectorConfig.SERVER_NAME, "server")

View File

@ -126,8 +126,8 @@ public void createAndDelete() throws Exception {
}
final SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.tableb");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.tableb");
Assertions.assertThat(tableA).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(tableB).hasSize(RECORDS_PER_TABLE);
for (int i = 0; i < RECORDS_PER_TABLE; i++) {
@ -153,8 +153,8 @@ public void createAndDelete() throws Exception {
connection.execute("DELETE FROM tableB");
final SourceRecords deleteRecords = consumeRecordsByTopic(2 * RECORDS_PER_TABLE);
final List<SourceRecord> deleteTableA = deleteRecords.recordsForTopic("server1.dbo.tablea");
final List<SourceRecord> deleteTableB = deleteRecords.recordsForTopic("server1.dbo.tableb");
final List<SourceRecord> deleteTableA = deleteRecords.recordsForTopic("server1.testDB1.dbo.tablea");
final List<SourceRecord> deleteTableB = deleteRecords.recordsForTopic("server1.testDB1.dbo.tableb");
Assertions.assertThat(deleteTableA).isNullOrEmpty();
Assertions.assertThat(deleteTableB).hasSize(2 * RECORDS_PER_TABLE);
@ -210,8 +210,8 @@ public void readOnlyApplicationIntent() throws Exception {
}
final SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES, 24);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.tableb");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.tableb");
Assertions.assertThat(tableA).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(tableB).hasSize(RECORDS_PER_TABLE);
for (int i = 0; i < RECORDS_PER_TABLE; i++) {
@ -290,8 +290,8 @@ public void timestampAndTimezone() throws Exception {
}
final SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.tableb");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.tableb");
Assertions.assertThat(tableA).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(tableB).hasSize(RECORDS_PER_TABLE);
for (int i = 0; i < RECORDS_PER_TABLE; i++) {
@ -335,8 +335,8 @@ public void deleteWithoutTombstone() throws Exception {
connection.execute("DELETE FROM tableB");
final SourceRecords deleteRecords = consumeRecordsByTopic(RECORDS_PER_TABLE);
final List<SourceRecord> deleteTableA = deleteRecords.recordsForTopic("server1.dbo.tablea");
final List<SourceRecord> deleteTableB = deleteRecords.recordsForTopic("server1.dbo.tableb");
final List<SourceRecord> deleteTableA = deleteRecords.recordsForTopic("server1.testDB1.dbo.tablea");
final List<SourceRecord> deleteTableB = deleteRecords.recordsForTopic("server1.testDB1.dbo.tableb");
Assertions.assertThat(deleteTableA).isNullOrEmpty();
Assertions.assertThat(deleteTableB).hasSize(RECORDS_PER_TABLE);
@ -382,7 +382,7 @@ public void update() throws Exception {
connection.execute("UPDATE tableb SET colb='z'");
final SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * 2);
final List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.tableb");
final List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.tableb");
Assertions.assertThat(tableB).hasSize(RECORDS_PER_TABLE * 2);
for (int i = 0; i < RECORDS_PER_TABLE; i++) {
final SourceRecord recordB = tableB.get(i);
@ -438,8 +438,8 @@ public void updatePrimaryKey() throws Exception {
"UPDATE tableb SET id=100 WHERE id=1");
final SourceRecords records = consumeRecordsByTopic(6);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.tableb");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.tableb");
Assertions.assertThat(tableA).hasSize(3);
Assertions.assertThat(tableB).hasSize(3);
@ -546,9 +546,9 @@ public void updatePrimaryKeyWithRestartInMiddle() throws Exception {
assertConnectorIsRunning();
final SourceRecords records2 = consumeRecordsByTopic(4);
final List<SourceRecord> tableA = records1.recordsForTopic("server1.dbo.tablea");
tableA.addAll(records2.recordsForTopic("server1.dbo.tablea"));
final List<SourceRecord> tableB = records2.recordsForTopic("server1.dbo.tableb");
final List<SourceRecord> tableA = records1.recordsForTopic("server1.testDB1.dbo.tablea");
tableA.addAll(records2.recordsForTopic("server1.testDB1.dbo.tablea"));
final List<SourceRecord> tableB = records2.recordsForTopic("server1.testDB1.dbo.tableb");
Assertions.assertThat(tableA).hasSize(3);
Assertions.assertThat(tableB).hasSize(3);
@ -661,8 +661,8 @@ public void updatePrimaryKeyTwiceWithRestartInMiddleOfTx() throws Exception {
assertConnectorIsRunning();
final SourceRecords records2 = consumeRecordsByTopic(6);
final List<SourceRecord> tableB = records1.recordsForTopic("server1.dbo.tableb");
tableB.addAll(records2.recordsForTopic("server1.dbo.tableb"));
final List<SourceRecord> tableB = records1.recordsForTopic("server1.testDB1.dbo.tableb");
tableB.addAll(records2.recordsForTopic("server1.testDB1.dbo.tableb"));
Assertions.assertThat(tableB).hasSize(20);
@ -707,8 +707,8 @@ public void streamChangesWhileStopped() throws Exception {
assertConnectorIsRunning();
final SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.tablea");
List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.tableb");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.tablea");
List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.tableb");
Assertions.assertThat(tableA).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(tableB).hasSize(RECORDS_PER_TABLE);
@ -762,21 +762,21 @@ public void verifyOffsets() throws Exception {
Awaitility.await().atMost(30, TimeUnit.SECONDS).until(() -> {
// Wait for max lsn to be available
if (!connection.getMaxLsn(TestHelper.TEST_DATABASE).isAvailable()) {
if (!connection.getMaxLsn(TestHelper.TEST_DATABASE_1).isAvailable()) {
return false;
}
// verify pre-snapshot inserts have succeeded
Map<String, Boolean> resultMap = new HashMap<>();
connection.getChangeTables(TestHelper.TEST_DATABASE).forEach(ct -> {
connection.getChangeTables(TestHelper.TEST_DATABASE_1).forEach(ct -> {
final String tableName = ct.getChangeTableId().table();
if (tableName.endsWith("dbo_" + tableaCT) || tableName.endsWith("dbo_" + tablebCT)) {
try {
final Lsn minLsn = connection.getMinLsn(TestHelper.TEST_DATABASE, tableName);
final Lsn maxLsn = connection.getMaxLsn(TestHelper.TEST_DATABASE);
final Lsn minLsn = connection.getMinLsn(TestHelper.TEST_DATABASE_1, tableName);
final Lsn maxLsn = connection.getMaxLsn(TestHelper.TEST_DATABASE_1);
SqlServerChangeTable[] tables = Collections.singletonList(ct).toArray(new SqlServerChangeTable[]{});
final List<Integer> ids = new ArrayList<>();
connection.getChangesForTables(TestHelper.TEST_DATABASE, tables, minLsn, maxLsn, resultsets -> {
connection.getChangesForTables(TestHelper.TEST_DATABASE_1, tables, minLsn, maxLsn, resultsets -> {
final ResultSet rs = resultsets[0];
while (rs.next()) {
ids.add(rs.getInt("id"));
@ -826,8 +826,8 @@ public void verifyOffsets() throws Exception {
assertConnectorIsRunning();
final SourceRecords sourceRecords = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
final List<SourceRecord> tableA = sourceRecords.recordsForTopic("server1.dbo.tablea");
final List<SourceRecord> tableB = sourceRecords.recordsForTopic("server1.dbo.tableb");
final List<SourceRecord> tableA = sourceRecords.recordsForTopic("server1.testDB1.dbo.tablea");
final List<SourceRecord> tableB = sourceRecords.recordsForTopic("server1.testDB1.dbo.tableb");
Assertions.assertThat(tableA).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(tableB).hasSize(RECORDS_PER_TABLE);
@ -888,8 +888,8 @@ public void testIncludeTable() throws Exception {
}
final SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.tableb");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.tableb");
Assertions.assertThat(tableA == null || tableA.isEmpty()).isTrue();
Assertions.assertThat(tableB).hasSize(RECORDS_PER_TABLE);
@ -961,7 +961,7 @@ public void testTableIncludeList() throws Exception {
// Wait for snapshot completion
final SourceRecords snapshotRecords = consumeRecordsByTopic(1);
Assertions.assertThat(snapshotRecords.recordsForTopic("server1.dbo.tableb")).isNotEmpty();
Assertions.assertThat(snapshotRecords.recordsForTopic("server1.testDB1.dbo.tableb")).isNotEmpty();
for (int i = 0; i < RECORDS_PER_TABLE; i++) {
final int id = ID_START + i;
@ -972,8 +972,8 @@ public void testTableIncludeList() throws Exception {
}
final SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.tableb");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.tableb");
Assertions.assertThat(tableA == null || tableA.isEmpty()).isTrue();
Assertions.assertThat(tableB).hasSize(RECORDS_PER_TABLE);
@ -1007,8 +1007,8 @@ public void testTableExcludeList() throws Exception {
}
final SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.tableb");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.tableb");
Assertions.assertThat(tableA == null || tableA.isEmpty()).isTrue();
Assertions.assertThat(tableB).hasSize(RECORDS_PER_TABLE);
@ -1037,11 +1037,11 @@ public void blacklistColumnWhenCdcColumnsDoNotMatchWithOriginalSnapshot() throws
connection.execute("INSERT INTO table_a VALUES(10, 'some_name', 120, 'some_string')");
final SourceRecords records = consumeRecordsByTopic(1);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.table_a");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.table_a");
Schema expectedSchemaA = SchemaBuilder.struct()
.optional()
.name("server1.dbo.table_a.Value")
.name("server1.testDB1.dbo.table_a.Value")
.field("id", Schema.INT32_SCHEMA)
.field("name", Schema.OPTIONAL_STRING_SCHEMA)
.field("amount", Schema.OPTIONAL_INT32_SCHEMA)
@ -1083,12 +1083,12 @@ public void testColumnExcludeList() throws Exception {
connection.execute("INSERT INTO blacklist_column_table_b VALUES(11, 'some_name', 447)");
final SourceRecords records = consumeRecordsByTopic(2);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.blacklist_column_table_a");
final List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.blacklist_column_table_b");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.blacklist_column_table_a");
final List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.blacklist_column_table_b");
Schema expectedSchemaA = SchemaBuilder.struct()
.optional()
.name("server1.dbo.blacklist_column_table_a.Value")
.name("server1.testDB1.dbo.blacklist_column_table_a.Value")
.field("id", Schema.INT32_SCHEMA)
.field("name", Schema.OPTIONAL_STRING_SCHEMA)
.build();
@ -1098,7 +1098,7 @@ public void testColumnExcludeList() throws Exception {
Schema expectedSchemaB = SchemaBuilder.struct()
.optional()
.name("server1.dbo.blacklist_column_table_b.Value")
.name("server1.testDB1.dbo.blacklist_column_table_b.Value")
.field("id", Schema.INT32_SCHEMA)
.field("name", Schema.OPTIONAL_STRING_SCHEMA)
.field("amount", Schema.OPTIONAL_INT32_SCHEMA)
@ -1137,7 +1137,7 @@ public void testColumnIncludeList() throws Exception {
start(SqlServerConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted("sql_server", "server1");
TestHelper.waitForSnapshotToBeCompleted();
// Wait for snapshot completion
consumeRecordsByTopic(1);
@ -1146,12 +1146,12 @@ public void testColumnIncludeList() throws Exception {
connection.execute("INSERT INTO include_list_column_table_b VALUES(11, 'some_name', 447)");
final SourceRecords records = consumeRecordsByTopic(2);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.include_list_column_table_a");
final List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.include_list_column_table_b");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.include_list_column_table_a");
final List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.include_list_column_table_b");
Schema expectedSchemaA = SchemaBuilder.struct()
.optional()
.name("server1.dbo.include_list_column_table_a.Value")
.name("server1.testDB1.dbo.include_list_column_table_a.Value")
.field("id", Schema.INT32_SCHEMA)
.field("name", Schema.OPTIONAL_STRING_SCHEMA)
.build();
@ -1161,7 +1161,7 @@ public void testColumnIncludeList() throws Exception {
Schema expectedSchemaB = SchemaBuilder.struct()
.optional()
.name("server1.dbo.include_list_column_table_b.Value")
.name("server1.testDB1.dbo.include_list_column_table_b.Value")
.field("id", Schema.INT32_SCHEMA)
.field("name", Schema.OPTIONAL_STRING_SCHEMA)
.field("amount", Schema.OPTIONAL_INT32_SCHEMA)
@ -1201,7 +1201,7 @@ public void shouldHandleInvalidColumnFilter() throws Exception {
// should be no more records
assertNoRecordsToConsume();
final String message = "All columns in table testDB.dbo.tablea were excluded due to include/exclude lists, defaulting to selecting all columns";
final String message = "All columns in table testDB1.dbo.tablea were excluded due to include/exclude lists, defaulting to selecting all columns";
stopConnector(value -> assertThat(logInterceptor.containsMessage(message)).isTrue());
}
@ -1216,7 +1216,7 @@ public void shouldConsumeEventsWithMaskedHashedColumns() throws Exception {
final Configuration config = TestHelper.defaultConfig()
.with(SqlServerConnectorConfig.SNAPSHOT_MODE, SnapshotMode.SCHEMA_ONLY)
.with("column.mask.hash.SHA-256.with.salt.CzQMA0cB5K", "testDB.dbo.masked_hashed_column_table_a.name, testDB.dbo.masked_hashed_column_table_b.name")
.with("column.mask.hash.SHA-256.with.salt.CzQMA0cB5K", "testDB1.dbo.masked_hashed_column_table_a.name, testDB1.dbo.masked_hashed_column_table_b.name")
.build();
start(SqlServerConnector.class, config);
@ -1229,8 +1229,8 @@ public void shouldConsumeEventsWithMaskedHashedColumns() throws Exception {
connection.execute("INSERT INTO masked_hashed_column_table_b VALUES(11, 'some_name')");
final SourceRecords records = consumeRecordsByTopic(2);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.masked_hashed_column_table_a");
final List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.masked_hashed_column_table_b");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.masked_hashed_column_table_a");
final List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.masked_hashed_column_table_b");
assertThat(tableA).hasSize(1);
SourceRecord record = tableA.get(0);
@ -1264,8 +1264,8 @@ public void shouldConsumeEventsWithMaskedAndTruncatedColumns() throws Exception
final Configuration config = TestHelper.defaultConfig()
.with(SqlServerConnectorConfig.SNAPSHOT_MODE, SnapshotMode.SCHEMA_ONLY)
.with("column.mask.with.12.chars", "testDB.dbo.masked_hashed_column_table.name")
.with("column.truncate.to.4.chars", "testDB.dbo.truncated_column_table.name")
.with("column.mask.with.12.chars", "testDB1.dbo.masked_hashed_column_table.name")
.with("column.truncate.to.4.chars", "testDB1.dbo.truncated_column_table.name")
.build();
start(SqlServerConnector.class, config);
@ -1278,8 +1278,8 @@ public void shouldConsumeEventsWithMaskedAndTruncatedColumns() throws Exception
connection.execute("INSERT INTO truncated_column_table VALUES(11, 'some_name')");
final SourceRecords records = consumeRecordsByTopic(2);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.masked_hashed_column_table");
final List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.truncated_column_table");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.masked_hashed_column_table");
final List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.truncated_column_table");
assertThat(tableA).hasSize(1);
SourceRecord record = tableA.get(0);
@ -1317,16 +1317,16 @@ public void whenCaptureInstanceExcludesColumnsExpectSnapshotAndStreamingToExclud
start(SqlServerConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted("sql_server", "server1");
TestHelper.waitForSnapshotToBeCompleted();
connection.execute("INSERT INTO excluded_column_table_a VALUES(11, 'some_name', 120)");
final SourceRecords records = consumeRecordsByTopic(3);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.excluded_column_table_a");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.excluded_column_table_a");
Schema expectedSchemaA = SchemaBuilder.struct()
.optional()
.name("server1.dbo.excluded_column_table_a.Value")
.name("server1.testDB1.dbo.excluded_column_table_a.Value")
.field("id", Schema.INT32_SCHEMA)
.field("name", Schema.OPTIONAL_STRING_SCHEMA)
.build();
@ -1367,16 +1367,16 @@ public void whenMultipleCaptureInstancesExcludesColumnsExpectLatestCDCTableUtili
start(SqlServerConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted("sql_server", "server1");
TestHelper.waitForSnapshotToBeCompleted();
connection.execute("INSERT INTO excluded_column_table_a VALUES(11, 'some_name', 120, 'a note')");
final SourceRecords records = consumeRecordsByTopic(3);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.excluded_column_table_a");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.excluded_column_table_a");
Schema expectedSchema = SchemaBuilder.struct()
.optional()
.name("server1.dbo.excluded_column_table_a.Value")
.name("server1.testDB1.dbo.excluded_column_table_a.Value")
.field("id", Schema.INT32_SCHEMA)
.field("name", Schema.OPTIONAL_STRING_SCHEMA)
.field("note", Schema.OPTIONAL_STRING_SCHEMA)
@ -1419,7 +1419,7 @@ public void whenCaptureInstanceExcludesColumnsAndColumnsRenamedExpectNoErrors()
start(SqlServerConnector.class, config);
assertConnectorIsRunning();
waitForStreamingRunning("sql_server", "server1");
TestHelper.waitForStreamingStarted();
TestHelper.disableTableCdc(connection, "excluded_column_table_a");
connection.execute("EXEC sp_RENAME 'excluded_column_table_a.name', 'first_name', 'COLUMN'");
@ -1430,11 +1430,11 @@ public void whenCaptureInstanceExcludesColumnsAndColumnsRenamedExpectNoErrors()
TestHelper.waitForCdcRecord(connection, "excluded_column_table_a", "dbo_excluded_column_table_a", rs -> rs.getInt("id") == 11);
final SourceRecords records = consumeRecordsByTopic(2);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.excluded_column_table_a");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.excluded_column_table_a");
Schema expectedSchema1 = SchemaBuilder.struct()
.optional()
.name("server1.dbo.excluded_column_table_a.Value")
.name("server1.testDB1.dbo.excluded_column_table_a.Value")
.field("id", Schema.INT32_SCHEMA)
.field("name", Schema.OPTIONAL_STRING_SCHEMA)
.build();
@ -1443,7 +1443,7 @@ public void whenCaptureInstanceExcludesColumnsAndColumnsRenamedExpectNoErrors()
.put("name", "a name");
Schema expectedSchema2 = SchemaBuilder.struct()
.optional()
.name("server1.dbo.excluded_column_table_a.Value")
.name("server1.testDB1.dbo.excluded_column_table_a.Value")
.field("id", Schema.INT32_SCHEMA)
.field("first_name", Schema.OPTIONAL_STRING_SCHEMA)
.build();
@ -1477,15 +1477,15 @@ public void excludeColumnWhenCaptureInstanceExcludesColumns() throws Exception {
start(SqlServerConnector.class, config);
assertConnectorIsRunning();
waitForStreamingRunning("sql_server", "server1");
TestHelper.waitForStreamingStarted();
connection.execute("INSERT INTO excluded_column_table_a VALUES(10, 'some_name', 120)");
final SourceRecords records = consumeRecordsByTopic(1);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.excluded_column_table_a");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.excluded_column_table_a");
Schema expectedSchemaA = SchemaBuilder.struct()
.optional()
.name("server1.dbo.excluded_column_table_a.Value")
.name("server1.testDB1.dbo.excluded_column_table_a.Value")
.field("id", Schema.INT32_SCHEMA)
.field("name", Schema.OPTIONAL_STRING_SCHEMA)
.build();
@ -1517,17 +1517,17 @@ public void excludeColumnWhenCaptureInstanceExcludesColumnInMiddleOfTable() thro
start(SqlServerConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted("sql_server", "server1");
TestHelper.waitForSnapshotToBeCompleted();
connection.execute("INSERT INTO exclude_list_column_table_a VALUES(11, 120, 'some_name')");
TestHelper.waitForCdcRecord(connection, "exclude_list_column_table_a", rs -> rs.getInt("id") == 11);
final SourceRecords records = consumeRecordsByTopic(2);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.exclude_list_column_table_a");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.exclude_list_column_table_a");
Schema expectedSchemaA = SchemaBuilder.struct()
.optional()
.name("server1.dbo.exclude_list_column_table_a.Value")
.name("server1.testDB1.dbo.exclude_list_column_table_a.Value")
.field("id", Schema.INT32_SCHEMA)
.field("name", Schema.OPTIONAL_STRING_SCHEMA)
.build();
@ -1564,17 +1564,17 @@ public void includeColumnsWhenCaptureInstanceExcludesColumnInMiddleOfTable() thr
start(SqlServerConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted("sql_server", "server1");
TestHelper.waitForSnapshotToBeCompleted();
connection.execute("INSERT INTO include_list_column_table_a VALUES(10, 120, 'some_name')");
TestHelper.waitForCdcRecord(connection, "include_list_column_table_a", rs -> rs.getInt("id") == 10);
final SourceRecords records = consumeRecordsByTopic(1);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.include_list_column_table_a");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.include_list_column_table_a");
Schema expectedSchemaA = SchemaBuilder.struct()
.optional()
.name("server1.dbo.include_list_column_table_a.Value")
.name("server1.testDB1.dbo.include_list_column_table_a.Value")
.field("id", Schema.INT32_SCHEMA)
.field("name", Schema.OPTIONAL_STRING_SCHEMA)
.build();
@ -1606,17 +1606,17 @@ public void excludeMultipleColumnsWhenCaptureInstanceExcludesSingleColumn() thro
start(SqlServerConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted("sql_server", "server1");
TestHelper.waitForSnapshotToBeCompleted();
connection.execute("INSERT INTO exclude_list_column_table_a VALUES(10, 120, 'a note', 'some_name')");
TestHelper.waitForCdcRecord(connection, "exclude_list_column_table_a", rs -> rs.getInt("id") == 10);
final SourceRecords records = consumeRecordsByTopic(1);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.exclude_list_column_table_a");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.exclude_list_column_table_a");
Schema expectedSchemaA = SchemaBuilder.struct()
.optional()
.name("server1.dbo.exclude_list_column_table_a.Value")
.name("server1.testDB1.dbo.exclude_list_column_table_a.Value")
.field("id", Schema.INT32_SCHEMA)
.field("name", Schema.OPTIONAL_STRING_SCHEMA)
.build();
@ -1648,17 +1648,17 @@ public void includeMultipleColumnsWhenCaptureInstanceExcludesSingleColumn() thro
start(SqlServerConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted("sql_server", "server1");
TestHelper.waitForSnapshotToBeCompleted();
connection.execute("INSERT INTO include_list_column_table_a VALUES(10, 120, 'a note', 'some_name')");
TestHelper.waitForCdcRecord(connection, "include_list_column_table_a", rs -> rs.getInt("id") == 10);
final SourceRecords records = consumeRecordsByTopic(1);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.include_list_column_table_a");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.include_list_column_table_a");
Schema expectedSchemaA = SchemaBuilder.struct()
.optional()
.name("server1.dbo.include_list_column_table_a.Value")
.name("server1.testDB1.dbo.include_list_column_table_a.Value")
.field("id", Schema.INT32_SCHEMA)
.field("name", Schema.OPTIONAL_STRING_SCHEMA)
.build();
@ -1720,7 +1720,7 @@ private void restartInTheMiddleOfTx(boolean restartJustAfterSnapshot, boolean af
}
start(SqlServerConnector.class, config, record -> {
if (!"server1.dbo.tablea.Envelope".equals(record.valueSchema().name())) {
if (!"server1.testDB1.dbo.tablea.Envelope".equals(record.valueSchema().name())) {
return false;
}
final Struct envelope = (Struct) record.value();
@ -1774,8 +1774,8 @@ private void restartInTheMiddleOfTx(boolean restartJustAfterSnapshot, boolean af
records = sourceRecords.allRecordsInOrder();
assertThat(records).hasSize(RECORDS_PER_TABLE);
List<SourceRecord> tableA = sourceRecords.recordsForTopic("server1.dbo.tablea");
List<SourceRecord> tableB = sourceRecords.recordsForTopic("server1.dbo.tableb");
List<SourceRecord> tableA = sourceRecords.recordsForTopic("server1.testDB1.dbo.tablea");
List<SourceRecord> tableB = sourceRecords.recordsForTopic("server1.testDB1.dbo.tableb");
for (int i = 0; i < RECORDS_PER_TABLE / 2; i++) {
final int id = HALF_ID + i;
final SourceRecord recordA = tableA.get(i);
@ -1809,8 +1809,8 @@ private void restartInTheMiddleOfTx(boolean restartJustAfterSnapshot, boolean af
TestHelper.waitForCdcRecord(connection, "tableb", rs -> rs.getInt("id") == (ID_RESTART + RECORDS_PER_TABLE - 1));
sourceRecords = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
tableA = sourceRecords.recordsForTopic("server1.dbo.tablea");
tableB = sourceRecords.recordsForTopic("server1.dbo.tableb");
tableA = sourceRecords.recordsForTopic("server1.testDB1.dbo.tablea");
tableB = sourceRecords.recordsForTopic("server1.testDB1.dbo.tableb");
Assertions.assertThat(tableA).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(tableB).hasSize(RECORDS_PER_TABLE);
@ -1917,19 +1917,19 @@ public void keylessTable() throws Exception {
// Wait for snapshot completion
SourceRecords records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic("server1.dbo.keyless").get(0).key()).isNull();
assertThat(records.recordsForTopic("server1.dbo.keyless").get(0).keySchema()).isNull();
assertThat(records.recordsForTopic("server1.testDB1.dbo.keyless").get(0).key()).isNull();
assertThat(records.recordsForTopic("server1.testDB1.dbo.keyless").get(0).keySchema()).isNull();
connection.execute(
"INSERT INTO keyless VALUES(2, 'k')");
records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic("server1.dbo.keyless").get(0).key()).isNull();
assertThat(records.recordsForTopic("server1.dbo.keyless").get(0).key()).isNull();
assertThat(records.recordsForTopic("server1.testDB1.dbo.keyless").get(0).key()).isNull();
assertThat(records.recordsForTopic("server1.testDB1.dbo.keyless").get(0).key()).isNull();
connection.execute(
"UPDATE keyless SET id=3 WHERE ID=2");
records = consumeRecordsByTopic(3);
final SourceRecord update1 = records.recordsForTopic("server1.dbo.keyless").get(0);
final SourceRecord update1 = records.recordsForTopic("server1.testDB1.dbo.keyless").get(0);
assertThat(update1.key()).isNull();
assertThat(update1.keySchema()).isNull();
@ -1939,9 +1939,9 @@ public void keylessTable() throws Exception {
connection.execute(
"DELETE FROM keyless WHERE id=3");
records = consumeRecordsByTopic(2, false);
assertThat(records.recordsForTopic("server1.dbo.keyless").get(0).key()).isNull();
assertThat(records.recordsForTopic("server1.dbo.keyless").get(0).keySchema()).isNull();
assertNull(records.recordsForTopic("server1.dbo.keyless").get(1).value());
assertThat(records.recordsForTopic("server1.testDB1.dbo.keyless").get(0).key()).isNull();
assertThat(records.recordsForTopic("server1.testDB1.dbo.keyless").get(0).keySchema()).isNull();
assertNull(records.recordsForTopic("server1.testDB1.dbo.keyless").get(1).value());
stopConnector();
}
@ -1965,7 +1965,7 @@ public void shouldRewriteIdentityKey() throws InterruptedException, SQLException
start(SqlServerConnector.class, config);
SourceRecords records = consumeRecordsByTopic(1);
List<SourceRecord> recordsForTopic = records.recordsForTopic("server1.dbo.keyless");
List<SourceRecord> recordsForTopic = records.recordsForTopic("server1.testDB1.dbo.keyless");
assertThat(recordsForTopic.get(0).key()).isNotNull();
Struct key = (Struct) recordsForTopic.get(0).key();
Assertions.assertThat(key.get("id")).isNotNull();
@ -1997,17 +1997,17 @@ public void shouldCaptureTableSchema() throws SQLException, InterruptedException
connection.execute(
"INSERT INTO table_schema_test (key_cola, key_colb, cola, colb, colc, cold) VALUES(1, 'a', 100, '2019-01-01 10:20:39.1234567 +02:00', 'some_value', 100.20)");
List<SourceRecord> records = consumeRecordsByTopic(1).recordsForTopic("server1.dbo.table_schema_test");
List<SourceRecord> records = consumeRecordsByTopic(1).recordsForTopic("server1.testDB1.dbo.table_schema_test");
assertThat(records).hasSize(1);
SourceRecordAssert.assertThat(records.get(0))
.keySchemaIsEqualTo(SchemaBuilder.struct()
.name("server1.dbo.table_schema_test.Key")
.name("server1.testDB1.dbo.table_schema_test.Key")
.field("key_cola", Schema.INT32_SCHEMA)
.field("key_colb", Schema.STRING_SCHEMA)
.build())
.valueAfterFieldSchemaIsEqualTo(SchemaBuilder.struct()
.optional()
.name("server1.dbo.table_schema_test.Value")
.name("server1.testDB1.dbo.table_schema_test.Value")
.field("key_cola", Schema.INT32_SCHEMA)
.field("key_colb", Schema.STRING_SCHEMA)
.field("cola", Schema.INT32_SCHEMA)
@ -2041,7 +2041,7 @@ public void shouldDetectPurgedHistory() throws Exception {
Awaitility.await().atMost(30, TimeUnit.SECONDS).pollInterval(100, TimeUnit.MILLISECONDS).until(() -> {
Testing.debug("Waiting for initial changes to be propagated to CDC structures");
return connection.getMaxLsn(TestHelper.TEST_DATABASE).isAvailable();
return connection.getMaxLsn(TestHelper.TEST_DATABASE_1).isAvailable();
});
start(SqlServerConnector.class, config);
@ -2103,8 +2103,8 @@ public void useShortTableNamesForColumnMapper() throws Exception {
}
final SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.tableb");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.tableb");
Assertions.assertThat(tableA).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(tableB).hasSize(RECORDS_PER_TABLE);
for (int i = 0; i < RECORDS_PER_TABLE; i++) {
@ -2133,7 +2133,7 @@ public void useLongTableNamesForColumnMapper() throws Exception {
final int ID_START = 10;
final Configuration config = TestHelper.defaultConfig()
.with(SqlServerConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL)
.with("column.mask.with.4.chars", "testDB.dbo.tablea.cola")
.with("column.mask.with.4.chars", "testDB1.dbo.tablea.cola")
.build();
start(SqlServerConnector.class, config);
@ -2151,8 +2151,8 @@ public void useLongTableNamesForColumnMapper() throws Exception {
}
final SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.tableb");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.tableb");
Assertions.assertThat(tableA).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(tableB).hasSize(RECORDS_PER_TABLE);
for (int i = 0; i < RECORDS_PER_TABLE; i++) {
@ -2181,7 +2181,7 @@ public void useLongTableNamesForKeyMapper() throws Exception {
final int ID_START = 10;
final Configuration config = TestHelper.defaultConfig()
.with(SqlServerConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL)
.with(SqlServerConnectorConfig.MSG_KEY_COLUMNS, "testDB.dbo.tablea:cola")
.with(SqlServerConnectorConfig.MSG_KEY_COLUMNS, "testDB1.dbo.tablea:cola")
.build();
start(SqlServerConnector.class, config);
@ -2199,8 +2199,8 @@ public void useLongTableNamesForKeyMapper() throws Exception {
}
final SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.tableb");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.tableb");
Assertions.assertThat(tableA).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(tableB).hasSize(RECORDS_PER_TABLE);
for (int i = 0; i < RECORDS_PER_TABLE; i++) {
@ -2247,8 +2247,8 @@ public void useShortTableNamesForKeyMapper() throws Exception {
}
final SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.tableb");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.tableb");
Assertions.assertThat(tableA).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(tableB).hasSize(RECORDS_PER_TABLE);
for (int i = 0; i < RECORDS_PER_TABLE; i++) {
@ -2283,11 +2283,11 @@ public void shouldPropagateSourceTypeByDatatype() throws Exception {
start(SqlServerConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted("sql_server", "server1");
TestHelper.waitForSnapshotToBeCompleted();
connection.execute("INSERT INTO dt_table (id,c1,c2,c3a,c3b,f1,f2) values (1, 123, 456, 789.01, 'test', 1.228, 234.56)");
SourceRecords records = consumeRecordsByTopic(1);
List<SourceRecord> recordsForTopic = records.recordsForTopic("server1.dbo.dt_table");
List<SourceRecord> recordsForTopic = records.recordsForTopic("server1.testDB1.dbo.dt_table");
final SourceRecord record = recordsForTopic.get(0);
final Field before = record.valueSchema().field("before");
@ -2364,8 +2364,8 @@ public void testMaxLsnSelectStatementWithoutLimit() throws Exception {
}
final SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.tableb");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.tableb");
Assertions.assertThat(tableA).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(tableB).hasSize(RECORDS_PER_TABLE);
@ -2399,8 +2399,8 @@ public void testMaxLsnSelectStatementWithLimit() throws Exception {
}
final SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.tableb");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.tableb");
Assertions.assertThat(tableA).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(tableB).hasSize(RECORDS_PER_TABLE);
@ -2428,7 +2428,7 @@ public void shouldEmitNoEventsForSkippedUpdateAndDeleteOperations() throws Excep
connection.execute("INSERT INTO tablea VALUES(203, 'insert203')");
final SourceRecords records = consumeRecordsByTopic(3);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.tablea");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.tablea");
Assertions.assertThat(tableA).hasSize(3);
tableA.forEach((SourceRecord record) -> {
Struct value = (Struct) record.value();
@ -2476,22 +2476,22 @@ public void shouldReturnTwoTaskConfigs() {
@Test
@FixFor("DBZ-2975")
public void shouldIncludeDatabaseNameIntoTopicAndSchemaNamesInMultiPartitionMode() throws Exception {
final Configuration config = TestHelper.defaultMultiPartitionConfig()
final Configuration config = TestHelper.defaultConfig()
.with(SqlServerConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL)
.build();
start(SqlServerConnector.class, config);
assertConnectorIsRunning();
TestHelper.waitForDatabaseSnapshotToBeCompleted(TestHelper.TEST_DATABASE);
TestHelper.waitForDatabaseSnapshotToBeCompleted(TestHelper.TEST_DATABASE_1);
final SourceRecords records = consumeRecordsByTopic(1);
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB.dbo.tablea");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.tablea");
Assertions.assertThat(tableA).hasSize(1);
final SourceRecord record = tableA.get(0);
assertThat(record.keySchema().name()).isEqualTo("server1.testDB.dbo.tablea.Key");
assertThat(record.valueSchema().name()).isEqualTo("server1.testDB.dbo.tablea.Envelope");
assertThat(record.keySchema().name()).isEqualTo("server1.testDB1.dbo.tablea.Key");
assertThat(record.valueSchema().name()).isEqualTo("server1.testDB1.dbo.tablea.Envelope");
}
@Test
@ -2539,12 +2539,12 @@ public void shouldApplySchemaFilters() throws Exception {
connection.execute("INSERT INTO tableb VALUES(1001, 'b')");
SourceRecords records = consumeRecordsByTopic(4);
List<SourceRecord> tableS1A = records.recordsForTopic("server1.s1.tablea");
List<SourceRecord> tableS1B = records.recordsForTopic("server1.s1.tableb");
List<SourceRecord> tableS2A = records.recordsForTopic("server1.s2.tablea");
List<SourceRecord> tableS2B = records.recordsForTopic("server1.s2.tableb");
List<SourceRecord> tableDboA = records.recordsForTopic("server1.dbo.tablea");
List<SourceRecord> tableDboB = records.recordsForTopic("server1.dbo.tableb");
List<SourceRecord> tableS1A = records.recordsForTopic("server1.testDB1.s1.tablea");
List<SourceRecord> tableS1B = records.recordsForTopic("server1.testDB1.s1.tableb");
List<SourceRecord> tableS2A = records.recordsForTopic("server1.testDB1.s2.tablea");
List<SourceRecord> tableS2B = records.recordsForTopic("server1.testDB1.s2.tableb");
List<SourceRecord> tableDboA = records.recordsForTopic("server1.testDB1.dbo.tablea");
List<SourceRecord> tableDboB = records.recordsForTopic("server1.testDB1.dbo.tableb");
assertNull(tableS1A);
assertNull(tableS1B);
@ -2572,12 +2572,12 @@ public void shouldApplySchemaFilters() throws Exception {
connection.execute("INSERT INTO s1.tableb VALUES(2, 2)");
records = consumeRecordsByTopic(2);
tableS2A = records.recordsForTopic("server1.s2.tablea");
tableS2B = records.recordsForTopic("server1.s2.tableb");
tableDboA = records.recordsForTopic("server1.dbo.tablea");
tableDboB = records.recordsForTopic("server1.dbo.tableb");
tableS1A = records.recordsForTopic("server1.s1.tablea");
tableS1B = records.recordsForTopic("server1.s1.tableb");
tableS2A = records.recordsForTopic("server1.testDB1.s2.tablea");
tableS2B = records.recordsForTopic("server1.testDB1.s2.tableb");
tableDboA = records.recordsForTopic("server1.testDB1.dbo.tablea");
tableDboB = records.recordsForTopic("server1.testDB1.dbo.tableb");
tableS1A = records.recordsForTopic("server1.testDB1.s1.tablea");
tableS1B = records.recordsForTopic("server1.testDB1.s1.tableb");
Assertions.assertThat(tableS1A).hasSize(1);
Assertions.assertThat(tableS1B).hasSize(1);
@ -2592,17 +2592,17 @@ public void shouldApplySchemaFilters() throws Exception {
@Test
@FixFor("DBZ-5033")
public void shouldIgnoreNullOffsetsWhenRecoveringHistory() {
final Configuration config1 = TestHelper.defaultMultiPartitionConfig()
final Configuration config1 = TestHelper.defaultConfig()
.with(SqlServerConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL_ONLY)
.build();
start(SqlServerConnector.class, config1);
assertConnectorIsRunning();
TestHelper.waitForDatabaseSnapshotToBeCompleted(TestHelper.TEST_DATABASE);
TestHelper.waitForDatabaseSnapshotToBeCompleted(TestHelper.TEST_DATABASE_1);
stopConnector();
TestHelper.createTestDatabases(TestHelper.TEST_DATABASE_2);
final Configuration config2 = TestHelper.defaultMultiPartitionConfig(
TestHelper.TEST_DATABASE, TestHelper.TEST_DATABASE_2)
final Configuration config2 = TestHelper.defaultConfig(
TestHelper.TEST_DATABASE_1, TestHelper.TEST_DATABASE_2)
.with(SqlServerConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL)
.build();
start(SqlServerConnector.class, config2);

View File

@ -60,7 +60,7 @@ public void after() throws SQLException {
@Test
public void snapshotAndStreaming() throws Exception {
final Configuration config = TestHelper.defaultMultiPartitionConfig(
final Configuration config = TestHelper.defaultConfig(
TestHelper.TEST_DATABASE_1,
TestHelper.TEST_DATABASE_2)
.with(SqlServerConnectorConfig.SNAPSHOT_MODE, SqlServerConnectorConfig.SnapshotMode.INITIAL)
@ -94,7 +94,7 @@ public void snapshotAndStreaming() throws Exception {
"USE " + TestHelper.TEST_DATABASE_2,
"INSERT INTO tableA VALUES(6, 'a2s')");
TestHelper.waitForTaskStreamingStarted();
TestHelper.waitForStreamingStarted();
records = consumeRecordsByTopic(2);
tableA1 = records.recordsForTopic(TestHelper.topicName(TestHelper.TEST_DATABASE_1, "tableA"));

View File

@ -35,7 +35,7 @@ public void testValidateUnableToConnectNoThrow() {
config.put(SqlServerConnectorConfig.SERVER_NAME.name(), "dbserver1");
config.put(SqlServerConnectorConfig.HOSTNAME.name(), "narnia");
config.put(SqlServerConnectorConfig.PORT.name(), "4321");
config.put(SqlServerConnectorConfig.DATABASE_NAME.name(), "sqlserver");
config.put(SqlServerConnectorConfig.DATABASE_NAMES.name(), "sqlserver");
config.put(SqlServerConnectorConfig.USER.name(), "pikachu");
config.put(SqlServerConnectorConfig.PASSWORD.name(), "raichu");

View File

@ -11,11 +11,11 @@ public class SqlServerPartitionTest extends AbstractPartitionTest<SqlServerParti
@Override
protected SqlServerPartition createPartition1() {
return new SqlServerPartition("server1", "database1", false);
return new SqlServerPartition("server1", "database1");
}
@Override
protected SqlServerPartition createPartition2() {
return new SqlServerPartition("server2", "database2", false);
return new SqlServerPartition("server2", "database2");
}
}

View File

@ -76,7 +76,7 @@ private Struct consume(SchemaNameAdjustmentMode adjustmentMode) throws Interrupt
TestHelper.waitForSnapshotToBeCompleted();
SourceRecords records = consumeRecordsByTopic(1);
final List<SourceRecord> results = records.recordsForTopic("server1.dbo.name-adjustment");
final List<SourceRecord> results = records.recordsForTopic("server1.testDB1.dbo.name-adjustment");
Assertions.assertThat(results).hasSize(1);
return (Struct) results.get(0).value();

View File

@ -70,8 +70,8 @@ public void shouldProcessFromSnapshot() throws Exception {
final int expectedRecordsCount = 1;
final SourceRecords records = consumeRecordsByTopic(expectedRecordsCount);
Assertions.assertThat(records.recordsForTopic("server1.dbo.t1").get(0).keySchema().field("key1")).isNotNull();
Assertions.assertThat(records.recordsForTopic("server1.dbo.t1").get(0).keySchema().field("key2")).isNotNull();
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.t1").get(0).keySchema().field("key1")).isNotNull();
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.t1").get(0).keySchema().field("key2")).isNotNull();
}
@Test
@ -90,14 +90,14 @@ public void shouldProcessFromStreaming() throws Exception {
final int expectedRecordsCount = 1;
SourceRecords records = consumeRecordsByTopic(expectedRecordsCount);
Assertions.assertThat(records.recordsForTopic("server1.dbo.t1").get(0).keySchema().field("key1")).isNotNull();
Assertions.assertThat(records.recordsForTopic("server1.dbo.t1").get(0).keySchema().field("key2")).isNotNull();
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.t1").get(0).keySchema().field("key1")).isNotNull();
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.t1").get(0).keySchema().field("key2")).isNotNull();
connection.execute("INSERT INTO t1 VALUES (2, 20, 'data2', 200);");
records = consumeRecordsByTopic(expectedRecordsCount);
Assertions.assertThat(records.recordsForTopic("server1.dbo.t1").get(0).keySchema().field("key1")).isNotNull();
Assertions.assertThat(records.recordsForTopic("server1.dbo.t1").get(0).keySchema().field("key2")).isNotNull();
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.t1").get(0).keySchema().field("key1")).isNotNull();
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.t1").get(0).keySchema().field("key2")).isNotNull();
connection.execute(DDL_STATEMENTS_STREAM);
TestHelper.enableTableCdc(connection, "t2", "t2_CT", Collect.arrayListOf("key1", "key2"));
@ -105,7 +105,7 @@ public void shouldProcessFromStreaming() throws Exception {
connection.execute("INSERT INTO t2 VALUES (2, 20, 'data2', 200);");
records = consumeRecordsByTopic(expectedRecordsCount);
Assertions.assertThat(records.recordsForTopic("server1.dbo.t2").get(0).keySchema().field("key1")).isNotNull();
Assertions.assertThat(records.recordsForTopic("server1.dbo.t2").get(0).keySchema().field("key2")).isNotNull();
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.t2").get(0).keySchema().field("key1")).isNotNull();
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.t2").get(0).keySchema().field("key2")).isNotNull();
}
}

View File

@ -68,11 +68,11 @@ public void shouldProcessFromSnapshot() throws Exception {
final int expectedRecordsCount = 1 + 1 + 1;
final SourceRecords records = consumeRecordsByTopic(expectedRecordsCount);
Assertions.assertThat(records.recordsForTopic("server1.dbo.t1").get(0).keySchema().field("pk")).isNotNull();
Assertions.assertThat(records.recordsForTopic("server1.dbo.t1").get(0).keySchema().fields()).hasSize(1);
Assertions.assertThat(records.recordsForTopic("server1.dbo.t2").get(0).keySchema().field("pk")).isNotNull();
Assertions.assertThat(records.recordsForTopic("server1.dbo.t2").get(0).keySchema().fields()).hasSize(1);
Assertions.assertThat(records.recordsForTopic("server1.dbo.t3").get(0).keySchema()).isNull();
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.t1").get(0).keySchema().field("pk")).isNotNull();
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.t1").get(0).keySchema().fields()).hasSize(1);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.t2").get(0).keySchema().field("pk")).isNotNull();
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.t2").get(0).keySchema().fields()).hasSize(1);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.t3").get(0).keySchema()).isNull();
}
@Test
@ -120,10 +120,10 @@ public void shouldProcessFromStreaming() throws Exception {
final int expectedRecordsCount = 1 + 1 + 1;
final SourceRecords records = consumeRecordsByTopic(expectedRecordsCount, 24);
Assertions.assertThat(records.recordsForTopic("server1.dbo.t1").get(0).keySchema().field("pk")).isNotNull();
Assertions.assertThat(records.recordsForTopic("server1.dbo.t1").get(0).keySchema().fields()).hasSize(1);
Assertions.assertThat(records.recordsForTopic("server1.dbo.t2").get(0).keySchema().field("pk")).isNotNull();
Assertions.assertThat(records.recordsForTopic("server1.dbo.t2").get(0).keySchema().fields()).hasSize(1);
Assertions.assertThat(records.recordsForTopic("server1.dbo.t3").get(0).keySchema()).isNull();
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.t1").get(0).keySchema().field("pk")).isNotNull();
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.t1").get(0).keySchema().fields()).hasSize(1);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.t2").get(0).keySchema().field("pk")).isNotNull();
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.t2").get(0).keySchema().fields()).hasSize(1);
Assertions.assertThat(records.recordsForTopic("server1.testDB1.dbo.t3").get(0).keySchema()).isNull();
}
}

View File

@ -105,8 +105,8 @@ public void transactionMetadata() throws Exception {
// BEGIN, data, END, BEGIN, data
final SourceRecords records = consumeRecordsByTopic(1 + RECORDS_PER_TABLE * 2 + 1 + 1 + 1);
final List<SourceRecord> tableA = records.recordsForTopic("server1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.dbo.tableb");
final List<SourceRecord> tableA = records.recordsForTopic("server1.testDB1.dbo.tablea");
final List<SourceRecord> tableB = records.recordsForTopic("server1.testDB1.dbo.tableb");
final List<SourceRecord> tx = records.recordsForTopic("server1.transaction");
Assertions.assertThat(tableA).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(tableB).hasSize(RECORDS_PER_TABLE + 1);
@ -122,7 +122,7 @@ public void transactionMetadata() throws Exception {
}
assertEndTransaction(all.get(2 * RECORDS_PER_TABLE + 1), txId, 2 * RECORDS_PER_TABLE,
Collect.hashMapOf("testDB.dbo.tablea", RECORDS_PER_TABLE, "testDB.dbo.tableb", RECORDS_PER_TABLE));
Collect.hashMapOf("testDB1.dbo.tablea", RECORDS_PER_TABLE, "testDB1.dbo.tableb", RECORDS_PER_TABLE));
stopConnector();
}
@ -149,19 +149,19 @@ private void restartInTheMiddleOfTx(boolean restartJustAfterSnapshot, boolean af
connection.execute("INSERT INTO tablea VALUES(-1, '-a')");
Awaitility.await().atMost(30, TimeUnit.SECONDS).until(() -> {
if (!connection.getMaxLsn(TestHelper.TEST_DATABASE).isAvailable()) {
if (!connection.getMaxLsn(TestHelper.TEST_DATABASE_1).isAvailable()) {
return false;
}
for (SqlServerChangeTable ct : connection.getChangeTables(TestHelper.TEST_DATABASE)) {
for (SqlServerChangeTable ct : connection.getChangeTables(TestHelper.TEST_DATABASE_1)) {
final String tableName = ct.getChangeTableId().table();
if (tableName.endsWith("dbo_" + connection.getNameOfChangeTable("tablea"))) {
try {
final Lsn minLsn = connection.getMinLsn(TestHelper.TEST_DATABASE, tableName);
final Lsn maxLsn = connection.getMaxLsn(TestHelper.TEST_DATABASE);
final Lsn minLsn = connection.getMinLsn(TestHelper.TEST_DATABASE_1, tableName);
final Lsn maxLsn = connection.getMaxLsn(TestHelper.TEST_DATABASE_1);
final AtomicReference<Boolean> found = new AtomicReference<>(false);
SqlServerChangeTable[] tables = Collections.singletonList(ct).toArray(new SqlServerChangeTable[]{});
connection.getChangesForTables(TestHelper.TEST_DATABASE, tables, minLsn, maxLsn, resultsets -> {
connection.getChangesForTables(TestHelper.TEST_DATABASE_1, tables, minLsn, maxLsn, resultsets -> {
final ResultSet rs = resultsets[0];
while (rs.next()) {
if (rs.getInt("id") == -1) {
@ -182,7 +182,7 @@ private void restartInTheMiddleOfTx(boolean restartJustAfterSnapshot, boolean af
}
start(SqlServerConnector.class, config, record -> {
if (!"server1.dbo.tablea.Envelope".equals(record.valueSchema().name())) {
if (!"server1.testDB1.dbo.tablea.Envelope".equals(record.valueSchema().name())) {
return false;
}
final Struct envelope = (Struct) record.value();
@ -232,7 +232,7 @@ private void restartInTheMiddleOfTx(boolean restartJustAfterSnapshot, boolean af
assertThat(records).hasSize(expectedRecords);
if (firstTxId != null) {
assertEndTransaction(records.get(0), firstTxId, 1, Collect.hashMapOf("testDB.dbo.tablea", 1));
assertEndTransaction(records.get(0), firstTxId, 1, Collect.hashMapOf("testDB1.dbo.tablea", 1));
}
final String batchTxId = assertBeginTransaction(records.get(txBeginIndex));
@ -252,8 +252,8 @@ private void restartInTheMiddleOfTx(boolean restartJustAfterSnapshot, boolean af
records = sourceRecords.allRecordsInOrder();
assertThat(records).hasSize(RECORDS_PER_TABLE);
List<SourceRecord> tableA = sourceRecords.recordsForTopic("server1.dbo.tablea");
List<SourceRecord> tableB = sourceRecords.recordsForTopic("server1.dbo.tableb");
List<SourceRecord> tableA = sourceRecords.recordsForTopic("server1.testDB1.dbo.tablea");
List<SourceRecord> tableB = sourceRecords.recordsForTopic("server1.testDB1.dbo.tableb");
for (int i = 0; i < RECORDS_PER_TABLE / 2; i++) {
final int id = HALF_ID + i;
final SourceRecord recordA = tableA.get(i);
@ -288,15 +288,15 @@ private void restartInTheMiddleOfTx(boolean restartJustAfterSnapshot, boolean af
// END of previous TX, data records, BEGIN of TX for every pair of record, END of TX for every pair of record but last
sourceRecords = consumeRecordsByTopic(1 + RECORDS_PER_TABLE * TABLES + (2 * RECORDS_PER_TABLE - 1));
tableA = sourceRecords.recordsForTopic("server1.dbo.tablea");
tableB = sourceRecords.recordsForTopic("server1.dbo.tableb");
tableA = sourceRecords.recordsForTopic("server1.testDB1.dbo.tablea");
tableB = sourceRecords.recordsForTopic("server1.testDB1.dbo.tableb");
List<SourceRecord> txMetadata = sourceRecords.recordsForTopic("server1.transaction");
Assertions.assertThat(tableA).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(tableB).hasSize(RECORDS_PER_TABLE);
Assertions.assertThat(txMetadata).hasSize(1 + 2 * RECORDS_PER_TABLE - 1);
assertEndTransaction(txMetadata.get(0), batchTxId, 2 * RECORDS_PER_TABLE,
Collect.hashMapOf("testDB.dbo.tablea", RECORDS_PER_TABLE, "testDB.dbo.tableb", RECORDS_PER_TABLE));
Collect.hashMapOf("testDB1.dbo.tablea", RECORDS_PER_TABLE, "testDB1.dbo.tableb", RECORDS_PER_TABLE));
for (int i = 0; i < RECORDS_PER_TABLE; i++) {
final int id = i + ID_RESTART;
@ -321,7 +321,7 @@ private void restartInTheMiddleOfTx(boolean restartJustAfterSnapshot, boolean af
assertRecordTransactionMetadata(recordA, txId, 1, 1);
assertRecordTransactionMetadata(recordB, txId, 2, 1);
if (i < RECORDS_PER_TABLE - 1) {
assertEndTransaction(txMetadata.get(2 * i + 2), txId, 2, Collect.hashMapOf("testDB.dbo.tablea", 1, "testDB.dbo.tableb", 1));
assertEndTransaction(txMetadata.get(2 * i + 2), txId, 2, Collect.hashMapOf("testDB1.dbo.tablea", 1, "testDB1.dbo.tableb", 1));
}
}
}

View File

@ -56,7 +56,6 @@ public class TestHelper {
private static final Logger LOGGER = LoggerFactory.getLogger(TestHelper.class);
public static final Path DB_HISTORY_PATH = Testing.Files.createTestingPath("file-db-history-connect.txt").toAbsolutePath();
public static final String TEST_DATABASE = "testDB";
public static final String TEST_DATABASE_1 = "testDB1";
public static final String TEST_DATABASE_2 = "testDB2";
public static final String TEST_SERVER_NAME = "server1";
@ -137,24 +136,19 @@ public static Configuration.Builder defaultConnectorConfig() {
* in individual tests as needed.
*/
public static Configuration.Builder defaultConfig() {
return defaultConnectorConfig()
.with(SqlServerConnectorConfig.DATABASE_NAME.name(), TEST_DATABASE);
return defaultConfig(TEST_DATABASE_1);
}
/**
* Returns a default configuration for connectors in multi-partition mode.
*/
public static Configuration.Builder defaultMultiPartitionConfig(String... databaseNames) {
public static Configuration.Builder defaultConfig(String... databaseNames) {
return defaultConnectorConfig()
.with(SqlServerConnectorConfig.DATABASE_NAMES.name(), String.join(",", databaseNames));
}
public static Configuration.Builder defaultMultiPartitionConfig() {
return defaultMultiPartitionConfig(TEST_DATABASE);
}
public static void createTestDatabase() {
createTestDatabase(TEST_DATABASE);
createTestDatabase(TEST_DATABASE_1);
}
public static void createTestDatabases(String... databaseNames) {
@ -185,7 +179,7 @@ public static void createTestDatabase(String databaseName) {
public static void dropTestDatabase() {
try (SqlServerConnection connection = adminConnection()) {
connection.connect();
dropTestDatabase(connection, TEST_DATABASE);
dropTestDatabase(connection, TEST_DATABASE_1);
}
catch (SQLException e) {
throw new IllegalStateException("Error while dropping test database", e);
@ -244,11 +238,11 @@ private static void dropTestDatabase(SqlServerConnection connection, String data
public static SqlServerConnection adminConnection() {
return new SqlServerConnection(TestHelper.defaultJdbcConfig(), SourceTimestampMode.getDefaultMode(),
new SqlServerValueConverters(JdbcValueConverters.DecimalMode.PRECISE, TemporalPrecisionMode.ADAPTIVE, null), () -> TestHelper.class.getClassLoader(),
Collections.emptySet(), true);
Collections.emptySet());
}
public static SqlServerConnection testConnection() {
return testConnection(TEST_DATABASE);
return testConnection(TEST_DATABASE_1);
}
/**
@ -270,18 +264,18 @@ public static SqlServerConnection testConnection(String databaseName) {
public static SqlServerConnection testConnection(JdbcConfiguration config) {
return new SqlServerConnection(config, SourceTimestampMode.getDefaultMode(),
new SqlServerValueConverters(JdbcValueConverters.DecimalMode.PRECISE, TemporalPrecisionMode.ADAPTIVE, null), () -> TestHelper.class.getClassLoader(),
Collections.emptySet(), true);
Collections.emptySet());
}
public static SqlServerConnection testConnectionWithOptionRecompile() {
JdbcConfiguration config = JdbcConfiguration.adapt(defaultJdbcConfig()
.edit()
.with(JdbcConfiguration.ON_CONNECT_STATEMENTS, "USE [" + TEST_DATABASE + "]")
.with(JdbcConfiguration.ON_CONNECT_STATEMENTS, "USE [" + TEST_DATABASE_1 + "]")
.build());
return new SqlServerConnection(config, SourceTimestampMode.getDefaultMode(),
new SqlServerValueConverters(JdbcValueConverters.DecimalMode.PRECISE, TemporalPrecisionMode.ADAPTIVE, null), () -> TestHelper.class.getClassLoader(),
Collections.emptySet(), true, true);
Collections.emptySet(), true);
}
/**
@ -297,7 +291,7 @@ public static void enableDbCdc(SqlServerConnection connection, String name) thro
Objects.requireNonNull(name);
connection.execute(ENABLE_DB_CDC.replace(STATEMENTS_PLACEHOLDER, name));
// make sure testDB has cdc-enabled before proceeding; throwing exception if it fails
// make sure the test database has cdc-enabled before proceeding; throwing exception if it fails
Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> {
final String sql = IS_CDC_ENABLED.replace(STATEMENTS_PLACEHOLDER, name);
return connection.queryAndMap(sql, connection.singleResultMapper(rs -> rs.getLong(1), "")) == 1L;
@ -426,7 +420,7 @@ public static void disableTableCdc(JdbcConnection connection, String name) throw
}
public static void waitForSnapshotToBeCompleted() {
waitForSnapshotToBeCompleted(getObjectName("snapshot", "server1"));
waitForDatabaseSnapshotToBeCompleted(TEST_DATABASE_1);
}
public static void waitForDatabaseSnapshotToBeCompleted(String databaseName) {
@ -464,14 +458,8 @@ public static void waitForTaskStreamingStarted(String taskId) {
"context", "streaming")));
}
public static void waitForTaskStreamingStarted() {
waitForTaskStreamingStarted(TEST_TASK_ID);
}
public static void waitForStreamingStarted() {
waitForStreamingStarted(getObjectName(Collect.linkMapOf(
"context", "streaming",
"server", "server1")));
waitForTaskStreamingStarted(TEST_TASK_ID);
}
public static void waitForStreamingStarted(ObjectName objectName) {
@ -493,7 +481,7 @@ public static void waitForStreamingStarted(ObjectName objectName) {
}
public static void waitForMaxLsnAvailable(SqlServerConnection connection) throws Exception {
waitForMaxLsnAvailable(connection, TEST_DATABASE);
waitForMaxLsnAvailable(connection, TEST_DATABASE_1);
}
public static void waitForMaxLsnAvailable(SqlServerConnection connection, String databaseName) throws Exception {
@ -556,19 +544,19 @@ public static void waitForCdcRecord(SqlServerConnection connection, String table
.atMost(60, TimeUnit.SECONDS)
.pollDelay(Duration.ofSeconds(0))
.pollInterval(Duration.ofMillis(100)).until(() -> {
if (!connection.getMaxLsn(TEST_DATABASE).isAvailable()) {
if (!connection.getMaxLsn(TEST_DATABASE_1).isAvailable()) {
return false;
}
for (SqlServerChangeTable ct : connection.getChangeTables(TEST_DATABASE)) {
for (SqlServerChangeTable ct : connection.getChangeTables(TEST_DATABASE_1)) {
final String ctTableName = ct.getChangeTableId().table();
if (ctTableName.endsWith("dbo_" + connection.getNameOfChangeTable(tableName))) {
try {
final Lsn minLsn = connection.getMinLsn(TEST_DATABASE, ctTableName);
final Lsn maxLsn = connection.getMaxLsn(TEST_DATABASE);
final Lsn minLsn = connection.getMinLsn(TEST_DATABASE_1, ctTableName);
final Lsn maxLsn = connection.getMaxLsn(TEST_DATABASE_1);
final CdcRecordFoundBlockingMultiResultSetConsumer consumer = new CdcRecordFoundBlockingMultiResultSetConsumer(handler);
SqlServerChangeTable[] tables = Collections.singletonList(ct).toArray(new SqlServerChangeTable[]{});
connection.getChangesForTables(TEST_DATABASE, tables, minLsn, maxLsn, consumer);
connection.getChangesForTables(TEST_DATABASE_1, tables, minLsn, maxLsn, consumer);
return consumer.isFound();
}
catch (Exception e) {
@ -611,19 +599,19 @@ public static void waitForCdcRecord(SqlServerConnection connection, String table
.atMost(30, TimeUnit.SECONDS)
.pollDelay(Duration.ofSeconds(0))
.pollInterval(Duration.ofMillis(100)).until(() -> {
if (!connection.getMaxLsn(TEST_DATABASE).isAvailable()) {
if (!connection.getMaxLsn(TEST_DATABASE_1).isAvailable()) {
return false;
}
for (SqlServerChangeTable ct : connection.getChangeTables(TEST_DATABASE)) {
for (SqlServerChangeTable ct : connection.getChangeTables(TEST_DATABASE_1)) {
final String ctTableName = ct.getChangeTableId().table();
if (ctTableName.endsWith(connection.getNameOfChangeTable(captureInstanceName))) {
try {
final Lsn minLsn = connection.getMinLsn(TEST_DATABASE, ctTableName);
final Lsn maxLsn = connection.getMaxLsn(TEST_DATABASE);
final Lsn minLsn = connection.getMinLsn(TEST_DATABASE_1, ctTableName);
final Lsn maxLsn = connection.getMaxLsn(TEST_DATABASE_1);
final CdcRecordFoundBlockingMultiResultSetConsumer consumer = new CdcRecordFoundBlockingMultiResultSetConsumer(handler);
SqlServerChangeTable[] tables = Collections.singletonList(ct).toArray(new SqlServerChangeTable[]{});
connection.getChangesForTables(TEST_DATABASE, tables, minLsn, maxLsn, consumer);
connection.getChangesForTables(TEST_DATABASE_1, tables, minLsn, maxLsn, consumer);
return consumer.isFound();
}
catch (Exception e) {

View File

@ -204,10 +204,10 @@ _tableName_:: The name of the database table in which the change event occurred.
For example, if `fulfillment` is the server name, and `dbo` is the schema name, and the database contains tables with the names `products`, `products_on_hand`, `customers`, and `orders`,
the connector would stream change event records to the following Kafka topics:
* `fulfillment.dbo.products`
* `fulfillment.dbo.products_on_hand`
* `fulfillment.dbo.customers`
* `fulfillment.dbo.orders`
* `fulfillment.testDB.dbo.products`
* `fulfillment.testDB.dbo.products_on_hand`
* `fulfillment.testDB.dbo.customers`
* `fulfillment.testDB.dbo.orders`
The connector applies similar naming conventions to label its internal database history topics, xref:about-the-debezium-sqlserver-connector-schema-change-topic[schema change topics], and xref:sqlserver-transaction-metadata[transaction metadata topics].
@ -539,7 +539,7 @@ Every change event that captures a change to the `customers` table has the same
}
],
"optional": false, // <3>
"name": "server1.dbo.customers.Key" // <4>
"name": "server1.testDB.dbo.customers.Key" // <4>
},
"payload": { // <5>
"id": 1004
@ -565,7 +565,7 @@ Every change event that captures a change to the `customers` table has the same
|Indicates whether the event key must contain a value in its `payload` field. In this example, a value in the key's payload is required. A value in the key's payload field is optional when a table does not have a primary key.
|4
|`server1.dbo.customers.Key`
|`server1.dbo.testDB.customers.Key`
a|Name of the schema that defines the structure of the key's payload. This schema describes the structure of the primary key for the table that was changed. Key schema names have the format _connector-name_._database-schema-name_._table-name_.`Key`. In this example: +
* `server1` is the name of the connector that generated this event. +
@ -657,7 +657,7 @@ The following example shows the value portion of a change event that the connect
}
],
"optional": true,
"name": "server1.dbo.customers.Value", // <2>
"name": "server1.dbo.testDB.customers.Value", // <2>
"field": "before"
},
{
@ -685,7 +685,7 @@ The following example shows the value portion of a change event that the connect
}
],
"optional": true,
"name": "server1.dbo.customers.Value",
"name": "server1.dbo.testDB.customers.Value",
"field": "after"
},
{
@ -764,7 +764,7 @@ The following example shows the value portion of a change event that the connect
}
],
"optional": false,
"name": "server1.dbo.customers.Envelope" // <4>
"name": "server1.dbo.testDB.customers.Envelope" // <4>
},
"payload": { // <5>
"before": null, // <6>
@ -807,7 +807,7 @@ The following example shows the value portion of a change event that the connect
|`name`
a|In the `schema` section, each `name` field specifies the schema for a field in the value's payload. +
+
`server1.dbo.customers.Value` is the schema for the payload's `before` and `after` fields. This schema is specific to the `customers` table. +
`server1.dbo.testDB.customers.Value` is the schema for the payload's `before` and `after` fields. This schema is specific to the `customers` table. +
+
Names of schemas for `before` and `after` fields are of the form `_logicalName_._database-schemaName_._tableName_.Value`, which ensures that the schema name is unique in the database.
This means that when using the xref:{link-avro-serialization}#avro-serialization[Avro converter], the resulting Avro schema for each table in each logical source has its own evolution and history.
@ -818,7 +818,7 @@ a|`io.debezium.connector.sqlserver.Source` is the schema for the payload's `sour
|4
|`name`
a|`server1.dbo.customers.Envelope` is the schema for the overall structure of the payload, where `server1` is the connector name, `dbo` is the database schema name, and `customers` is the table.
a|`server1.dbo.testDB.customers.Envelope` is the schema for the overall structure of the payload, where `server1` is the connector name, `dbo` is the database schema name, and `customers` is the table.
|5
|`payload`
@ -1094,11 +1094,11 @@ The following example shows a typical transaction boundary message:
"event_count": 2,
"data_collections": [
{
"data_collection": "testDB.dbo.tablea",
"data_collection": "testDB.dbo.testDB.tablea",
"event_count": 1
},
{
"data_collection": "testDB.dbo.tableb",
"data_collection": "testDB.dbo.testDB.tableb",
"event_count": 1
}
]
@ -2059,12 +2059,6 @@ Can be omitted when using Kerberos authentication, which can be configured using
|No default
|Password to use when connecting to the SQL Server database server.
|[[sqlserver-property-database-dbname]]<<sqlserver-property-database-dbname, `+database.dbname+`>>
|No default
|The name of the SQL Server database from which to stream the changes.
ifdef::community[]
Must not be used with `database.names`.
endif::community[]
|[[sqlserver-property-database-instance]] <<sqlserver-property-database-instance, `+database.instance+`>>
|No default
|Specifies the instance name of the link:https://docs.microsoft.com/en-us/sql/database-engine/configure-windows/database-engine-instances-sql-server?view=sql-server-latest#instances[SQL Server named instance].
@ -2073,15 +2067,6 @@ ifdef::community[]
|[[sqlserver-property-database-names]]<<sqlserver-property-database-names, `+database.names+`>>
|No default
|The comma-separated list of the SQL Server database names from which to stream the changes.
Currently, only one database name is supported. Must not be used with `database.dbname`.
This option is *experimental* and must not be used in production. Using it will make the behavior of the connector
incompatible with the default configuration with no upgrade or downgrade path:
* The connector will use different keys for its committed offset messages.
* The SQL statements used in `snapshot.select.statement.overrides` will have to use the database name
as part of the fully-qualified table name.
* The structure of the exposed connector metrics will be different.
endif::community[]
|[[sqlserver-property-database-server-name]]<<sqlserver-property-database-server-name, `+database.server.name+`>>
|No default