readableMetadataList;
+
+ /**
+ * Cache to compute schema differences for ALTER events.
+ *
+ * This cache is runtime-only and will be reconstructed from checkpointed split state (see
+ * {@link #initializeTableSchemaCacheFromSplitSchemas(Map)}). It must not be {@code final}
+ * because Java deserialization bypasses field initializers for {@code transient} fields.
+ */
+ private transient Map tableSchemaCache;
+
+ private static final FlinkJsonTableChangeSerializer TABLE_CHANGE_SERIALIZER =
+ new FlinkJsonTableChangeSerializer();
+
+ public SqlServerEventDeserializer(
+ DebeziumChangelogMode changelogMode, boolean includeSchemaChanges) {
+ super(new SqlServerSchemaDataTypeInference(), changelogMode);
+ this.includeSchemaChanges = includeSchemaChanges;
+ this.readableMetadataList = new ArrayList<>();
+ this.tableSchemaCache = new HashMap<>();
+ }
+
+ public SqlServerEventDeserializer(
+ DebeziumChangelogMode changelogMode,
+ boolean includeSchemaChanges,
+ List readableMetadataList) {
+ super(new SqlServerSchemaDataTypeInference(), changelogMode);
+ this.includeSchemaChanges = includeSchemaChanges;
+ this.readableMetadataList = readableMetadataList;
+ this.tableSchemaCache = new HashMap<>();
+ }
+
+ /**
+ * Initializes schema cache from checkpointing split state.
+ *
+ * The incremental source checkpoints Debezium {@link TableChange}s in {@code StreamSplit}'s
+ * {@code tableSchemas}. We use it as the source-of-truth to (re)build the local {@link Schema}
+ * cache after failover or task redistribution.
+ */
+ public void initializeTableSchemaCacheFromSplitSchemas(
+ Map tableSchemas) {
+ if (!includeSchemaChanges || tableSchemas == null || tableSchemas.isEmpty()) {
+ return;
+ }
+ final Map cache = getTableSchemaCache();
+ for (Map.Entry entry :
+ tableSchemas.entrySet()) {
+ final io.debezium.relational.TableId dbzTableId = entry.getKey();
+ final TableChange tableChange = entry.getValue();
+ if (dbzTableId == null || tableChange == null || tableChange.getTable() == null) {
+ continue;
+ }
+ final TableId tableId =
+ TableId.tableId(dbzTableId.catalog(), dbzTableId.schema(), dbzTableId.table());
+ cache.putIfAbsent(tableId, SqlServerSchemaUtils.toSchema(tableChange.getTable()));
+ }
+ }
+
+ private Map getTableSchemaCache() {
+ if (tableSchemaCache == null) {
+ tableSchemaCache = new HashMap<>();
+ }
+ return tableSchemaCache;
+ }
+
+ @Override
+ protected List deserializeSchemaChangeRecord(SourceRecord record) {
+ if (!includeSchemaChanges) {
+ return Collections.emptyList();
+ }
+
+ try {
+ TableChanges changes =
+ TABLE_CHANGE_SERIALIZER.deserialize(
+ getHistoryRecord(record)
+ .document()
+ .getArray(
+ io.debezium.relational.history.HistoryRecord.Fields
+ .TABLE_CHANGES),
+ true);
+
+ final Map cache = getTableSchemaCache();
+ List events = new ArrayList<>();
+ for (TableChange change : changes) {
+ TableId tableId =
+ TableId.tableId(
+ change.getId().catalog(),
+ change.getId().schema(),
+ change.getId().table());
+ Schema newSchema = SqlServerSchemaUtils.toSchema(change.getTable());
+ switch (change.getType()) {
+ case CREATE:
+ events.add(
+ new org.apache.flink.cdc.common.event.CreateTableEvent(
+ tableId, newSchema));
+ cache.put(tableId, newSchema);
+ break;
+ case ALTER:
+ Schema oldSchema = cache.get(tableId);
+ if (oldSchema == null) {
+ events.add(
+ new org.apache.flink.cdc.common.event.CreateTableEvent(
+ tableId, newSchema));
+ } else {
+ events.addAll(
+ SchemaMergingUtils.getSchemaDifference(
+ tableId, oldSchema, newSchema));
+ }
+ cache.put(tableId, newSchema);
+ break;
+ case DROP:
+ events.add(new org.apache.flink.cdc.common.event.DropTableEvent(tableId));
+ cache.remove(tableId);
+ break;
+ default:
+ // ignore others
+ }
+ }
+ return events;
+ } catch (Exception e) {
+ throw new IllegalStateException(
+ "Failed to deserialize SQL Server schema change event", e);
+ }
+ }
+
+ @Override
+ protected boolean isDataChangeRecord(SourceRecord record) {
+ org.apache.kafka.connect.data.Schema valueSchema = record.valueSchema();
+ Struct value = (Struct) record.value();
+ return value != null
+ && valueSchema != null
+ && valueSchema.field(Envelope.FieldName.OPERATION) != null
+ && value.getString(Envelope.FieldName.OPERATION) != null;
+ }
+
+ @Override
+ protected boolean isSchemaChangeRecord(SourceRecord record) {
+ return isSchemaChangeEvent(record);
+ }
+
+ @Override
+ protected TableId getTableId(SourceRecord record) {
+ // Debezium source record contains database/schema/table information in the source struct.
+ // Using SourceRecordUtils keeps the namespace (database) in the TableId so that schema
+ // change events and data change events refer to the same identifier.
+ io.debezium.relational.TableId dbzTableId =
+ org.apache.flink.cdc.connectors.base.utils.SourceRecordUtils.getTableId(record);
+ return SqlServerSchemaUtils.toCdcTableId(dbzTableId);
+ }
+
+ @Override
+ protected Map getMetadata(SourceRecord record) {
+ Map metadataMap = new HashMap<>();
+ if (readableMetadataList == null || readableMetadataList.isEmpty()) {
+ return metadataMap;
+ }
+ readableMetadataList.forEach(
+ (sqlServerReadableMetadata -> {
+ Object metadata = sqlServerReadableMetadata.getConverter().read(record);
+ if (sqlServerReadableMetadata.equals(SqlServerReadableMetadata.OP_TS)) {
+ metadataMap.put(
+ sqlServerReadableMetadata.getKey(),
+ String.valueOf(((TimestampData) metadata).getMillisecond()));
+ } else {
+ metadataMap.put(
+ sqlServerReadableMetadata.getKey(), String.valueOf(metadata));
+ }
+ }));
+ return metadataMap;
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerMetadataAccessor.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerMetadataAccessor.java
new file mode 100644
index 00000000000..7d6281a4024
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerMetadataAccessor.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.sqlserver.source;
+
+import org.apache.flink.cdc.common.annotation.Internal;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.common.source.MetadataAccessor;
+import org.apache.flink.cdc.connectors.sqlserver.source.config.SqlServerSourceConfig;
+import org.apache.flink.cdc.connectors.sqlserver.utils.SqlServerSchemaUtils;
+import org.apache.flink.table.api.ValidationException;
+
+import javax.annotation.Nullable;
+
+import java.util.List;
+
+/** {@link MetadataAccessor} for {@link SqlServerDataSource}. */
+@Internal
+public class SqlServerMetadataAccessor implements MetadataAccessor {
+
+ private final SqlServerSourceConfig sourceConfig;
+
+ public SqlServerMetadataAccessor(SqlServerSourceConfig sourceConfig) {
+ this.sourceConfig = sourceConfig;
+ }
+
+ /**
+ * List all databases from SQL Server.
+ *
+ * @return The list of database names
+ */
+ @Override
+ public List listNamespaces() {
+ return SqlServerSchemaUtils.listNamespaces(sourceConfig);
+ }
+
+ /**
+ * List all schemas from SQL Server database.
+ *
+ * @param namespace The database name to list schemas from.
+ * @return The list of schema names
+ */
+ @Override
+ public List listSchemas(@Nullable String namespace) {
+ return SqlServerSchemaUtils.listSchemas(sourceConfig, resolveNamespace(namespace));
+ }
+
+ private String resolveNamespace(@Nullable String namespace) {
+ if (namespace != null) {
+ return namespace;
+ }
+
+ List configuredDatabases = sourceConfig.getDatabaseList();
+ if (configuredDatabases != null && !configuredDatabases.isEmpty()) {
+ return configuredDatabases.get(0);
+ }
+
+ throw new ValidationException(
+ "Namespace must not be null when listing SQL Server schemas and no database "
+ + "is configured in the source configuration.");
+ }
+
+ /**
+ * List tables from SQL Server.
+ *
+ * @param namespace The database name. If null, uses the configured database.
+ * @param schemaName The schema name. If null, list tables from all schemas.
+ * @return The list of {@link TableId}s.
+ */
+ @Override
+ public List listTables(@Nullable String namespace, @Nullable String schemaName) {
+ return SqlServerSchemaUtils.listTables(sourceConfig, namespace, schemaName);
+ }
+
+ /**
+ * Get the {@link Schema} of the given table.
+ *
+ * @param tableId The {@link TableId} of the given table.
+ * @return The {@link Schema} of the table.
+ */
+ @Override
+ public Schema getTableSchema(TableId tableId) {
+ return SqlServerSchemaUtils.getTableSchema(sourceConfig, tableId);
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerPipelineSource.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerPipelineSource.java
new file mode 100644
index 00000000000..4a00be4e1ef
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerPipelineSource.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.sqlserver.source;
+
+import org.apache.flink.cdc.common.annotation.Internal;
+import org.apache.flink.cdc.common.event.Event;
+import org.apache.flink.cdc.connectors.base.config.SourceConfig;
+import org.apache.flink.cdc.connectors.base.source.meta.split.SourceRecords;
+import org.apache.flink.cdc.connectors.base.source.meta.split.SourceSplitState;
+import org.apache.flink.cdc.connectors.base.source.metrics.SourceReaderMetrics;
+import org.apache.flink.cdc.connectors.sqlserver.source.config.SqlServerSourceConfig;
+import org.apache.flink.cdc.connectors.sqlserver.source.config.SqlServerSourceConfigFactory;
+import org.apache.flink.cdc.connectors.sqlserver.source.dialect.SqlServerDialect;
+import org.apache.flink.cdc.connectors.sqlserver.source.offset.LsnFactory;
+import org.apache.flink.cdc.connectors.sqlserver.source.reader.SqlServerPipelineRecordEmitter;
+import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema;
+import org.apache.flink.connector.base.source.reader.RecordEmitter;
+
+/**
+ * The SQL Server CDC Source for Pipeline connector, which supports parallel reading snapshot of
+ * table and then continue to capture data change from transaction log.
+ *
+ * This source extends {@link SqlServerSourceBuilder.SqlServerIncrementalSource} and overrides
+ * the record emitter to use {@link SqlServerPipelineRecordEmitter} for proper handling of schema
+ * events in the CDC pipeline.
+ */
+@Internal
+public class SqlServerPipelineSource
+ extends SqlServerSourceBuilder.SqlServerIncrementalSource {
+
+ private static final long serialVersionUID = 1L;
+
+ public SqlServerPipelineSource(
+ SqlServerSourceConfigFactory configFactory,
+ DebeziumDeserializationSchema deserializationSchema,
+ LsnFactory offsetFactory,
+ SqlServerDialect dataSourceDialect) {
+ super(configFactory, deserializationSchema, offsetFactory, dataSourceDialect);
+ }
+
+ @Override
+ protected RecordEmitter createRecordEmitter(
+ SourceConfig sourceConfig, SourceReaderMetrics sourceReaderMetrics) {
+ SqlServerSourceConfig sqlServerSourceConfig = (SqlServerSourceConfig) sourceConfig;
+ SqlServerDialect sqlServerDialect = (SqlServerDialect) dataSourceDialect;
+ return new SqlServerPipelineRecordEmitter<>(
+ deserializationSchema,
+ sourceReaderMetrics,
+ sqlServerSourceConfig,
+ offsetFactory,
+ sqlServerDialect);
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerSchemaDataTypeInference.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerSchemaDataTypeInference.java
new file mode 100644
index 00000000000..8974a3bae64
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerSchemaDataTypeInference.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.sqlserver.source;
+
+import org.apache.flink.cdc.common.annotation.Internal;
+import org.apache.flink.cdc.common.types.DataType;
+import org.apache.flink.cdc.debezium.event.DebeziumSchemaDataTypeInference;
+
+import org.apache.kafka.connect.data.Schema;
+
+/** {@link DataType} inference for SQL Server debezium {@link Schema}. */
+@Internal
+public class SqlServerSchemaDataTypeInference extends DebeziumSchemaDataTypeInference {
+
+ private static final long serialVersionUID = 1L;
+
+ // SQL Server has database-specific types, but no special handling is currently
+ // needed here, so this class uses the default implementation from the parent class.
+ // If SQL Server-specific types require special handling in the future,
+ // it can be added here by overriding the inferStruct method.
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/reader/SqlServerPipelineRecordEmitter.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/reader/SqlServerPipelineRecordEmitter.java
new file mode 100644
index 00000000000..7080b7bfffc
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/reader/SqlServerPipelineRecordEmitter.java
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.sqlserver.source.reader;
+
+import org.apache.flink.api.connector.source.SourceOutput;
+import org.apache.flink.cdc.common.event.CreateTableEvent;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.connectors.base.options.StartupOptions;
+import org.apache.flink.cdc.connectors.base.source.meta.offset.OffsetFactory;
+import org.apache.flink.cdc.connectors.base.source.meta.split.SourceSplitState;
+import org.apache.flink.cdc.connectors.base.source.metrics.SourceReaderMetrics;
+import org.apache.flink.cdc.connectors.base.source.reader.IncrementalSourceRecordEmitter;
+import org.apache.flink.cdc.connectors.sqlserver.source.SqlServerEventDeserializer;
+import org.apache.flink.cdc.connectors.sqlserver.source.config.SqlServerSourceConfig;
+import org.apache.flink.cdc.connectors.sqlserver.source.dialect.SqlServerDialect;
+import org.apache.flink.cdc.connectors.sqlserver.source.utils.SqlServerConnectionUtils;
+import org.apache.flink.cdc.connectors.sqlserver.utils.SqlServerSchemaUtils;
+import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema;
+import org.apache.flink.connector.base.source.reader.RecordEmitter;
+
+import io.debezium.connector.sqlserver.SqlServerConnection;
+import org.apache.kafka.connect.source.SourceRecord;
+
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.flink.cdc.connectors.base.source.meta.wartermark.WatermarkEvent.isLowWatermarkEvent;
+import static org.apache.flink.cdc.connectors.base.utils.SourceRecordUtils.getTableId;
+import static org.apache.flink.cdc.connectors.base.utils.SourceRecordUtils.isDataChangeRecord;
+import static org.apache.flink.cdc.connectors.base.utils.SourceRecordUtils.isSchemaChangeEvent;
+import static org.apache.flink.cdc.connectors.sqlserver.source.utils.SqlServerConnectionUtils.createSqlServerConnection;
+
+/** The {@link RecordEmitter} implementation for SQL Server pipeline connector. */
+public class SqlServerPipelineRecordEmitter extends IncrementalSourceRecordEmitter {
+ private final SqlServerSourceConfig sourceConfig;
+ private final SqlServerDialect sqlServerDialect;
+
+ // Track tables that have already sent CreateTableEvent
+ private final Set alreadySendCreateTableTables;
+
+ // Used when startup mode is snapshot (bounded mode)
+ private boolean shouldEmitAllCreateTableEventsInSnapshotMode = true;
+ private final boolean isBounded;
+
+ // Cache for CreateTableEvent, using Map for O(1) lookup
+ private final Map createTableEventCache;
+
+ public SqlServerPipelineRecordEmitter(
+ DebeziumDeserializationSchema debeziumDeserializationSchema,
+ SourceReaderMetrics sourceReaderMetrics,
+ SqlServerSourceConfig sourceConfig,
+ OffsetFactory offsetFactory,
+ SqlServerDialect sqlServerDialect) {
+ super(
+ debeziumDeserializationSchema,
+ sourceReaderMetrics,
+ sourceConfig.isIncludeSchemaChanges(),
+ offsetFactory);
+ this.sourceConfig = sourceConfig;
+ this.sqlServerDialect = sqlServerDialect;
+ this.alreadySendCreateTableTables = new HashSet<>();
+ this.createTableEventCache = new HashMap<>();
+ this.isBounded = StartupOptions.snapshot().equals(sourceConfig.getStartupOptions());
+ generateCreateTableEvents();
+ }
+
+ @Override
+ protected void processElement(
+ SourceRecord element, SourceOutput output, SourceSplitState splitState)
+ throws Exception {
+ // Rebuild schema cache from checkpointing split state before handling schema change
+ // records.
+ // The stream split checkpoints Debezium TableChange(s) (table schemas) and will be restored
+ // on failover; deserializer's local cache is runtime-only and must be reinitialized.
+ if (isSchemaChangeEvent(element)
+ && splitState.isStreamSplitState()
+ && debeziumDeserializationSchema instanceof SqlServerEventDeserializer) {
+ ((SqlServerEventDeserializer) debeziumDeserializationSchema)
+ .initializeTableSchemaCacheFromSplitSchemas(
+ splitState.asStreamSplitState().getTableSchemas());
+ }
+
+ if (shouldEmitAllCreateTableEventsInSnapshotMode && isBounded) {
+ // In snapshot mode, emit all schemas at once.
+ emitAllCreateTableEvents(output);
+ shouldEmitAllCreateTableEventsInSnapshotMode = false;
+ } else if (isLowWatermarkEvent(element) && splitState.isSnapshotSplitState()) {
+ // In Snapshot phase of INITIAL startup mode, lazily send CreateTableEvent
+ // to downstream to avoid checkpoint timeout.
+ io.debezium.relational.TableId tableId =
+ splitState.asSnapshotSplitState().toSourceSplit().getTableId();
+ emitCreateTableEventIfNeeded(tableId, output);
+ } else if (isDataChangeRecord(element)) {
+ // Handle data change events, schema change events are handled downstream directly
+ io.debezium.relational.TableId tableId = getTableId(element);
+ emitCreateTableEventIfNeeded(tableId, output);
+ }
+ super.processElement(element, output, splitState);
+ }
+
+ @SuppressWarnings("unchecked")
+ private void emitAllCreateTableEvents(SourceOutput output) {
+ createTableEventCache.forEach(
+ (tableId, createTableEvent) -> {
+ output.collect((T) createTableEvent);
+ alreadySendCreateTableTables.add(tableId);
+ });
+ }
+
+ @SuppressWarnings("unchecked")
+ private void emitCreateTableEventIfNeeded(
+ io.debezium.relational.TableId tableId, SourceOutput output) {
+ if (alreadySendCreateTableTables.contains(tableId)) {
+ return;
+ }
+
+ CreateTableEvent createTableEvent = createTableEventCache.get(tableId);
+ if (createTableEvent != null) {
+ output.collect((T) createTableEvent);
+ } else {
+ // Table not in cache, fetch schema from database
+ try (SqlServerConnection jdbc =
+ createSqlServerConnection(sourceConfig.getDbzConnectorConfig())) {
+ createTableEvent = buildCreateTableEvent(jdbc, tableId);
+ output.collect((T) createTableEvent);
+ createTableEventCache.put(tableId, createTableEvent);
+ } catch (SQLException e) {
+ throw new RuntimeException("Failed to get table schema for " + tableId, e);
+ }
+ }
+ alreadySendCreateTableTables.add(tableId);
+ }
+
+ private CreateTableEvent buildCreateTableEvent(
+ SqlServerConnection jdbc, io.debezium.relational.TableId tableId) {
+ Schema schema = SqlServerSchemaUtils.getTableSchema(tableId, jdbc, sqlServerDialect);
+ return new CreateTableEvent(
+ TableId.tableId(tableId.catalog(), tableId.schema(), tableId.table()), schema);
+ }
+
+ private void generateCreateTableEvents() {
+ try (SqlServerConnection jdbc =
+ createSqlServerConnection(sourceConfig.getDbzConnectorConfig())) {
+ List capturedTableIds =
+ SqlServerConnectionUtils.listTables(
+ jdbc, sourceConfig.getTableFilters(), sourceConfig.getDatabaseList());
+ for (io.debezium.relational.TableId tableId : capturedTableIds) {
+ CreateTableEvent createTableEvent = buildCreateTableEvent(jdbc, tableId);
+ createTableEventCache.put(tableId, createTableEvent);
+ }
+ } catch (SQLException e) {
+ throw new RuntimeException("Cannot start emitter to fetch table schema.", e);
+ }
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/main/java/org/apache/flink/cdc/connectors/sqlserver/utils/SqlServerSchemaUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/main/java/org/apache/flink/cdc/connectors/sqlserver/utils/SqlServerSchemaUtils.java
new file mode 100644
index 00000000000..5569947501a
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/main/java/org/apache/flink/cdc/connectors/sqlserver/utils/SqlServerSchemaUtils.java
@@ -0,0 +1,326 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.sqlserver.utils;
+
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.schema.Column;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.connectors.sqlserver.source.config.SqlServerSourceConfig;
+import org.apache.flink.cdc.connectors.sqlserver.source.dialect.SqlServerDialect;
+import org.apache.flink.cdc.connectors.sqlserver.source.utils.SqlServerConnectionUtils;
+import org.apache.flink.table.api.ValidationException;
+
+import io.debezium.connector.sqlserver.SqlServerConnection;
+import io.debezium.jdbc.JdbcConnection;
+import io.debezium.relational.Table;
+import io.debezium.relational.history.TableChanges.TableChange;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/** Utilities for converting from debezium {@link Table} types to {@link Schema}. */
+public class SqlServerSchemaUtils {
+
+ private static final Logger LOG = LoggerFactory.getLogger(SqlServerSchemaUtils.class);
+
+ private static final String SQL_SERVER_AGENT_STATUS_QUERY =
+ "SELECT TOP(1) status_desc FROM sys.dm_server_services "
+ + "WHERE servicename LIKE 'SQL Server Agent (%'";
+
+ public static List listSchemas(SqlServerSourceConfig sourceConfig, String namespace) {
+ try (JdbcConnection jdbc =
+ getSqlServerDialect(sourceConfig).openJdbcConnection(sourceConfig)) {
+ return listSchemas(jdbc, namespace);
+ } catch (SQLException e) {
+ throw new RuntimeException("Error to list schemas: " + e.getMessage(), e);
+ }
+ }
+
+ public static List listNamespaces(SqlServerSourceConfig sourceConfig) {
+ try (JdbcConnection jdbc =
+ getSqlServerDialect(sourceConfig).openJdbcConnection(sourceConfig)) {
+ return listNamespaces(jdbc);
+ } catch (SQLException e) {
+ throw new RuntimeException("Error to list namespaces: " + e.getMessage(), e);
+ }
+ }
+
+ public static void validateSqlServerAgentRunning(SqlServerSourceConfig sourceConfig) {
+ try (JdbcConnection jdbc =
+ getSqlServerDialect(sourceConfig).openJdbcConnection(sourceConfig)) {
+ String agentStatus =
+ jdbc.queryAndMap(
+ SQL_SERVER_AGENT_STATUS_QUERY,
+ rs -> rs.next() ? rs.getString(1) : null);
+ if (!"Running".equalsIgnoreCase(agentStatus)) {
+ throw new ValidationException(
+ "SQL Server Agent is not running. Please start SQL Server Agent before"
+ + " creating the SQL Server pipeline.");
+ }
+ } catch (SQLException e) {
+ throw new ValidationException(
+ "Failed to validate SQL Server Agent status from sys.dm_server_services.", e);
+ }
+ }
+
+ public static List listTables(
+ SqlServerSourceConfig sourceConfig,
+ @Nullable String dbName,
+ @Nullable String schemaName) {
+ try (JdbcConnection jdbc =
+ getSqlServerDialect(sourceConfig).openJdbcConnection(sourceConfig)) {
+ List databases =
+ dbName != null
+ ? Collections.singletonList(dbName)
+ : sourceConfig.getDatabaseList();
+
+ List dbzTableIds =
+ SqlServerConnectionUtils.listTables(
+ jdbc, sourceConfig.getTableFilters(), databases);
+
+ return dbzTableIds.stream()
+ .filter(
+ tableId ->
+ schemaName == null
+ || schemaName.equalsIgnoreCase(tableId.schema()))
+ .map(SqlServerSchemaUtils::toCdcTableId)
+ .collect(Collectors.toList());
+ } catch (SQLException e) {
+ throw new RuntimeException("Error to list tables: " + e.getMessage(), e);
+ }
+ }
+
+ public static Schema getTableSchema(SqlServerSourceConfig sourceConfig, TableId tableId) {
+ SqlServerDialect dialect = getSqlServerDialect(sourceConfig);
+ try (JdbcConnection jdbc = dialect.openJdbcConnection(sourceConfig)) {
+ return getTableSchema(tableId, (SqlServerConnection) jdbc, dialect);
+ } catch (SQLException e) {
+ throw new RuntimeException("Error to get table schema: " + e.getMessage(), e);
+ }
+ }
+
+ public static SqlServerDialect getSqlServerDialect(SqlServerSourceConfig sourceConfig) {
+ return new SqlServerDialect(sourceConfig);
+ }
+
+ public static List listSchemas(JdbcConnection jdbc, String namespace)
+ throws SQLException {
+ LOG.info("Read list of available schemas");
+ final List schemaNames = new ArrayList<>();
+
+ String querySql =
+ String.format(
+ "SELECT SCHEMA_NAME FROM %s.INFORMATION_SCHEMA.SCHEMATA", quote(namespace));
+
+ jdbc.query(
+ querySql,
+ rs -> {
+ while (rs.next()) {
+ schemaNames.add(rs.getString(1));
+ }
+ });
+ LOG.info("\t list of available schemas are: {}", schemaNames);
+ return schemaNames;
+ }
+
+ public static List listNamespaces(JdbcConnection jdbc) throws SQLException {
+ LOG.info("Read list of available namespaces (databases)");
+ final List namespaceNames = new ArrayList<>();
+ jdbc.query(
+ "SELECT name FROM sys.databases",
+ rs -> {
+ while (rs.next()) {
+ namespaceNames.add(rs.getString(1));
+ }
+ });
+ LOG.info("\t list of available namespaces are: {}", namespaceNames);
+ return namespaceNames;
+ }
+
+ public static String quote(String dbOrTableName) {
+ return "[" + dbOrTableName.replace("]", "]]") + "]";
+ }
+
+ public static Schema getTableSchema(
+ TableId tableId, SqlServerConnection jdbc, SqlServerDialect dialect) {
+ io.debezium.relational.TableId dbzTableId = toDbzTableId(tableId);
+ return getTableSchema(dbzTableId, jdbc, dialect);
+ }
+
+ public static Schema getTableSchema(
+ io.debezium.relational.TableId tableId,
+ SqlServerConnection jdbc,
+ SqlServerDialect dialect) {
+ try {
+ TableChange tableChange = dialect.queryTableSchema(jdbc, tableId);
+ if (tableChange == null || tableChange.getTable() == null) {
+ throw new RuntimeException("Cannot find table schema for " + tableId);
+ }
+ return toSchema(tableChange.getTable());
+ } catch (Exception e) {
+ throw new RuntimeException("Failed to get table schema for " + tableId, e);
+ }
+ }
+
+ public static Schema toSchema(Table table) {
+ List columns =
+ table.columns().stream()
+ .map(SqlServerSchemaUtils::toColumn)
+ .collect(Collectors.toList());
+
+ return Schema.newBuilder()
+ .setColumns(columns)
+ .primaryKey(table.primaryKeyColumnNames())
+ .comment(table.comment())
+ .build();
+ }
+
+ public static Column toColumn(io.debezium.relational.Column column) {
+ if (column.defaultValueExpression().isPresent()) {
+ String defaultValueExpression =
+ normalizeDefaultValueExpression(column.defaultValueExpression().get());
+ return Column.physicalColumn(
+ column.name(),
+ SqlServerTypeUtils.fromDbzColumn(column),
+ column.comment(),
+ defaultValueExpression);
+ } else {
+ return Column.physicalColumn(
+ column.name(), SqlServerTypeUtils.fromDbzColumn(column), column.comment());
+ }
+ }
+
+ private static String normalizeDefaultValueExpression(String defaultValueExpression) {
+ if (defaultValueExpression == null) {
+ return null;
+ }
+ String trimmed = defaultValueExpression.trim();
+ if (trimmed.isEmpty()) {
+ return trimmed;
+ }
+ String unwrapped = stripOuterParentheses(trimmed);
+ return unquoteSqlServerStringLiteral(unwrapped);
+ }
+
+ private static String stripOuterParentheses(String expression) {
+ String current = expression;
+ while (isWrappedByParentheses(current)) {
+ current = current.substring(1, current.length() - 1).trim();
+ }
+ return current;
+ }
+
+ private static boolean isWrappedByParentheses(String expression) {
+ if (expression.length() < 2
+ || expression.charAt(0) != '('
+ || expression.charAt(expression.length() - 1) != ')') {
+ return false;
+ }
+ int depth = 0;
+ boolean inSingleQuote = false;
+ for (int i = 0; i < expression.length(); i++) {
+ char c = expression.charAt(i);
+ if (c == '\'') {
+ if (inSingleQuote
+ && i + 1 < expression.length()
+ && expression.charAt(i + 1) == '\'') {
+ i++;
+ continue;
+ }
+ inSingleQuote = !inSingleQuote;
+ continue;
+ }
+ if (inSingleQuote) {
+ continue;
+ }
+ if (c == '(') {
+ depth++;
+ } else if (c == ')') {
+ depth--;
+ if (depth == 0 && i < expression.length() - 1) {
+ return false;
+ }
+ }
+ }
+ return depth == 0 && !inSingleQuote;
+ }
+
+ private static String unquoteSqlServerStringLiteral(String expression) {
+ String trimmed = expression.trim();
+ if (trimmed.isEmpty()) {
+ return trimmed;
+ }
+ int quoteIndex = -1;
+ if (trimmed.startsWith("N'") || trimmed.startsWith("n'")) {
+ quoteIndex = 1;
+ } else if (trimmed.charAt(0) == '\'') {
+ quoteIndex = 0;
+ }
+ if (quoteIndex < 0 || trimmed.charAt(quoteIndex) != '\'') {
+ return expression;
+ }
+ StringBuilder literal = new StringBuilder();
+ for (int i = quoteIndex + 1; i < trimmed.length(); i++) {
+ char c = trimmed.charAt(i);
+ if (c == '\'') {
+ if (i + 1 < trimmed.length() && trimmed.charAt(i + 1) == '\'') {
+ literal.append('\'');
+ i++;
+ continue;
+ }
+ if (i == trimmed.length() - 1) {
+ return literal.toString();
+ }
+ return expression;
+ }
+ literal.append(c);
+ }
+ return expression;
+ }
+
+ public static io.debezium.relational.TableId toDbzTableId(TableId tableId) {
+ // SQL Server TableId format: database.schema.table
+ // CDC TableId: namespace (database), schemaName (schema), tableName (table)
+ return new io.debezium.relational.TableId(
+ tableId.getNamespace(), tableId.getSchemaName(), tableId.getTableName());
+ }
+
+ public static TableId toCdcTableId(io.debezium.relational.TableId dbzTableId) {
+ // SQL Server uses database.schema.table structure
+ // Debezium TableId: catalog (database), schema, table
+ // CDC TableId: namespace (database), schemaName (schema), tableName (table)
+ String catalog = dbzTableId.catalog();
+ String schema = dbzTableId.schema();
+ String table = dbzTableId.table();
+
+ LOG.debug(
+ "Converting Debezium TableId to CDC TableId - catalog: {}, schema: {}, table: {}",
+ catalog,
+ schema,
+ table);
+
+ return TableId.tableId(catalog, schema, table);
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/main/java/org/apache/flink/cdc/connectors/sqlserver/utils/SqlServerTypeUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/main/java/org/apache/flink/cdc/connectors/sqlserver/utils/SqlServerTypeUtils.java
new file mode 100644
index 00000000000..4e077c4706f
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/main/java/org/apache/flink/cdc/connectors/sqlserver/utils/SqlServerTypeUtils.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.sqlserver.utils;
+
+import org.apache.flink.cdc.common.types.DataType;
+import org.apache.flink.cdc.common.types.DataTypes;
+import org.apache.flink.table.types.logical.DecimalType;
+
+import io.debezium.relational.Column;
+
+import java.sql.Types;
+
+/** A utility class for converting SQL Server types to Flink CDC types. */
+public class SqlServerTypeUtils {
+
+ // SQL Server specific type names
+ static final String UNIQUE_IDENTIFIER = "uniqueidentifier";
+ static final String XML = "xml";
+ static final String SQL_VARIANT = "sql_variant";
+ static final String HIERARCHY_ID = "hierarchyid";
+ static final String GEOMETRY = "geometry";
+ static final String GEOGRAPHY = "geography";
+ static final String MONEY = "money";
+ static final String SMALL_MONEY = "smallmoney";
+ static final String DATETIME_OFFSET = "datetimeoffset";
+ static final String DATETIME2 = "datetime2";
+ static final String DATETIME = "datetime";
+ static final String SMALL_DATETIME = "smalldatetime";
+ static final String IMAGE = "image";
+ static final String TIMESTAMP = "timestamp";
+ static final String ROW_VERSION = "rowversion";
+ static final String TEXT = "text";
+ static final String N_TEXT = "ntext";
+
+ /** Returns a corresponding Flink CDC data type from a debezium {@link Column}. */
+ public static DataType fromDbzColumn(Column column) {
+ DataType dataType = convertFromColumn(column);
+ if (column.isOptional()) {
+ return dataType;
+ } else {
+ return dataType.notNull();
+ }
+ }
+
+ /**
+ * Returns a corresponding Flink CDC data type from a debezium {@link Column} with nullable
+ * always be true.
+ */
+ private static DataType convertFromColumn(Column column) {
+ int precision = column.length();
+ int scale = column.scale().orElse(0);
+
+ switch (column.jdbcType()) {
+ case Types.BIT:
+ case Types.BOOLEAN:
+ return DataTypes.BOOLEAN();
+ case Types.TINYINT:
+ // SQL Server TINYINT is unsigned 0-255, maps to SMALLINT
+ return DataTypes.SMALLINT();
+ case Types.SMALLINT:
+ return DataTypes.SMALLINT();
+ case Types.INTEGER:
+ return DataTypes.INT();
+ case Types.BIGINT:
+ return DataTypes.BIGINT();
+ case Types.REAL:
+ return DataTypes.FLOAT();
+ case Types.FLOAT:
+ return DataTypes.DOUBLE();
+ case Types.DOUBLE:
+ return DataTypes.DOUBLE();
+ case Types.NUMERIC:
+ case Types.DECIMAL:
+ if (precision > 0 && precision <= DecimalType.MAX_PRECISION) {
+ return DataTypes.DECIMAL(precision, scale);
+ }
+ return DataTypes.DECIMAL(DecimalType.MAX_PRECISION, DecimalType.DEFAULT_SCALE);
+ case Types.CHAR:
+ case Types.NCHAR:
+ return DataTypes.CHAR(precision);
+ case Types.VARCHAR:
+ case Types.NVARCHAR:
+ case Types.LONGVARCHAR:
+ case Types.LONGNVARCHAR:
+ if (precision > 0) {
+ return DataTypes.VARCHAR(precision);
+ }
+ return DataTypes.STRING();
+ case Types.CLOB:
+ case Types.NCLOB:
+ return DataTypes.STRING();
+ case Types.BINARY:
+ case Types.VARBINARY:
+ case Types.LONGVARBINARY:
+ case Types.BLOB:
+ return DataTypes.BYTES();
+ case Types.DATE:
+ return DataTypes.DATE();
+ case Types.TIME:
+ case Types.TIME_WITH_TIMEZONE:
+ return DataTypes.TIME(Math.max(scale, 0));
+ case Types.TIMESTAMP:
+ return DataTypes.TIMESTAMP(scale > 0 ? scale : 6);
+ case Types.TIMESTAMP_WITH_TIMEZONE:
+ return DataTypes.TIMESTAMP_LTZ(scale > 0 ? scale : 6);
+ case Types.STRUCT:
+ // SQL Server specific types like unique identifier, xml, etc.
+ String typeName = column.typeName();
+ if (UNIQUE_IDENTIFIER.equalsIgnoreCase(typeName)) {
+ return DataTypes.STRING();
+ }
+ return DataTypes.STRING();
+ default:
+ // For unknown types, try to handle them as STRING
+ String unknownTypeName = column.typeName();
+ if (unknownTypeName != null) {
+ // Handle SQL Server specific types
+ switch (unknownTypeName.toLowerCase()) {
+ case UNIQUE_IDENTIFIER:
+ case XML:
+ case SQL_VARIANT:
+ case HIERARCHY_ID:
+ case GEOMETRY:
+ case GEOGRAPHY:
+ return DataTypes.STRING();
+ case MONEY:
+ case SMALL_MONEY:
+ return DataTypes.DECIMAL(10, 4);
+ case DATETIME_OFFSET:
+ return DataTypes.TIMESTAMP_LTZ(scale > 0 ? scale : 7);
+ case DATETIME2:
+ return DataTypes.TIMESTAMP(scale > 0 ? scale : 7);
+ case DATETIME:
+ return DataTypes.TIMESTAMP(3);
+ case SMALL_DATETIME:
+ return DataTypes.TIMESTAMP(0);
+ case IMAGE:
+ case TIMESTAMP:
+ case ROW_VERSION:
+ return DataTypes.BYTES();
+ case TEXT:
+ case N_TEXT:
+ return DataTypes.STRING();
+ default:
+ // Fall through to exception
+ }
+ }
+ throw new UnsupportedOperationException(
+ String.format(
+ "Doesn't support SQL Server type '%s', JDBC type '%d' yet.",
+ column.typeName(), column.jdbcType()));
+ }
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/main/resources/META-INF/services/org.apache.flink.cdc.common.factories.Factory b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/main/resources/META-INF/services/org.apache.flink.cdc.common.factories.Factory
new file mode 100644
index 00000000000..341e31b46df
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/main/resources/META-INF/services/org.apache.flink.cdc.common.factories.Factory
@@ -0,0 +1,16 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+org.apache.flink.cdc.connectors.sqlserver.factory.SqlServerDataSourceFactory
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/factory/SqlServerDataSourceFactoryTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/factory/SqlServerDataSourceFactoryTest.java
new file mode 100644
index 00000000000..9057f876bae
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/factory/SqlServerDataSourceFactoryTest.java
@@ -0,0 +1,387 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.sqlserver.factory;
+
+import org.apache.flink.cdc.common.annotation.Internal;
+import org.apache.flink.cdc.common.configuration.ConfigOption;
+import org.apache.flink.cdc.common.configuration.Configuration;
+import org.apache.flink.cdc.common.factories.Factory;
+import org.apache.flink.cdc.connectors.base.options.StartupOptions;
+import org.apache.flink.cdc.connectors.sqlserver.SqlServerTestBase;
+import org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSource;
+import org.apache.flink.table.api.ValidationException;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.HOSTNAME;
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.PASSWORD;
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.PORT;
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_KEY_COLUMN;
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.SCAN_STARTUP_MODE;
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.SCAN_STARTUP_TIMESTAMP_MILLIS;
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.TABLES;
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.TABLES_EXCLUDE;
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.USERNAME;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.testcontainers.containers.MSSQLServerContainer.MS_SQL_SERVER_PORT;
+
+/** Tests for {@link SqlServerDataSourceFactory}. */
+@Internal
+public class SqlServerDataSourceFactoryTest extends SqlServerTestBase {
+
+ private static final String DATABASE_NAME = "inventory";
+
+ @BeforeEach
+ public void before() {
+ initializeSqlServerTable(DATABASE_NAME);
+ }
+
+ @Test
+ public void testCreateDataSource() {
+ Map options = new HashMap<>();
+ options.put(HOSTNAME.key(), MSSQL_SERVER_CONTAINER.getHost());
+ options.put(
+ PORT.key(),
+ String.valueOf(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT)));
+ options.put(USERNAME.key(), MSSQL_SERVER_CONTAINER.getUsername());
+ options.put(PASSWORD.key(), MSSQL_SERVER_CONTAINER.getPassword());
+ options.put(TABLES.key(), DATABASE_NAME + ".dbo.prod\\.*");
+ Factory.Context context = new MockContext(Configuration.fromMap(options));
+ SqlServerDataSourceFactory factory = new SqlServerDataSourceFactory();
+ SqlServerDataSource dataSource = (SqlServerDataSource) factory.createDataSource(context);
+ assertThat(dataSource.getSqlServerSourceConfig().getTableList())
+ .isEqualTo(Arrays.asList("dbo.products", "dbo.products_on_hand"));
+ }
+
+ @Test
+ public void testNoMatchedTable() {
+ Map options = new HashMap<>();
+ options.put(HOSTNAME.key(), MSSQL_SERVER_CONTAINER.getHost());
+ options.put(
+ PORT.key(),
+ String.valueOf(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT)));
+ options.put(USERNAME.key(), MSSQL_SERVER_CONTAINER.getUsername());
+ options.put(PASSWORD.key(), MSSQL_SERVER_CONTAINER.getPassword());
+ String tables = DATABASE_NAME + ".dbo.nonexistent";
+ options.put(TABLES.key(), tables);
+ Factory.Context context = new MockContext(Configuration.fromMap(options));
+
+ SqlServerDataSourceFactory factory = new SqlServerDataSourceFactory();
+ assertThatThrownBy(() -> factory.createDataSource(context))
+ .isInstanceOf(IllegalArgumentException.class)
+ .hasMessageContaining("Cannot find any table by the option 'tables' = " + tables);
+ }
+
+ @Test
+ public void testExcludeTable() {
+ Map options = new HashMap<>();
+ options.put(HOSTNAME.key(), MSSQL_SERVER_CONTAINER.getHost());
+ options.put(
+ PORT.key(),
+ String.valueOf(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT)));
+ options.put(USERNAME.key(), MSSQL_SERVER_CONTAINER.getUsername());
+ options.put(PASSWORD.key(), MSSQL_SERVER_CONTAINER.getPassword());
+ options.put(TABLES.key(), DATABASE_NAME + ".dbo.\\.*");
+ String tableExclude = DATABASE_NAME + ".dbo.orders";
+ options.put(TABLES_EXCLUDE.key(), tableExclude);
+ Factory.Context context = new MockContext(Configuration.fromMap(options));
+
+ SqlServerDataSourceFactory factory = new SqlServerDataSourceFactory();
+ SqlServerDataSource dataSource = (SqlServerDataSource) factory.createDataSource(context);
+ List actualTableList =
+ new ArrayList<>(dataSource.getSqlServerSourceConfig().getTableList());
+ Collections.sort(actualTableList);
+ assertThat(actualTableList).doesNotContain("dbo.orders");
+ }
+
+ @Test
+ public void testExcludeAllTable() {
+ Map options = new HashMap<>();
+ options.put(HOSTNAME.key(), MSSQL_SERVER_CONTAINER.getHost());
+ options.put(
+ PORT.key(),
+ String.valueOf(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT)));
+ options.put(USERNAME.key(), MSSQL_SERVER_CONTAINER.getUsername());
+ options.put(PASSWORD.key(), MSSQL_SERVER_CONTAINER.getPassword());
+ options.put(TABLES.key(), DATABASE_NAME + ".dbo.products");
+ String tableExclude = DATABASE_NAME + ".dbo.products";
+ options.put(TABLES_EXCLUDE.key(), tableExclude);
+ Factory.Context context = new MockContext(Configuration.fromMap(options));
+
+ SqlServerDataSourceFactory factory = new SqlServerDataSourceFactory();
+ assertThatThrownBy(() -> factory.createDataSource(context))
+ .isInstanceOf(IllegalArgumentException.class)
+ .hasMessageContaining(
+ "Cannot find any table with by the option 'tables.exclude' = "
+ + tableExclude);
+ }
+
+ @Test
+ public void testLackRequireOption() {
+ Map options = new HashMap<>();
+ options.put(HOSTNAME.key(), MSSQL_SERVER_CONTAINER.getHost());
+ options.put(
+ PORT.key(),
+ String.valueOf(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT)));
+ options.put(USERNAME.key(), MSSQL_SERVER_CONTAINER.getUsername());
+ options.put(PASSWORD.key(), MSSQL_SERVER_CONTAINER.getPassword());
+ options.put(TABLES.key(), DATABASE_NAME + ".dbo.prod\\.*");
+
+ SqlServerDataSourceFactory factory = new SqlServerDataSourceFactory();
+ List requireKeys =
+ factory.requiredOptions().stream()
+ .map(ConfigOption::key)
+ .collect(Collectors.toList());
+ for (String requireKey : requireKeys) {
+ Map remainingOptions = new HashMap<>(options);
+ remainingOptions.remove(requireKey);
+ Factory.Context context = new MockContext(Configuration.fromMap(remainingOptions));
+
+ assertThatThrownBy(() -> factory.createDataSource(context))
+ .isInstanceOf(ValidationException.class)
+ .hasMessageContaining(
+ String.format(
+ "One or more required options are missing.\n\n"
+ + "Missing required options are:\n\n"
+ + "%s",
+ requireKey));
+ }
+ }
+
+ @Test
+ public void testUnsupportedOption() {
+ Map options = new HashMap<>();
+ options.put(HOSTNAME.key(), MSSQL_SERVER_CONTAINER.getHost());
+ options.put(
+ PORT.key(),
+ String.valueOf(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT)));
+ options.put(USERNAME.key(), MSSQL_SERVER_CONTAINER.getUsername());
+ options.put(PASSWORD.key(), MSSQL_SERVER_CONTAINER.getPassword());
+ options.put(TABLES.key(), DATABASE_NAME + ".dbo.prod\\.*");
+ options.put("unsupported_key", "unsupported_value");
+
+ SqlServerDataSourceFactory factory = new SqlServerDataSourceFactory();
+ Factory.Context context = new MockContext(Configuration.fromMap(options));
+
+ assertThatThrownBy(() -> factory.createDataSource(context))
+ .isInstanceOf(ValidationException.class)
+ .hasMessageContaining(
+ "Unsupported options found for 'sqlserver'.\n\n"
+ + "Unsupported options:\n\n"
+ + "unsupported_key");
+ }
+
+ @Test
+ public void testOptionalOption() {
+ Map options = new HashMap<>();
+ options.put(HOSTNAME.key(), MSSQL_SERVER_CONTAINER.getHost());
+ options.put(
+ PORT.key(),
+ String.valueOf(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT)));
+ options.put(USERNAME.key(), MSSQL_SERVER_CONTAINER.getUsername());
+ options.put(PASSWORD.key(), MSSQL_SERVER_CONTAINER.getPassword());
+ options.put(TABLES.key(), DATABASE_NAME + ".dbo.prod\\.*");
+
+ Factory.Context context = new MockContext(Configuration.fromMap(options));
+ SqlServerDataSourceFactory factory = new SqlServerDataSourceFactory();
+ assertThat(factory.optionalOptions()).contains(PORT);
+
+ SqlServerDataSource dataSource = (SqlServerDataSource) factory.createDataSource(context);
+ assertThat(dataSource.getSqlServerSourceConfig().getPort())
+ .isEqualTo(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT));
+ }
+
+ @Test
+ public void testChunkKeyColumnOptionIsSupported() {
+ Map options = new HashMap<>();
+ options.put(HOSTNAME.key(), MSSQL_SERVER_CONTAINER.getHost());
+ options.put(
+ PORT.key(),
+ String.valueOf(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT)));
+ options.put(USERNAME.key(), MSSQL_SERVER_CONTAINER.getUsername());
+ options.put(PASSWORD.key(), MSSQL_SERVER_CONTAINER.getPassword());
+ options.put(TABLES.key(), DATABASE_NAME + ".dbo.products");
+ options.put(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_KEY_COLUMN.key(), "id");
+
+ Factory.Context context = new MockContext(Configuration.fromMap(options));
+ SqlServerDataSourceFactory factory = new SqlServerDataSourceFactory();
+
+ assertThat(factory.optionalOptions()).contains(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_KEY_COLUMN);
+ SqlServerDataSource dataSource = (SqlServerDataSource) factory.createDataSource(context);
+ assertThat(dataSource.getSqlServerSourceConfig().getChunkKeyColumn()).isEqualTo("id");
+ }
+
+ @Test
+ public void testStartupFromTimestamp() {
+ Map options = new HashMap<>();
+ options.put(HOSTNAME.key(), MSSQL_SERVER_CONTAINER.getHost());
+ options.put(
+ PORT.key(),
+ String.valueOf(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT)));
+ options.put(USERNAME.key(), MSSQL_SERVER_CONTAINER.getUsername());
+ options.put(PASSWORD.key(), MSSQL_SERVER_CONTAINER.getPassword());
+ options.put(TABLES.key(), DATABASE_NAME + ".dbo.prod\\.*");
+ options.put(SCAN_STARTUP_MODE.key(), "timestamp");
+ options.put(SCAN_STARTUP_TIMESTAMP_MILLIS.key(), "1667232000000");
+
+ SqlServerDataSourceFactory factory = new SqlServerDataSourceFactory();
+ Factory.Context context = new MockContext(Configuration.fromMap(options));
+
+ SqlServerDataSource dataSource = (SqlServerDataSource) factory.createDataSource(context);
+ assertThat(dataSource.getSqlServerSourceConfig().getStartupOptions())
+ .isEqualTo(StartupOptions.timestamp(1667232000000L));
+ }
+
+ @Test
+ public void testTimestampStartupRequiresTimestampMillis() {
+ Map options = new HashMap<>();
+ options.put(HOSTNAME.key(), MSSQL_SERVER_CONTAINER.getHost());
+ options.put(
+ PORT.key(),
+ String.valueOf(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT)));
+ options.put(USERNAME.key(), MSSQL_SERVER_CONTAINER.getUsername());
+ options.put(PASSWORD.key(), MSSQL_SERVER_CONTAINER.getPassword());
+ options.put(TABLES.key(), DATABASE_NAME + ".dbo.prod\\.*");
+ options.put(SCAN_STARTUP_MODE.key(), "timestamp");
+
+ SqlServerDataSourceFactory factory = new SqlServerDataSourceFactory();
+ Factory.Context context = new MockContext(Configuration.fromMap(options));
+
+ assertThatThrownBy(() -> factory.createDataSource(context))
+ .isInstanceOf(ValidationException.class)
+ .hasMessageContaining("scan.startup.timestamp-millis");
+ }
+
+ @Test
+ public void testPrefixRequireOption() {
+ Map options = new HashMap<>();
+ options.put(HOSTNAME.key(), MSSQL_SERVER_CONTAINER.getHost());
+ options.put(
+ PORT.key(),
+ String.valueOf(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT)));
+ options.put(USERNAME.key(), MSSQL_SERVER_CONTAINER.getUsername());
+ options.put(PASSWORD.key(), MSSQL_SERVER_CONTAINER.getPassword());
+ options.put(TABLES.key(), DATABASE_NAME + ".dbo.prod\\.*");
+ options.put("debezium.snapshot.mode", "initial");
+ Factory.Context context = new MockContext(Configuration.fromMap(options));
+
+ SqlServerDataSourceFactory factory = new SqlServerDataSourceFactory();
+ SqlServerDataSource dataSource = (SqlServerDataSource) factory.createDataSource(context);
+ assertThat(dataSource.getSqlServerSourceConfig().getTableList())
+ .isEqualTo(Arrays.asList("dbo.products", "dbo.products_on_hand"));
+ }
+
+ @Test
+ public void testJdbcPropertiesAreForwardedToDatabaseConfig() {
+ Map options = new HashMap<>();
+ options.put(HOSTNAME.key(), MSSQL_SERVER_CONTAINER.getHost());
+ options.put(
+ PORT.key(),
+ String.valueOf(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT)));
+ options.put(USERNAME.key(), MSSQL_SERVER_CONTAINER.getUsername());
+ options.put(PASSWORD.key(), MSSQL_SERVER_CONTAINER.getPassword());
+ options.put(TABLES.key(), DATABASE_NAME + ".dbo.prod\\.*");
+ options.put("jdbc.properties.encrypt", "true");
+ options.put("jdbc.properties.trustServerCertificate", "true");
+
+ SqlServerDataSourceFactory factory = new SqlServerDataSourceFactory();
+ Factory.Context context = new MockContext(Configuration.fromMap(options));
+
+ SqlServerDataSource dataSource = (SqlServerDataSource) factory.createDataSource(context);
+ assertThat(dataSource.getSqlServerSourceConfig().getDbzProperties())
+ .containsEntry("database.encrypt", "true")
+ .containsEntry("database.trustServerCertificate", "true");
+ }
+
+ @Test
+ public void testTableValidationWithDifferentDatabases() {
+ Map options = new HashMap<>();
+ options.put(HOSTNAME.key(), MSSQL_SERVER_CONTAINER.getHost());
+ options.put(
+ PORT.key(),
+ String.valueOf(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT)));
+ options.put(USERNAME.key(), MSSQL_SERVER_CONTAINER.getUsername());
+ options.put(PASSWORD.key(), MSSQL_SERVER_CONTAINER.getPassword());
+ options.put(TABLES.key(), "db1.dbo.table1,db2.dbo.table2");
+
+ SqlServerDataSourceFactory factory = new SqlServerDataSourceFactory();
+ Factory.Context context = new MockContext(Configuration.fromMap(options));
+
+ assertThatThrownBy(() -> factory.createDataSource(context))
+ .isInstanceOf(IllegalStateException.class)
+ .hasMessageContaining(
+ "The value of option `tables` is `db1.dbo.table1,db2.dbo.table2`, but not all table names have the same database name");
+ }
+
+ @Test
+ public void testTableValidationRequiresDatabaseSchemaTableFormat() {
+ Map options = new HashMap<>();
+ options.put(HOSTNAME.key(), MSSQL_SERVER_CONTAINER.getHost());
+ options.put(
+ PORT.key(),
+ String.valueOf(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT)));
+ options.put(USERNAME.key(), MSSQL_SERVER_CONTAINER.getUsername());
+ options.put(PASSWORD.key(), MSSQL_SERVER_CONTAINER.getPassword());
+ options.put(TABLES.key(), DATABASE_NAME + ".dbo");
+
+ SqlServerDataSourceFactory factory = new SqlServerDataSourceFactory();
+ Factory.Context context = new MockContext(Configuration.fromMap(options));
+
+ assertThatThrownBy(() -> factory.createDataSource(context))
+ .isInstanceOf(IllegalStateException.class)
+ .hasMessageContaining(
+ "Table '"
+ + DATABASE_NAME
+ + ".dbo' does not match the expected 'database.schema.table' format.")
+ .hasMessageContaining(TABLES.key());
+ }
+
+ static class MockContext implements Factory.Context {
+
+ Configuration factoryConfiguration;
+
+ public MockContext(Configuration factoryConfiguration) {
+ this.factoryConfiguration = factoryConfiguration;
+ }
+
+ @Override
+ public Configuration getFactoryConfiguration() {
+ return factoryConfiguration;
+ }
+
+ @Override
+ public Configuration getPipelineConfiguration() {
+ return null;
+ }
+
+ @Override
+ public ClassLoader getClassLoader() {
+ return this.getClass().getClassLoader();
+ }
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerEventDeserializerTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerEventDeserializerTest.java
new file mode 100644
index 00000000000..f11495c649f
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerEventDeserializerTest.java
@@ -0,0 +1,203 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.sqlserver.source;
+
+import org.apache.flink.cdc.common.event.AddColumnEvent;
+import org.apache.flink.cdc.common.event.CreateTableEvent;
+import org.apache.flink.cdc.common.event.DropTableEvent;
+import org.apache.flink.cdc.common.event.Event;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.debezium.table.DebeziumChangelogMode;
+import org.apache.flink.util.Collector;
+
+import io.debezium.document.DocumentWriter;
+import io.debezium.relational.Column;
+import io.debezium.relational.Table;
+import io.debezium.relational.TableEditor;
+import io.debezium.relational.TableId;
+import io.debezium.relational.history.HistoryRecord;
+import io.debezium.relational.history.TableChanges;
+import io.debezium.relational.history.TableChanges.TableChangeType;
+import org.apache.kafka.connect.data.SchemaBuilder;
+import org.apache.kafka.connect.data.Struct;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link SqlServerEventDeserializer} schema change handling. */
+class SqlServerEventDeserializerTest {
+
+ private static final DocumentWriter DOCUMENT_WRITER = DocumentWriter.defaultWriter();
+
+ @Test
+ void testCreateAlterDropAreEmitted() throws Exception {
+ SqlServerEventDeserializer deserializer =
+ new SqlServerEventDeserializer(DebeziumChangelogMode.ALL, true);
+ List events = new ArrayList<>();
+ TestCollector collector = new TestCollector(events);
+
+ // CREATE
+ SourceRecord createRecord =
+ buildSchemaChangeRecord(
+ TableChangeType.CREATE, Collections.singletonList(col("id", false, 1)));
+ deserializer.deserialize(createRecord, collector);
+
+ assertThat(events).hasSize(1);
+ assertThat(events.get(0)).isInstanceOf(CreateTableEvent.class);
+ Schema createSchema = ((CreateTableEvent) events.get(0)).getSchema();
+ assertThat(createSchema.getColumns()).hasSize(1);
+
+ // ALTER add column
+ SourceRecord alterRecord =
+ buildSchemaChangeRecord(
+ TableChangeType.ALTER,
+ Arrays.asList(col("id", false, 1), col("age", true, 2)));
+ deserializer.deserialize(alterRecord, collector);
+
+ assertThat(events).hasSize(2);
+ assertThat(events.get(1)).isInstanceOf(AddColumnEvent.class);
+ AddColumnEvent addColumnEvent = (AddColumnEvent) events.get(1);
+ assertThat(addColumnEvent.getAddedColumns()).hasSize(1);
+ assertThat(addColumnEvent.getAddedColumns().get(0).getAddColumn().getName())
+ .isEqualTo("age");
+
+ // DROP
+ SourceRecord dropRecord =
+ buildSchemaChangeRecord(TableChangeType.DROP, Collections.emptyList());
+ deserializer.deserialize(dropRecord, collector);
+
+ assertThat(events).hasSize(3);
+ assertThat(events.get(2)).isInstanceOf(DropTableEvent.class);
+ }
+
+ private static Column col(String name, boolean optional, int position) {
+ return Column.editor()
+ .name(name)
+ .jdbcType(java.sql.Types.INTEGER)
+ .type("INT", "INT")
+ .position(position)
+ .optional(optional)
+ .create();
+ }
+
+ private static SourceRecord buildSchemaChangeRecord(TableChangeType type, List columns)
+ throws Exception {
+ TableId tableId = new TableId("db0", "dbo", "users");
+ TableEditor editor = Table.editor().tableId(tableId);
+ columns.forEach(editor::addColumn);
+ if (!columns.isEmpty()) {
+ editor.setPrimaryKeyNames("id");
+ }
+ Table table = editor.create();
+ TableChanges tableChanges = new TableChanges();
+ switch (type) {
+ case CREATE:
+ tableChanges.create(table);
+ break;
+ case ALTER:
+ tableChanges.alter(table);
+ break;
+ case DROP:
+ tableChanges.drop(table);
+ break;
+ default:
+ throw new IllegalArgumentException("Unsupported type " + type);
+ }
+
+ HistoryRecord historyRecord =
+ new HistoryRecord(
+ Collections.singletonMap("file", "test"),
+ Collections.singletonMap("pos", "1"),
+ tableId.catalog(),
+ tableId.schema(),
+ "ddl",
+ tableChanges);
+
+ String historyJson = DOCUMENT_WRITER.write(historyRecord.document());
+
+ org.apache.kafka.connect.data.Schema keySchema =
+ SchemaBuilder.struct()
+ .name("io.debezium.connector.sqlserver.SchemaChangeKey")
+ .field("databaseName", org.apache.kafka.connect.data.Schema.STRING_SCHEMA)
+ .build();
+ Struct keyStruct = new Struct(keySchema).put("databaseName", tableId.catalog());
+
+ org.apache.kafka.connect.data.Schema sourceSchema =
+ SchemaBuilder.struct()
+ .name("source")
+ .field("dummy", org.apache.kafka.connect.data.Schema.OPTIONAL_STRING_SCHEMA)
+ .optional()
+ .build();
+ org.apache.kafka.connect.data.Schema valueSchema =
+ SchemaBuilder.struct()
+ .name("io.debezium.connector.sqlserver.SchemaChangeValue")
+ .field("source", sourceSchema)
+ .field(
+ org.apache.flink.cdc.connectors.base.relational
+ .JdbcSourceEventDispatcher.HISTORY_RECORD_FIELD,
+ org.apache.kafka.connect.data.Schema.STRING_SCHEMA)
+ .build();
+
+ Struct valueStruct =
+ new Struct(valueSchema)
+ .put("source", new Struct(sourceSchema))
+ .put(
+ org.apache.flink.cdc.connectors.base.relational
+ .JdbcSourceEventDispatcher.HISTORY_RECORD_FIELD,
+ historyJson);
+
+ Map partition = new HashMap<>();
+ partition.put("server", "server1");
+ Map offset = new HashMap<>();
+ offset.put("lsn", "1");
+
+ return new SourceRecord(
+ partition,
+ offset,
+ "server1.db0.dbo.users",
+ null,
+ keySchema,
+ keyStruct,
+ valueSchema,
+ valueStruct);
+ }
+
+ private static class TestCollector implements Collector {
+ private final List results;
+
+ private TestCollector(List results) {
+ this.results = results;
+ }
+
+ @Override
+ public void collect(Event record) {
+ results.add(record);
+ }
+
+ @Override
+ public void close() {}
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerFullTypesITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerFullTypesITCase.java
new file mode 100644
index 00000000000..487ba588980
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerFullTypesITCase.java
@@ -0,0 +1,390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.sqlserver.source;
+
+import org.apache.flink.api.common.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.cdc.common.data.DateData;
+import org.apache.flink.cdc.common.data.DecimalData;
+import org.apache.flink.cdc.common.data.LocalZonedTimestampData;
+import org.apache.flink.cdc.common.data.RecordData;
+import org.apache.flink.cdc.common.data.TimeData;
+import org.apache.flink.cdc.common.data.TimestampData;
+import org.apache.flink.cdc.common.data.binary.BinaryStringData;
+import org.apache.flink.cdc.common.event.DataChangeEvent;
+import org.apache.flink.cdc.common.event.Event;
+import org.apache.flink.cdc.common.source.FlinkSourceProvider;
+import org.apache.flink.cdc.common.types.DataTypes;
+import org.apache.flink.cdc.common.types.RowType;
+import org.apache.flink.cdc.connectors.base.options.StartupOptions;
+import org.apache.flink.cdc.connectors.sqlserver.SqlServerTestBase;
+import org.apache.flink.cdc.connectors.sqlserver.factory.SqlServerDataSourceFactory;
+import org.apache.flink.cdc.connectors.sqlserver.source.config.SqlServerSourceConfigFactory;
+import org.apache.flink.cdc.runtime.typeutils.EventTypeInfo;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+import org.apache.flink.util.CloseableIterator;
+
+import org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.math.BigDecimal;
+import java.sql.Connection;
+import java.sql.Statement;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.util.List;
+
+import static org.apache.flink.cdc.connectors.sqlserver.testutils.RecordDataTestUtils.recordFields;
+import static org.apache.flink.cdc.connectors.sqlserver.testutils.SqlServerSourceTestUtils.fetchResultsAndCreateTableEvent;
+import static org.testcontainers.containers.MSSQLServerContainer.MS_SQL_SERVER_PORT;
+
+/** IT case for SQL Server full types support. */
+public class SqlServerFullTypesITCase extends SqlServerTestBase {
+
+ private static final String DATABASE_NAME = "column_type_test";
+
+ private static final StreamExecutionEnvironment env =
+ StreamExecutionEnvironment.getExecutionEnvironment();
+
+ @SuppressWarnings("deprecation")
+ @BeforeEach
+ public void before() {
+ TestValuesTableFactory.clearAllData();
+ env.setParallelism(4);
+ env.enableCheckpointing(200);
+ env.setRestartStrategy(RestartStrategies.noRestart());
+ initializeSqlServerTable(DATABASE_NAME);
+ }
+
+ @Test
+ public void testFullTypes() throws Exception {
+ SqlServerSourceConfigFactory configFactory =
+ (SqlServerSourceConfigFactory)
+ new SqlServerSourceConfigFactory()
+ .hostname(MSSQL_SERVER_CONTAINER.getHost())
+ .port(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT))
+ .username(MSSQL_SERVER_CONTAINER.getUsername())
+ .password(MSSQL_SERVER_CONTAINER.getPassword())
+ .databaseList(DATABASE_NAME)
+ .tableList("dbo.full_types")
+ .startupOptions(StartupOptions.initial())
+ .serverTimeZone("UTC");
+
+ FlinkSourceProvider sourceProvider =
+ (FlinkSourceProvider)
+ new SqlServerDataSource(configFactory).getEventSourceProvider();
+
+ CloseableIterator events =
+ env.fromSource(
+ sourceProvider.getSource(),
+ WatermarkStrategy.noWatermarks(),
+ SqlServerDataSourceFactory.IDENTIFIER,
+ new EventTypeInfo())
+ .executeAndCollect();
+
+ // Verify basic types (id through bigint)
+ Object[] expectedSnapshot =
+ new Object[] {
+ // id
+ 1,
+ // Character types
+ BinaryStringData.fromString("abc"),
+ BinaryStringData.fromString("varchar value"),
+ BinaryStringData.fromString("text value"),
+ BinaryStringData.fromString("中文 "),
+ BinaryStringData.fromString("nvarchar value"),
+ BinaryStringData.fromString("ntext value"),
+ // Numeric types
+ DecimalData.fromBigDecimal(new BigDecimal("123.456"), 6, 3),
+ DecimalData.fromBigDecimal(new BigDecimal("9876543.21"), 10, 2),
+ 3.14159265358979d,
+ 2.71828f,
+ DecimalData.fromBigDecimal(new BigDecimal("214748.3647"), 18, 4),
+ DecimalData.fromBigDecimal(new BigDecimal("130567005.7988"), 18, 4),
+ // Boolean and integer types
+ true,
+ (short) 255,
+ (short) 32767,
+ 2147483647,
+ 9223372036854775807L,
+ // Date and time types
+ DateData.fromEpochDay(18460), // 2020-07-17
+ TimeData.fromMillisOfDay(64822120), // 18:00:22.12
+ TimeData.fromMillisOfDay(64822123), // 18:00:22.1234
+ TimestampData.fromLocalDateTime(
+ LocalDateTime.parse("2020-07-17T18:00:22.123456")), // datetime2
+ LocalZonedTimestampData.fromInstant(
+ toInstant("2020-07-17 18:00:22.1234567")), // datetime offset
+ TimestampData.fromLocalDateTime(
+ LocalDateTime.parse("2020-07-17T18:00:22.123")), // datetime
+ TimestampData.fromLocalDateTime(
+ LocalDateTime.parse("2020-07-17T18:00:00")), // small datetime
+ // Other types
+ BinaryStringData.fromString("value"), // xml
+ new byte[] {1, 2, 3, 4, 5, 6, 7, 8}, // binary(8)
+ new byte[] {72, 101, 108, 108, 111}, // varbinary "Hello"
+ BinaryStringData.fromString(
+ "a0eebc99-9c0b-4ef8-bb6d-6bb9bd380a11"
+ .toUpperCase()) // uniqueidentifier
+ };
+
+ // Verify basic types match
+ List snapshotResults = fetchResultsAndCreateTableEvent(events, 1).f0;
+ RecordData snapshotRecord = ((DataChangeEvent) snapshotResults.get(0)).after();
+ Assertions.assertThat(recordFields(snapshotRecord, FULL_TYPES)).isEqualTo(expectedSnapshot);
+ }
+
+ @Test
+ public void testTimeTypes() throws Exception {
+ SqlServerSourceConfigFactory configFactory =
+ (SqlServerSourceConfigFactory)
+ new SqlServerSourceConfigFactory()
+ .hostname(MSSQL_SERVER_CONTAINER.getHost())
+ .port(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT))
+ .username(MSSQL_SERVER_CONTAINER.getUsername())
+ .password(MSSQL_SERVER_CONTAINER.getPassword())
+ .databaseList(DATABASE_NAME)
+ .tableList("dbo.time_types")
+ .startupOptions(StartupOptions.initial())
+ .serverTimeZone("UTC");
+
+ FlinkSourceProvider sourceProvider =
+ (FlinkSourceProvider)
+ new SqlServerDataSource(configFactory).getEventSourceProvider();
+
+ CloseableIterator events =
+ env.fromSource(
+ sourceProvider.getSource(),
+ WatermarkStrategy.noWatermarks(),
+ SqlServerDataSourceFactory.IDENTIFIER,
+ new EventTypeInfo())
+ .executeAndCollect();
+
+ Object[] expectedSnapshot =
+ new Object[] {
+ // id
+ 1,
+ // Date type
+ DateData.fromEpochDay(18460), // 2020-07-17
+ // Time types with different precisions
+ TimeData.fromMillisOfDay(64822000), // 18:00:22 (precision 0)
+ TimeData.fromMillisOfDay(64822123), // 18:00:22.123 (precision 3)
+ TimeData.fromMillisOfDay(
+ 64822123), // 18:00:22.123456 (precision 6, stored as millis)
+ // Datetime2 types with different precisions
+ TimestampData.fromLocalDateTime(
+ LocalDateTime.parse("2020-07-17T18:00:22")), // datetime2(0)
+ TimestampData.fromLocalDateTime(
+ LocalDateTime.parse("2020-07-17T18:00:22.123")), // datetime2(3)
+ TimestampData.fromLocalDateTime(
+ LocalDateTime.parse("2020-07-17T18:00:22.123456")), // datetime2(6)
+ // Datetimeoffset types with different precisions (UTC+00:00)
+ LocalZonedTimestampData.fromInstant(
+ toInstant("2020-07-17 18:00:22")), // datetimeoffset(0)
+ LocalZonedTimestampData.fromInstant(
+ toInstant("2020-07-17 18:00:22.123")), // datetimeoffset(3)
+ LocalZonedTimestampData.fromInstant(
+ toInstant("2020-07-17 18:00:22.123456")), // datetimeoffset(6)
+ // Datetime and smalldatetime
+ TimestampData.fromLocalDateTime(
+ LocalDateTime.parse("2020-07-17T18:00:22.123")), // datetime
+ TimestampData.fromLocalDateTime(
+ LocalDateTime.parse("2020-07-17T18:00:00")) // smalldatetime
+ };
+
+ List snapshotResults = fetchResultsAndCreateTableEvent(events, 1).f0;
+ RecordData snapshotRecord = ((DataChangeEvent) snapshotResults.get(0)).after();
+ Assertions.assertThat(recordFields(snapshotRecord, TIME_TYPES)).isEqualTo(expectedSnapshot);
+ }
+
+ @Test
+ public void testPrecisionTypes() throws Exception {
+ SqlServerSourceConfigFactory configFactory =
+ (SqlServerSourceConfigFactory)
+ new SqlServerSourceConfigFactory()
+ .hostname(MSSQL_SERVER_CONTAINER.getHost())
+ .port(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT))
+ .username(MSSQL_SERVER_CONTAINER.getUsername())
+ .password(MSSQL_SERVER_CONTAINER.getPassword())
+ .databaseList(DATABASE_NAME)
+ .tableList("dbo.precision_types")
+ .startupOptions(StartupOptions.initial())
+ .serverTimeZone("UTC");
+
+ FlinkSourceProvider sourceProvider =
+ (FlinkSourceProvider)
+ new SqlServerDataSource(configFactory).getEventSourceProvider();
+
+ CloseableIterator events =
+ env.fromSource(
+ sourceProvider.getSource(),
+ WatermarkStrategy.noWatermarks(),
+ SqlServerDataSourceFactory.IDENTIFIER,
+ new EventTypeInfo())
+ .executeAndCollect();
+
+ List snapshotResults = fetchResultsAndCreateTableEvent(events, 1).f0;
+ RecordData snapshotRecord = ((DataChangeEvent) snapshotResults.get(0)).after();
+
+ Object[] expectedSnapshot =
+ new Object[] {
+ 1,
+ DecimalData.fromBigDecimal(new BigDecimal("1234.56"), 6, 2),
+ DecimalData.fromBigDecimal(new BigDecimal("123456.7890"), 10, 4),
+ DecimalData.fromBigDecimal(new BigDecimal("12345678901234.567890"), 20, 6),
+ DecimalData.fromBigDecimal(
+ new BigDecimal("1234567890123456789012345678.9012345678"), 38, 10),
+ DecimalData.fromBigDecimal(new BigDecimal("1234.56"), 6, 2),
+ DecimalData.fromBigDecimal(new BigDecimal("123456.7890"), 10, 4),
+ 3.141592653589793d,
+ 2.7182818f,
+ DecimalData.fromBigDecimal(new BigDecimal("54975581.3896"), 19, 4),
+ DecimalData.fromBigDecimal(new BigDecimal("214748.3647"), 19, 4)
+ };
+
+ Object[] actualSnapshot = recordFields(snapshotRecord, PRECISION_TYPES);
+
+ Assertions.assertThat(actualSnapshot).isEqualTo(expectedSnapshot);
+ }
+
+ @Test
+ public void testStreamingUpdate() throws Exception {
+ SqlServerSourceConfigFactory configFactory =
+ (SqlServerSourceConfigFactory)
+ new SqlServerSourceConfigFactory()
+ .hostname(MSSQL_SERVER_CONTAINER.getHost())
+ .port(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT))
+ .username(MSSQL_SERVER_CONTAINER.getUsername())
+ .password(MSSQL_SERVER_CONTAINER.getPassword())
+ .databaseList(DATABASE_NAME)
+ .tableList("dbo.precision_types")
+ .startupOptions(StartupOptions.initial())
+ .serverTimeZone("UTC");
+
+ FlinkSourceProvider sourceProvider =
+ (FlinkSourceProvider)
+ new SqlServerDataSource(configFactory).getEventSourceProvider();
+
+ CloseableIterator events =
+ env.fromSource(
+ sourceProvider.getSource(),
+ WatermarkStrategy.noWatermarks(),
+ SqlServerDataSourceFactory.IDENTIFIER,
+ new EventTypeInfo())
+ .executeAndCollect();
+
+ // Wait for snapshot to complete
+ List snapshotResults = fetchResultsAndCreateTableEvent(events, 1).f0;
+ Assertions.assertThat(snapshotResults).hasSize(1);
+
+ // Perform update
+ try (Connection connection = getJdbcConnection();
+ Statement statement = connection.createStatement()) {
+ statement.execute("USE " + DATABASE_NAME);
+ statement.execute(
+ "UPDATE dbo.precision_types SET val_decimal_6_2 = 9999.99 WHERE id = 1");
+ }
+
+ // Verify update event
+ List streamResults = fetchResultsAndCreateTableEvent(events, 1).f0;
+ Assertions.assertThat(streamResults).hasSize(1);
+
+ RecordData afterRecord = ((DataChangeEvent) streamResults.get(0)).after();
+ Object[] afterFields = recordFields(afterRecord, PRECISION_TYPES);
+
+ // Verify updated value
+ Assertions.assertThat(afterFields[1])
+ .isEqualTo(DecimalData.fromBigDecimal(new BigDecimal("9999.99"), 6, 2));
+ }
+
+ private Instant toInstant(String ts) {
+ return Timestamp.valueOf(ts).toLocalDateTime().atZone(ZoneId.of("UTC")).toInstant();
+ }
+
+ private static final RowType FULL_TYPES =
+ RowType.of(
+ // id
+ DataTypes.INT().notNull(),
+ // Character types
+ DataTypes.CHAR(3),
+ DataTypes.VARCHAR(1000),
+ DataTypes.STRING(),
+ DataTypes.CHAR(3),
+ DataTypes.VARCHAR(1000),
+ DataTypes.STRING(),
+ // Numeric types
+ DataTypes.DECIMAL(6, 3),
+ DataTypes.DECIMAL(10, 2),
+ DataTypes.DOUBLE(),
+ DataTypes.FLOAT(),
+ DataTypes.DECIMAL(18, 4), // smallmoney - use precision 18 for compact storage
+ DataTypes.DECIMAL(18, 4), // money - use precision 18 for compact storage
+ // Boolean and integer types
+ DataTypes.BOOLEAN(),
+ DataTypes.SMALLINT(),
+ DataTypes.SMALLINT(),
+ DataTypes.INT(),
+ DataTypes.BIGINT(),
+ // Date and time types
+ DataTypes.DATE(),
+ DataTypes.TIME(2),
+ DataTypes.TIME(4),
+ DataTypes.TIMESTAMP(7),
+ DataTypes.TIMESTAMP_LTZ(7),
+ DataTypes.TIMESTAMP(3),
+ DataTypes.TIMESTAMP(0),
+ // Other types
+ DataTypes.STRING(),
+ DataTypes.BYTES(),
+ DataTypes.BYTES(),
+ DataTypes.STRING());
+
+ private static final RowType TIME_TYPES =
+ RowType.of(
+ DataTypes.INT().notNull(),
+ DataTypes.DATE(),
+ DataTypes.TIME(0),
+ DataTypes.TIME(3),
+ DataTypes.TIME(6),
+ DataTypes.TIMESTAMP(0),
+ DataTypes.TIMESTAMP(3),
+ DataTypes.TIMESTAMP(6),
+ DataTypes.TIMESTAMP_LTZ(0),
+ DataTypes.TIMESTAMP_LTZ(3),
+ DataTypes.TIMESTAMP_LTZ(6),
+ DataTypes.TIMESTAMP(3),
+ DataTypes.TIMESTAMP(0));
+
+ private static final RowType PRECISION_TYPES =
+ RowType.of(
+ DataTypes.INT().notNull(),
+ DataTypes.DECIMAL(6, 2),
+ DataTypes.DECIMAL(10, 4),
+ DataTypes.DECIMAL(20, 6),
+ DataTypes.DECIMAL(38, 10),
+ DataTypes.DECIMAL(6, 2),
+ DataTypes.DECIMAL(10, 4),
+ DataTypes.DOUBLE(),
+ DataTypes.FLOAT(),
+ DataTypes.DECIMAL(18, 4), // money - use precision 18 for compact storage
+ DataTypes.DECIMAL(18, 4)); // smallmoney - use precision 18 for compact storage
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerMetadataAccessorITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerMetadataAccessorITCase.java
new file mode 100644
index 00000000000..30e5048bcbe
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerMetadataAccessorITCase.java
@@ -0,0 +1,261 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.sqlserver.source;
+
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.common.types.DataType;
+import org.apache.flink.cdc.common.types.DataTypes;
+import org.apache.flink.cdc.common.types.RowType;
+import org.apache.flink.cdc.connectors.sqlserver.SqlServerTestBase;
+import org.apache.flink.cdc.connectors.sqlserver.source.config.SqlServerSourceConfig;
+import org.apache.flink.cdc.connectors.sqlserver.source.config.SqlServerSourceConfigFactory;
+import org.apache.flink.cdc.connectors.sqlserver.utils.SqlServerSchemaUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.sql.Connection;
+import java.sql.Statement;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.testcontainers.containers.MSSQLServerContainer.MS_SQL_SERVER_PORT;
+
+/** IT cases for {@link SqlServerMetadataAccessor}. */
+public class SqlServerMetadataAccessorITCase extends SqlServerTestBase {
+
+ private static final String DATABASE_NAME = "column_type_test";
+
+ private static final StreamExecutionEnvironment env =
+ StreamExecutionEnvironment.getExecutionEnvironment();
+
+ @BeforeEach
+ public void before() {
+ TestValuesTableFactory.clearAllData();
+ env.setParallelism(4);
+ env.enableCheckpointing(200);
+ initializeSqlServerTable(DATABASE_NAME);
+ }
+
+ @Test
+ public void testListNamespaces() {
+ String[] tables = new String[] {"dbo.full_types"};
+ SqlServerMetadataAccessor metadataAccessor = getMetadataAccessor(tables);
+
+ List namespaces = metadataAccessor.listNamespaces();
+ assertThat(namespaces).contains(DATABASE_NAME);
+ }
+
+ @Test
+ public void testListSchemas() {
+ String[] tables = new String[] {"dbo.full_types"};
+ SqlServerMetadataAccessor metadataAccessor = getMetadataAccessor(tables);
+
+ List schemas = metadataAccessor.listSchemas(DATABASE_NAME);
+ assertThat(schemas).contains("dbo");
+ }
+
+ @Test
+ public void testListSchemasUsesConfiguredDatabaseWhenNamespaceIsNull() {
+ String[] tables = new String[] {"dbo.full_types"};
+ SqlServerMetadataAccessor metadataAccessor = getMetadataAccessor(tables);
+
+ assertThat(metadataAccessor.listSchemas(null)).contains("dbo");
+ }
+
+ @Test
+ public void testListTables() {
+ String[] tables =
+ new String[] {
+ "dbo.full_types", "dbo.time_types", "dbo.precision_types", "axo.precision"
+ };
+ SqlServerMetadataAccessor metadataAccessor = getMetadataAccessor(tables);
+
+ List actualTables = metadataAccessor.listTables(DATABASE_NAME, "dbo");
+
+ List expectedTables =
+ Arrays.asList(
+ TableId.tableId(DATABASE_NAME, "dbo", "full_types"),
+ TableId.tableId(DATABASE_NAME, "dbo", "time_types"),
+ TableId.tableId(DATABASE_NAME, "dbo", "precision_types"));
+
+ assertThat(actualTables).containsExactlyInAnyOrderElementsOf(expectedTables);
+ }
+
+ @Test
+ public void testListTablesFiltersBySchemaName() throws Exception {
+ try (Connection connection = getJdbcConnection();
+ Statement statement = connection.createStatement()) {
+ statement.execute("USE " + DATABASE_NAME);
+ statement.execute("CREATE SCHEMA extra");
+ statement.execute("CREATE TABLE extra.schema_only_table (id INT NOT NULL PRIMARY KEY)");
+ }
+
+ String[] tables = new String[] {"dbo.full_types", "extra.schema_only_table"};
+ SqlServerMetadataAccessor metadataAccessor = getMetadataAccessor(tables);
+
+ assertThat(metadataAccessor.listTables(DATABASE_NAME, "dbo"))
+ .containsExactly(TableId.tableId(DATABASE_NAME, "dbo", "full_types"));
+ assertThat(metadataAccessor.listTables(DATABASE_NAME, "extra"))
+ .containsExactly(TableId.tableId(DATABASE_NAME, "extra", "schema_only_table"));
+ }
+
+ @Test
+ public void testGetSqlServerDialectCreatesPerConfigInstances() {
+ SqlServerSourceConfigFactory fullTypesFactory =
+ (SqlServerSourceConfigFactory)
+ new SqlServerSourceConfigFactory()
+ .hostname(MSSQL_SERVER_CONTAINER.getHost())
+ .port(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT))
+ .username(MSSQL_SERVER_CONTAINER.getUsername())
+ .password(MSSQL_SERVER_CONTAINER.getPassword())
+ .databaseList(DATABASE_NAME)
+ .tableList("dbo.full_types")
+ .serverTimeZone("UTC");
+ SqlServerSourceConfigFactory precisionTypesFactory =
+ (SqlServerSourceConfigFactory)
+ new SqlServerSourceConfigFactory()
+ .hostname(MSSQL_SERVER_CONTAINER.getHost())
+ .port(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT))
+ .username(MSSQL_SERVER_CONTAINER.getUsername())
+ .password(MSSQL_SERVER_CONTAINER.getPassword())
+ .databaseList(DATABASE_NAME)
+ .tableList("dbo.precision_types")
+ .serverTimeZone("UTC");
+
+ assertThat(SqlServerSchemaUtils.getSqlServerDialect(fullTypesFactory.create(0)))
+ .isNotSameAs(
+ SqlServerSchemaUtils.getSqlServerDialect(precisionTypesFactory.create(0)));
+ }
+
+ @Test
+ public void testAccessTimeTypesSchema() {
+ String[] tables = new String[] {"dbo.time_types"};
+ SqlServerMetadataAccessor metadataAccessor = getMetadataAccessor(tables);
+
+ Schema actualSchema =
+ metadataAccessor.getTableSchema(
+ TableId.tableId(DATABASE_NAME, "dbo", "time_types"));
+
+ Schema expectedSchema =
+ Schema.newBuilder()
+ .primaryKey("id")
+ .fromRowDataType(
+ RowType.of(
+ new DataType[] {
+ DataTypes.INT().notNull(),
+ DataTypes.DATE(),
+ DataTypes.TIME(0),
+ DataTypes.TIME(3),
+ DataTypes.TIME(6),
+ DataTypes.TIMESTAMP(6),
+ DataTypes.TIMESTAMP(3),
+ DataTypes.TIMESTAMP(6),
+ DataTypes.TIMESTAMP_LTZ(7),
+ DataTypes.TIMESTAMP_LTZ(3),
+ DataTypes.TIMESTAMP_LTZ(6),
+ DataTypes.TIMESTAMP(3),
+ DataTypes.TIMESTAMP(6)
+ },
+ new String[] {
+ "id",
+ "val_date",
+ "val_time_0",
+ "val_time_3",
+ "val_time_6",
+ "val_datetime2_0",
+ "val_datetime2_3",
+ "val_datetime2_6",
+ "val_datetimeoffset_0",
+ "val_datetimeoffset_3",
+ "val_datetimeoffset_6",
+ "val_datetime",
+ "val_smalldatetime"
+ }))
+ .build();
+
+ assertThat(actualSchema).isEqualTo(expectedSchema);
+ }
+
+ @Test
+ public void testAccessPrecisionTypesSchema() {
+ String[] tables = new String[] {"dbo.precision_types"};
+ SqlServerMetadataAccessor metadataAccessor = getMetadataAccessor(tables);
+
+ Schema actualSchema =
+ metadataAccessor.getTableSchema(
+ TableId.tableId(DATABASE_NAME, "dbo", "precision_types"));
+
+ Schema expectedSchema =
+ Schema.newBuilder()
+ .primaryKey("id")
+ .fromRowDataType(
+ RowType.of(
+ new DataType[] {
+ DataTypes.INT().notNull(),
+ DataTypes.DECIMAL(6, 2),
+ DataTypes.DECIMAL(10, 4),
+ DataTypes.DECIMAL(20, 6),
+ DataTypes.DECIMAL(38, 10),
+ DataTypes.DECIMAL(6, 2),
+ DataTypes.DECIMAL(10, 4),
+ DataTypes.DOUBLE(),
+ DataTypes.FLOAT(),
+ DataTypes.DECIMAL(19, 4),
+ DataTypes.DECIMAL(10, 4)
+ },
+ new String[] {
+ "id",
+ "val_decimal_6_2",
+ "val_decimal_10_4",
+ "val_decimal_20_6",
+ "val_decimal_38_10",
+ "val_numeric_6_2",
+ "val_numeric_10_4",
+ "val_float",
+ "val_real",
+ "val_money",
+ "val_smallmoney"
+ }))
+ .build();
+
+ assertThat(actualSchema).isEqualTo(expectedSchema);
+ }
+
+ private SqlServerMetadataAccessor getMetadataAccessor(String[] tables) {
+ // Debezium SQL Server table.include.list uses schema.table format (without database prefix)
+ SqlServerSourceConfigFactory configFactory =
+ (SqlServerSourceConfigFactory)
+ new SqlServerSourceConfigFactory()
+ .hostname(MSSQL_SERVER_CONTAINER.getHost())
+ .port(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT))
+ .username(MSSQL_SERVER_CONTAINER.getUsername())
+ .password(MSSQL_SERVER_CONTAINER.getPassword())
+ .databaseList(DATABASE_NAME)
+ .tableList(tables)
+ .serverTimeZone("UTC");
+
+ SqlServerSourceConfig sourceConfig = configFactory.create(0);
+
+ return new SqlServerMetadataAccessor(sourceConfig);
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerOnlineSchemaMigrationITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerOnlineSchemaMigrationITCase.java
new file mode 100644
index 00000000000..82422af577f
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerOnlineSchemaMigrationITCase.java
@@ -0,0 +1,270 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.sqlserver.source;
+
+import org.apache.flink.api.common.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.cdc.common.data.binary.BinaryRecordData;
+import org.apache.flink.cdc.common.data.binary.BinaryStringData;
+import org.apache.flink.cdc.common.event.AddColumnEvent;
+import org.apache.flink.cdc.common.event.AlterColumnTypeEvent;
+import org.apache.flink.cdc.common.event.CreateTableEvent;
+import org.apache.flink.cdc.common.event.DataChangeEvent;
+import org.apache.flink.cdc.common.event.DropColumnEvent;
+import org.apache.flink.cdc.common.event.Event;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.schema.PhysicalColumn;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.common.source.FlinkSourceProvider;
+import org.apache.flink.cdc.common.types.DataType;
+import org.apache.flink.cdc.common.types.DataTypes;
+import org.apache.flink.cdc.connectors.base.options.StartupOptions;
+import org.apache.flink.cdc.connectors.sqlserver.SqlServerTestBase;
+import org.apache.flink.cdc.connectors.sqlserver.factory.SqlServerDataSourceFactory;
+import org.apache.flink.cdc.connectors.sqlserver.source.config.SqlServerSourceConfigFactory;
+import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
+import org.apache.flink.cdc.runtime.typeutils.EventTypeInfo;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+import org.apache.flink.util.CloseableIterator;
+
+import org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.sql.Connection;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.cdc.connectors.sqlserver.testutils.SqlServerSourceTestUtils.fetchResults;
+import static org.testcontainers.containers.MSSQLServerContainer.MS_SQL_SERVER_PORT;
+
+/** IT case for evolving SQL Server schema during online migration. The flow mirrors */
+class SqlServerOnlineSchemaMigrationITCase extends SqlServerTestBase {
+
+ private static final String DATABASE_NAME = "customer";
+ private static final StreamExecutionEnvironment env =
+ StreamExecutionEnvironment.getExecutionEnvironment();
+
+ @SuppressWarnings("deprecation")
+ @BeforeEach
+ void before() {
+ initializeSqlServerTable(DATABASE_NAME);
+ TestValuesTableFactory.clearAllData();
+ env.setParallelism(4);
+ env.enableCheckpointing(200);
+ env.setRestartStrategy(RestartStrategies.noRestart());
+ }
+
+ @Test
+ void testSchemaMigrationFromScratch() throws Exception {
+ env.setParallelism(1);
+
+ TableId tableId = TableId.tableId(DATABASE_NAME, "dbo", "customers");
+ SqlServerSourceConfigFactory configFactory =
+ (SqlServerSourceConfigFactory)
+ new SqlServerSourceConfigFactory()
+ .hostname(MSSQL_SERVER_CONTAINER.getHost())
+ .port(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT))
+ .username(MSSQL_SERVER_CONTAINER.getUsername())
+ .password(MSSQL_SERVER_CONTAINER.getPassword())
+ .databaseList(DATABASE_NAME)
+ .tableList("dbo.customers")
+ .startupOptions(StartupOptions.initial())
+ .includeSchemaChanges(true)
+ .serverTimeZone("UTC");
+
+ FlinkSourceProvider sourceProvider =
+ (FlinkSourceProvider)
+ new SqlServerDataSource(configFactory).getEventSourceProvider();
+ CloseableIterator events =
+ env.fromSource(
+ sourceProvider.getSource(),
+ WatermarkStrategy.noWatermarks(),
+ SqlServerDataSourceFactory.IDENTIFIER,
+ new EventTypeInfo())
+ .executeAndCollect();
+
+ List expected = new ArrayList<>();
+ Schema schemaV1 =
+ Schema.newBuilder()
+ .physicalColumn("id", DataTypes.INT().notNull())
+ .physicalColumn("name", DataTypes.VARCHAR(255).notNull(), null, "flink")
+ .physicalColumn("address", DataTypes.VARCHAR(1024))
+ .physicalColumn("phone_number", DataTypes.VARCHAR(512))
+ .primaryKey(Collections.singletonList("id"))
+ .build();
+ expected.add(new CreateTableEvent(tableId, schemaV1));
+ expected.addAll(getSnapshotExpected(tableId, schemaV1));
+ List actual = fetchResults(events, expected.size());
+ Assertions.assertThat(actual.stream().map(Object::toString))
+ .containsExactlyInAnyOrderElementsOf(
+ expected.stream().map(Object::toString).collect(Collectors.toList()));
+
+ // ADD COLUMN
+ try (Connection connection = getJdbcConnection();
+ Statement statement = connection.createStatement()) {
+ statement.execute("USE " + DATABASE_NAME);
+ statement.execute("ALTER TABLE dbo.customers ADD ext INT");
+ statement.execute(
+ "EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'customers', "
+ + "@role_name = NULL, @supports_net_changes = 0, @capture_instance = 'dbo_customers_v2';");
+ statement.execute(
+ "INSERT INTO dbo.customers VALUES (10000, 'Alice', 'Beijing', '123567891234', 17);");
+ }
+
+ Schema schemaV2 =
+ Schema.newBuilder()
+ .physicalColumn("id", DataTypes.INT().notNull())
+ .physicalColumn("name", DataTypes.VARCHAR(255).notNull(), null, "flink")
+ .physicalColumn("address", DataTypes.VARCHAR(1024))
+ .physicalColumn("phone_number", DataTypes.VARCHAR(512))
+ .physicalColumn("ext", DataTypes.INT())
+ .primaryKey(Collections.singletonList("id"))
+ .build();
+
+ Assertions.assertThat(fetchResults(events, 2))
+ .containsExactly(
+ new AddColumnEvent(
+ tableId,
+ Collections.singletonList(
+ new AddColumnEvent.ColumnWithPosition(
+ new PhysicalColumn("ext", DataTypes.INT(), null),
+ AddColumnEvent.ColumnPosition.AFTER,
+ "phone_number"))),
+ DataChangeEvent.insertEvent(
+ tableId,
+ generate(schemaV2, 10000, "Alice", "Beijing", "123567891234", 17)));
+
+ // MODIFY COLUMN
+ try (Connection connection = getJdbcConnection();
+ Statement statement = connection.createStatement()) {
+ statement.execute("USE " + DATABASE_NAME);
+ statement.execute(
+ "EXEC sys.sp_cdc_disable_table @source_schema = 'dbo', @source_name = 'customers', "
+ + "@capture_instance = 'dbo_customers';");
+ statement.execute("ALTER TABLE dbo.customers ALTER COLUMN ext FLOAT");
+ statement.execute(
+ "EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'customers', "
+ + "@role_name = NULL, @supports_net_changes = 0, @capture_instance = 'dbo_customers_v3';");
+ statement.execute(
+ "INSERT INTO dbo.customers VALUES (10001, 'Bob', 'Chongqing', '123567891234', 2.718281828);");
+ }
+
+ Schema schemaV3 =
+ Schema.newBuilder()
+ .physicalColumn("id", DataTypes.INT().notNull())
+ .physicalColumn("name", DataTypes.VARCHAR(255).notNull(), null, "flink")
+ .physicalColumn("address", DataTypes.VARCHAR(1024))
+ .physicalColumn("phone_number", DataTypes.VARCHAR(512))
+ .physicalColumn("ext", DataTypes.DOUBLE())
+ .primaryKey(Collections.singletonList("id"))
+ .build();
+
+ Assertions.assertThat(fetchResults(events, 2))
+ .containsExactly(
+ new AlterColumnTypeEvent(
+ tableId,
+ Collections.singletonMap("ext", DataTypes.DOUBLE()),
+ Collections.singletonMap("ext", DataTypes.INT())),
+ DataChangeEvent.insertEvent(
+ tableId,
+ generate(
+ schemaV3,
+ 10001,
+ "Bob",
+ "Chongqing",
+ "123567891234",
+ 2.718281828)));
+
+ // DROP COLUMN
+ try (Connection connection = getJdbcConnection();
+ Statement statement = connection.createStatement()) {
+ statement.execute("USE " + DATABASE_NAME);
+ statement.execute(
+ "EXEC sys.sp_cdc_disable_table @source_schema = 'dbo', @source_name = 'customers', "
+ + "@capture_instance = 'dbo_customers_v2';");
+ statement.execute("ALTER TABLE dbo.customers DROP COLUMN ext");
+ statement.execute(
+ "EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'customers', "
+ + "@role_name = NULL, @supports_net_changes = 0, @capture_instance = 'dbo_customers_v4';");
+ statement.execute(
+ "INSERT INTO dbo.customers VALUES (10002, 'Cicada', 'Urumqi', '123567891234');");
+ }
+
+ Schema schemaV4 =
+ Schema.newBuilder()
+ .physicalColumn("id", DataTypes.INT().notNull())
+ .physicalColumn("name", DataTypes.VARCHAR(255).notNull(), null, "flink")
+ .physicalColumn("address", DataTypes.VARCHAR(1024))
+ .physicalColumn("phone_number", DataTypes.VARCHAR(512))
+ .primaryKey(Collections.singletonList("id"))
+ .build();
+
+ Assertions.assertThat(fetchResults(events, 2))
+ .containsExactly(
+ new DropColumnEvent(tableId, Collections.singletonList("ext")),
+ DataChangeEvent.insertEvent(
+ tableId,
+ generate(schemaV4, 10002, "Cicada", "Urumqi", "123567891234")));
+ }
+
+ private List getSnapshotExpected(TableId tableId, Schema schema) {
+ return Stream.of(
+ generate(schema, 101, "user_1", "Shanghai", "123567891234"),
+ generate(schema, 102, "user_2", "Shanghai", "123567891234"),
+ generate(schema, 103, "user_3", "Shanghai", "123567891234"),
+ generate(schema, 109, "user_4", "Shanghai", "123567891234"),
+ generate(schema, 110, "user_5", "Shanghai", "123567891234"),
+ generate(schema, 111, "user_6", "Shanghai", "123567891234"),
+ generate(schema, 118, "user_7", "Shanghai", "123567891234"),
+ generate(schema, 121, "user_8", "Shanghai", "123567891234"),
+ generate(schema, 123, "user_9", "Shanghai", "123567891234"),
+ generate(schema, 1009, "user_10", "Shanghai", "123567891234"),
+ generate(schema, 1010, "user_11", "Shanghai", "123567891234"),
+ generate(schema, 1011, "user_12", "Shanghai", "123567891234"),
+ generate(schema, 1012, "user_13", "Shanghai", "123567891234"),
+ generate(schema, 1013, "user_14", "Shanghai", "123567891234"),
+ generate(schema, 1014, "user_15", "Shanghai", "123567891234"),
+ generate(schema, 1015, "user_16", "Shanghai", "123567891234"),
+ generate(schema, 1016, "user_17", "Shanghai", "123567891234"),
+ generate(schema, 1017, "user_18", "Shanghai", "123567891234"),
+ generate(schema, 1018, "user_19", "Shanghai", "123567891234"),
+ generate(schema, 1019, "user_20", "Shanghai", "123567891234"),
+ generate(schema, 2000, "user_21", "Shanghai", "123567891234"))
+ .map(record -> DataChangeEvent.insertEvent(tableId, record))
+ .collect(Collectors.toList());
+ }
+
+ private BinaryRecordData generate(Schema schema, Object... fields) {
+ return (new BinaryRecordDataGenerator(schema.getColumnDataTypes().toArray(new DataType[0])))
+ .generate(
+ Arrays.stream(fields)
+ .map(
+ e ->
+ (e instanceof String)
+ ? BinaryStringData.fromString((String) e)
+ : e)
+ .toArray());
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerParallelizedPipelineITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerParallelizedPipelineITCase.java
new file mode 100644
index 00000000000..594d61983f3
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerParallelizedPipelineITCase.java
@@ -0,0 +1,380 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.sqlserver.source;
+
+import org.apache.flink.cdc.common.configuration.Configuration;
+import org.apache.flink.cdc.common.pipeline.PipelineOptions;
+import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior;
+import org.apache.flink.cdc.composer.PipelineExecution;
+import org.apache.flink.cdc.composer.definition.PipelineDef;
+import org.apache.flink.cdc.composer.definition.SinkDef;
+import org.apache.flink.cdc.composer.definition.SourceDef;
+import org.apache.flink.cdc.composer.flink.FlinkPipelineComposer;
+import org.apache.flink.cdc.connectors.sqlserver.SqlServerTestBase;
+import org.apache.flink.cdc.connectors.sqlserver.factory.SqlServerDataSourceFactory;
+import org.apache.flink.cdc.connectors.sqlserver.testutils.SqlServerSourceTestUtils;
+import org.apache.flink.cdc.connectors.values.ValuesDatabase;
+import org.apache.flink.cdc.connectors.values.factory.ValuesDataFactory;
+import org.apache.flink.cdc.connectors.values.sink.ValuesDataSinkOptions;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+import java.sql.Connection;
+import java.sql.Statement;
+import java.time.Duration;
+import java.util.Collections;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.testcontainers.containers.MSSQLServerContainer.MS_SQL_SERVER_PORT;
+
+/** Parallelized Integration test for SQL Server connector. */
+public class SqlServerParallelizedPipelineITCase extends SqlServerTestBase {
+
+ private static final int PARALLELISM = 4;
+ private static final int TEST_TABLE_NUMBER = 10;
+ private static final String DATABASE_NAME = "parallel_test";
+
+ private final PrintStream standardOut = System.out;
+ private final ByteArrayOutputStream outCaptor = new ByteArrayOutputStream();
+
+ @BeforeEach
+ public void init() {
+ System.setOut(new PrintStream(outCaptor));
+ ValuesDatabase.clear();
+ initializeParallelTestDatabase();
+ }
+
+ @AfterEach
+ public void cleanup() {
+ System.setOut(standardOut);
+ }
+
+ private void initializeParallelTestDatabase() {
+ try (Connection connection = getJdbcConnection();
+ Statement statement = connection.createStatement()) {
+ // Drop database if exists
+ statement.execute(
+ String.format(
+ "IF EXISTS(select 1 from sys.databases where name = '%s') "
+ + "BEGIN ALTER DATABASE [%s] SET SINGLE_USER WITH ROLLBACK IMMEDIATE; "
+ + "DROP DATABASE [%s]; END",
+ DATABASE_NAME, DATABASE_NAME, DATABASE_NAME));
+
+ // Create database
+ statement.execute(String.format("CREATE DATABASE %s;", DATABASE_NAME));
+ statement.execute(String.format("USE %s;", DATABASE_NAME));
+
+ // Wait for SQL Server Agent
+ statement.execute("WAITFOR DELAY '00:00:03';");
+ statement.execute("EXEC sys.sp_cdc_enable_db;");
+
+ // Create multiple test tables
+ for (int i = 1; i <= TEST_TABLE_NUMBER; i++) {
+ statement.execute(
+ String.format(
+ "CREATE TABLE dbo.TABLE%d (ID INT NOT NULL PRIMARY KEY, VERSION VARCHAR(17));",
+ i));
+ statement.execute(
+ String.format("INSERT INTO dbo.TABLE%d VALUES (%d, 'No.%d');", i, i, i));
+ statement.execute(
+ String.format(
+ "EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', "
+ + "@source_name = 'TABLE%d', @role_name = NULL, @supports_net_changes = 0;",
+ i));
+ }
+ } catch (Exception e) {
+ throw new RuntimeException("Failed to initialize parallel test database", e);
+ }
+ }
+
+ @Test
+ void testParallelizedSnapshotReading() throws Exception {
+ FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster();
+ // Start checkpointing so snapshot completion is check-pointed and stream splits can follow.
+ composer.getEnv().enableCheckpointing(2000);
+
+ // Setup SQL Server source
+ Configuration sourceConfig = new Configuration();
+ sourceConfig.set(SqlServerDataSourceOptions.HOSTNAME, MSSQL_SERVER_CONTAINER.getHost());
+ sourceConfig.set(
+ SqlServerDataSourceOptions.PORT,
+ MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT));
+ sourceConfig.set(SqlServerDataSourceOptions.USERNAME, MSSQL_SERVER_CONTAINER.getUsername());
+ sourceConfig.set(SqlServerDataSourceOptions.PASSWORD, MSSQL_SERVER_CONTAINER.getPassword());
+ sourceConfig.set(SqlServerDataSourceOptions.SERVER_TIME_ZONE, "UTC");
+ sourceConfig.set(SqlServerDataSourceOptions.TABLES, DATABASE_NAME + ".dbo.\\.*");
+
+ SourceDef sourceDef =
+ new SourceDef(
+ SqlServerDataSourceFactory.IDENTIFIER, "SQL Server Source", sourceConfig);
+
+ // Setup value sink
+ Configuration sinkConfig = new Configuration();
+ sinkConfig.set(ValuesDataSinkOptions.MATERIALIZED_IN_MEMORY, true);
+ SinkDef sinkDef = new SinkDef(ValuesDataFactory.IDENTIFIER, "Value Sink", sinkConfig);
+
+ // Setup pipeline
+ Configuration pipelineConfig = new Configuration();
+ pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, PARALLELISM);
+ pipelineConfig.set(
+ PipelineOptions.PIPELINE_SCHEMA_CHANGE_BEHAVIOR, SchemaChangeBehavior.EVOLVE);
+ PipelineDef pipelineDef =
+ new PipelineDef(
+ sourceDef,
+ sinkDef,
+ Collections.emptyList(),
+ Collections.emptyList(),
+ Collections.emptyList(),
+ pipelineConfig);
+
+ // Execute the pipeline
+ PipelineExecution execution = composer.compose(pipelineDef);
+ AtomicReference pipelineFailure = new AtomicReference<>();
+ Thread executeThread =
+ new Thread(
+ () -> {
+ try {
+ execution.execute();
+ } catch (InterruptedException ignored) {
+ Thread.currentThread().interrupt();
+ } catch (CancellationException ignored) {
+ } catch (Throwable t) {
+ pipelineFailure.compareAndSet(null, t);
+ }
+ });
+
+ executeThread.start();
+
+ // Expected: each sink subtask prints its CreateTableEvent plus one DataChangeEvent per
+ // table.
+ int expectedMinimumEvents = TEST_TABLE_NUMBER * (PARALLELISM + 1);
+
+ try {
+ SqlServerSourceTestUtils.loopCheck(
+ () -> {
+ if (pipelineFailure.get() != null) {
+ throw new RuntimeException(pipelineFailure.get());
+ }
+ return outCaptor.toString().trim().split("\n").length
+ >= expectedMinimumEvents;
+ },
+ "collect enough rows",
+ Duration.ofSeconds(120),
+ Duration.ofSeconds(1));
+ } finally {
+ executeThread.interrupt();
+ executeThread.join(Duration.ofSeconds(30).toMillis());
+ }
+
+ if (pipelineFailure.get() != null) {
+ throw new RuntimeException("Pipeline execution failed", pipelineFailure.get());
+ }
+
+ // Verify all tables have been captured
+ String outputEvents = outCaptor.toString();
+
+ // Verify CreateTableEvents for all tables
+ assertThat(outputEvents)
+ .contains(
+ IntStream.range(0, PARALLELISM)
+ .boxed()
+ .flatMap(
+ subtask ->
+ IntStream.rangeClosed(1, TEST_TABLE_NUMBER)
+ .mapToObj(
+ i ->
+ String.format(
+ "%d> CreateTableEvent{tableId=%s.dbo.TABLE%d",
+ subtask,
+ DATABASE_NAME,
+ i)))
+ .toArray(String[]::new));
+
+ // Verify DataChangeEvents for all tables
+ IntStream.rangeClosed(1, TEST_TABLE_NUMBER)
+ .forEach(
+ i ->
+ assertThat(outputEvents)
+ .contains(
+ String.format(
+ "DataChangeEvent{tableId=%s.dbo.TABLE%d",
+ DATABASE_NAME, i)));
+ }
+
+ @Test
+ void testParallelizedWithStreamingChanges() throws Exception {
+ FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster();
+ // Ensure checkpointing so the source can finalize snapshot splits and start streaming.
+ composer.getEnv().enableCheckpointing(2000);
+
+ // Setup SQL Server source
+ Configuration sourceConfig = new Configuration();
+ sourceConfig.set(SqlServerDataSourceOptions.HOSTNAME, MSSQL_SERVER_CONTAINER.getHost());
+ sourceConfig.set(
+ SqlServerDataSourceOptions.PORT,
+ MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT));
+ sourceConfig.set(SqlServerDataSourceOptions.USERNAME, MSSQL_SERVER_CONTAINER.getUsername());
+ sourceConfig.set(SqlServerDataSourceOptions.PASSWORD, MSSQL_SERVER_CONTAINER.getPassword());
+ sourceConfig.set(SqlServerDataSourceOptions.SERVER_TIME_ZONE, "UTC");
+ // Only capture first 3 tables
+ final int capturedTableCount = 3;
+ sourceConfig.set(
+ SqlServerDataSourceOptions.TABLES,
+ IntStream.rangeClosed(1, capturedTableCount)
+ .mapToObj(i -> DATABASE_NAME + ".dbo.TABLE" + i)
+ .collect(Collectors.joining(",")));
+
+ SourceDef sourceDef =
+ new SourceDef(
+ SqlServerDataSourceFactory.IDENTIFIER, "SQL Server Source", sourceConfig);
+
+ // Setup value sink
+ Configuration sinkConfig = new Configuration();
+ sinkConfig.set(ValuesDataSinkOptions.MATERIALIZED_IN_MEMORY, true);
+ SinkDef sinkDef = new SinkDef(ValuesDataFactory.IDENTIFIER, "Value Sink", sinkConfig);
+
+ // Setup pipeline
+ Configuration pipelineConfig = new Configuration();
+ pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, PARALLELISM);
+ pipelineConfig.set(
+ PipelineOptions.PIPELINE_SCHEMA_CHANGE_BEHAVIOR, SchemaChangeBehavior.EVOLVE);
+ PipelineDef pipelineDef =
+ new PipelineDef(
+ sourceDef,
+ sinkDef,
+ Collections.emptyList(),
+ Collections.emptyList(),
+ Collections.emptyList(),
+ pipelineConfig);
+
+ // Execute the pipeline
+ PipelineExecution execution = composer.compose(pipelineDef);
+ AtomicReference pipelineFailure = new AtomicReference<>();
+ Thread executeThread =
+ new Thread(
+ () -> {
+ try {
+ execution.execute();
+ } catch (InterruptedException ignored) {
+ Thread.currentThread().interrupt();
+ } catch (CancellationException ignored) {
+ } catch (Throwable t) {
+ pipelineFailure.compareAndSet(null, t);
+ }
+ });
+
+ executeThread.start();
+
+ // Wait for snapshot completion
+ int expectedSnapshotEvents = capturedTableCount * (PARALLELISM + 1);
+ try {
+ SqlServerSourceTestUtils.loopCheck(
+ () -> {
+ if (pipelineFailure.get() != null) {
+ throw new RuntimeException(pipelineFailure.get());
+ }
+ return outCaptor.toString().trim().split("\n").length
+ >= expectedSnapshotEvents;
+ },
+ "collect snapshot events",
+ Duration.ofSeconds(60),
+ Duration.ofSeconds(1));
+ } catch (Exception e) {
+ executeThread.interrupt();
+ throw e;
+ }
+
+ if (pipelineFailure.get() != null) {
+ throw new RuntimeException("Pipeline execution failed", pipelineFailure.get());
+ }
+
+ // Perform streaming updates
+ try (Connection connection = getJdbcConnection();
+ Statement statement = connection.createStatement()) {
+ statement.execute("USE " + DATABASE_NAME);
+ statement.execute("UPDATE dbo.TABLE1 SET VERSION = 'Updated' WHERE ID = 1");
+ statement.execute("INSERT INTO dbo.TABLE2 VALUES (100, 'New Record')");
+ statement.execute("DELETE FROM dbo.TABLE3 WHERE ID = 3");
+ }
+
+ // Wait for streaming events
+ int expectedTotalEvents = expectedSnapshotEvents + 3; // 3 additional streaming events
+ try {
+ SqlServerSourceTestUtils.loopCheck(
+ () -> {
+ if (pipelineFailure.get() != null) {
+ throw new RuntimeException(pipelineFailure.get());
+ }
+ return outCaptor.toString().trim().split("\n").length
+ >= expectedTotalEvents;
+ },
+ "collect streaming events",
+ Duration.ofSeconds(60),
+ Duration.ofSeconds(1));
+ } finally {
+ executeThread.interrupt();
+ executeThread.join(Duration.ofSeconds(30).toMillis());
+ }
+
+ if (pipelineFailure.get() != null) {
+ throw new RuntimeException("Pipeline execution failed", pipelineFailure.get());
+ }
+
+ String outputEvents = outCaptor.toString();
+
+ // Verify CreateTableEvents for captured tables (one per sink subtask)
+ assertThat(outputEvents)
+ .contains(
+ IntStream.range(0, PARALLELISM)
+ .boxed()
+ .flatMap(
+ subtask ->
+ IntStream.rangeClosed(1, capturedTableCount)
+ .mapToObj(
+ i ->
+ String.format(
+ "%d> CreateTableEvent{tableId=%s.dbo.TABLE%d",
+ subtask,
+ DATABASE_NAME,
+ i)))
+ .toArray(String[]::new));
+
+ // Verify snapshot events
+ IntStream.rangeClosed(1, capturedTableCount)
+ .forEach(
+ i ->
+ assertThat(outputEvents)
+ .contains(
+ String.format(
+ "DataChangeEvent{tableId=%s.dbo.TABLE%d",
+ DATABASE_NAME, i)));
+
+ // Verify streaming events
+ assertThat(outputEvents).contains("op=UPDATE");
+ assertThat(outputEvents).contains("100, New Record");
+ assertThat(outputEvents).contains("op=DELETE");
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerPipelineITCaseTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerPipelineITCaseTest.java
new file mode 100644
index 00000000000..06034937566
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerPipelineITCaseTest.java
@@ -0,0 +1,468 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.sqlserver.source;
+
+import org.apache.flink.api.common.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.cdc.common.configuration.Configuration;
+import org.apache.flink.cdc.common.data.binary.BinaryStringData;
+import org.apache.flink.cdc.common.event.CreateTableEvent;
+import org.apache.flink.cdc.common.event.DataChangeEvent;
+import org.apache.flink.cdc.common.event.Event;
+import org.apache.flink.cdc.common.event.SchemaChangeEvent;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.factories.Factory;
+import org.apache.flink.cdc.common.factories.FactoryHelper;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.common.source.FlinkSourceProvider;
+import org.apache.flink.cdc.common.types.DataType;
+import org.apache.flink.cdc.common.types.DataTypes;
+import org.apache.flink.cdc.common.types.RowType;
+import org.apache.flink.cdc.connectors.base.options.StartupOptions;
+import org.apache.flink.cdc.connectors.sqlserver.SqlServerTestBase;
+import org.apache.flink.cdc.connectors.sqlserver.factory.SqlServerDataSourceFactory;
+import org.apache.flink.cdc.connectors.sqlserver.source.config.SqlServerSourceConfigFactory;
+import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
+import org.apache.flink.cdc.runtime.typeutils.EventTypeInfo;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.sql.Connection;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.testcontainers.containers.MSSQLServerContainer.MS_SQL_SERVER_PORT;
+
+/** Integration tests for SQL Server pipeline source. */
+public class SqlServerPipelineITCaseTest extends SqlServerTestBase {
+ private static final String DATABASE_NAME = "inventory";
+
+ private static final StreamExecutionEnvironment env =
+ StreamExecutionEnvironment.getExecutionEnvironment();
+
+ @SuppressWarnings("deprecation")
+ @BeforeEach
+ public void before() {
+ TestValuesTableFactory.clearAllData();
+ env.setParallelism(4);
+ env.enableCheckpointing(200);
+ env.setRestartStrategy(RestartStrategies.noRestart());
+ initializeSqlServerTable(DATABASE_NAME);
+ }
+
+ @Test
+ public void testInitialStartupMode() throws Exception {
+ SqlServerSourceConfigFactory configFactory =
+ (SqlServerSourceConfigFactory)
+ new SqlServerSourceConfigFactory()
+ .hostname(MSSQL_SERVER_CONTAINER.getHost())
+ .port(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT))
+ .username(MSSQL_SERVER_CONTAINER.getUsername())
+ .password(MSSQL_SERVER_CONTAINER.getPassword())
+ .databaseList(DATABASE_NAME)
+ .tableList("dbo.products")
+ .startupOptions(StartupOptions.initial())
+ .serverTimeZone("UTC");
+
+ FlinkSourceProvider sourceProvider =
+ (FlinkSourceProvider)
+ new SqlServerDataSource(configFactory).getEventSourceProvider();
+ CloseableIterator events =
+ env.fromSource(
+ sourceProvider.getSource(),
+ WatermarkStrategy.noWatermarks(),
+ SqlServerDataSourceFactory.IDENTIFIER,
+ new EventTypeInfo())
+ .executeAndCollect();
+
+ TableId tableId = TableId.tableId(DATABASE_NAME, "dbo", "products");
+ CreateTableEvent createTableEvent = getProductsCreateTableEvent(tableId);
+
+ // generate snapshot data
+ List expectedSnapshot = getSnapshotExpected(tableId);
+
+ // In this configuration, several subtasks might emit their corresponding CreateTableEvent
+ // to downstream. Since it is not possible to predict how many CreateTableEvents should we
+ // expect, we simply filter them out from expected sets, and assert there's at least one.
+ List actual = fetchResultsExcept(events, expectedSnapshot.size(), createTableEvent);
+ assertThat(actual.subList(0, expectedSnapshot.size()))
+ .containsExactlyInAnyOrder(expectedSnapshot.toArray(new Event[0]));
+ }
+
+ @ParameterizedTest(name = "unboundedChunkFirst: {0}")
+ @ValueSource(booleans = {true, false})
+ public void testInitialStartupModeWithOpts(boolean unboundedChunkFirst) throws Exception {
+ Configuration sourceConfiguration = new Configuration();
+ sourceConfiguration.set(
+ SqlServerDataSourceOptions.HOSTNAME, MSSQL_SERVER_CONTAINER.getHost());
+ sourceConfiguration.set(
+ SqlServerDataSourceOptions.PORT,
+ MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT));
+ sourceConfiguration.set(
+ SqlServerDataSourceOptions.USERNAME, MSSQL_SERVER_CONTAINER.getUsername());
+ sourceConfiguration.set(
+ SqlServerDataSourceOptions.PASSWORD, MSSQL_SERVER_CONTAINER.getPassword());
+ sourceConfiguration.set(
+ SqlServerDataSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP, false);
+ sourceConfiguration.set(SqlServerDataSourceOptions.TABLES, DATABASE_NAME + ".dbo.products");
+ sourceConfiguration.set(SqlServerDataSourceOptions.SERVER_TIME_ZONE, "UTC");
+ sourceConfiguration.set(
+ SqlServerDataSourceOptions.METADATA_LIST,
+ "database_name,schema_name,table_name,op_ts");
+ sourceConfiguration.set(
+ SqlServerDataSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED,
+ unboundedChunkFirst);
+
+ Factory.Context context =
+ new FactoryHelper.DefaultContext(
+ sourceConfiguration, new Configuration(), this.getClass().getClassLoader());
+ FlinkSourceProvider sourceProvider =
+ (FlinkSourceProvider)
+ new SqlServerDataSourceFactory()
+ .createDataSource(context)
+ .getEventSourceProvider();
+ CloseableIterator events =
+ env.fromSource(
+ sourceProvider.getSource(),
+ WatermarkStrategy.noWatermarks(),
+ SqlServerDataSourceFactory.IDENTIFIER,
+ new EventTypeInfo())
+ .executeAndCollect();
+
+ TableId tableId = TableId.tableId(DATABASE_NAME, "dbo", "products");
+ CreateTableEvent createTableEvent = getProductsCreateTableEvent(tableId);
+
+ // generate snapshot data
+ Map meta = new HashMap<>();
+ meta.put("database_name", DATABASE_NAME);
+ meta.put("schema_name", "dbo");
+ meta.put("table_name", "products");
+ meta.put("op_ts", "0");
+
+ // generate snapshot data
+ List expectedSnapshot =
+ getSnapshotExpected(tableId).stream()
+ .map(
+ event -> {
+ DataChangeEvent dataChangeEvent = (DataChangeEvent) event;
+ return DataChangeEvent.insertEvent(
+ dataChangeEvent.tableId(),
+ dataChangeEvent.after(),
+ meta);
+ })
+ .collect(Collectors.toList());
+
+ String startTime = String.valueOf(System.currentTimeMillis());
+ Thread.sleep(1000);
+
+ List expectedLog = new ArrayList<>();
+
+ try (Connection connection = getJdbcConnection();
+ Statement statement = connection.createStatement()) {
+ statement.execute("USE " + DATABASE_NAME);
+
+ RowType rowType =
+ RowType.of(
+ new DataType[] {
+ DataTypes.INT().notNull(),
+ DataTypes.VARCHAR(255).notNull(),
+ DataTypes.VARCHAR(512),
+ DataTypes.DOUBLE()
+ },
+ new String[] {"id", "name", "description", "weight"});
+ BinaryRecordDataGenerator generator = new BinaryRecordDataGenerator(rowType);
+ statement.execute(
+ "INSERT INTO dbo.products(name,description,weight) VALUES ('scooter','c-2',5.5)"); // 110
+ expectedLog.add(
+ DataChangeEvent.insertEvent(
+ tableId,
+ generator.generate(
+ new Object[] {
+ 110,
+ BinaryStringData.fromString("scooter"),
+ BinaryStringData.fromString("c-2"),
+ 5.5d
+ })));
+ statement.execute(
+ "INSERT INTO dbo.products(name,description,weight) VALUES ('football','c-11',6.6)"); // 111
+ expectedLog.add(
+ DataChangeEvent.insertEvent(
+ tableId,
+ generator.generate(
+ new Object[] {
+ 111,
+ BinaryStringData.fromString("football"),
+ BinaryStringData.fromString("c-11"),
+ 6.6d
+ })));
+ statement.execute("UPDATE dbo.products SET description='c-12' WHERE id=110");
+
+ expectedLog.add(
+ DataChangeEvent.updateEvent(
+ tableId,
+ generator.generate(
+ new Object[] {
+ 110,
+ BinaryStringData.fromString("scooter"),
+ BinaryStringData.fromString("c-2"),
+ 5.5d
+ }),
+ generator.generate(
+ new Object[] {
+ 110,
+ BinaryStringData.fromString("scooter"),
+ BinaryStringData.fromString("c-12"),
+ 5.5d
+ })));
+
+ statement.execute("DELETE FROM dbo.products WHERE id = 111");
+ expectedLog.add(
+ DataChangeEvent.deleteEvent(
+ tableId,
+ generator.generate(
+ new Object[] {
+ 111,
+ BinaryStringData.fromString("football"),
+ BinaryStringData.fromString("c-11"),
+ 6.6d
+ })));
+ }
+
+ // In this configuration, several subtasks might emit their corresponding CreateTableEvent
+ // to downstream. Since it is not possible to predict how many CreateTableEvents should we
+ // expect, we simply filter them out from expected sets, and assert there's at least one.
+ int snapshotRecordsCount = expectedSnapshot.size();
+ int logRecordsCount = expectedLog.size();
+
+ // Ditto, CreateTableEvent might be emitted in multiple partitions.
+ List actual =
+ fetchResultsExcept(
+ events, snapshotRecordsCount + logRecordsCount, createTableEvent);
+
+ List actualSnapshotEvents = actual.subList(0, snapshotRecordsCount);
+ List actualLogEvents = actual.subList(snapshotRecordsCount, actual.size());
+
+ assertThat(actualSnapshotEvents).containsExactlyInAnyOrderElementsOf(expectedSnapshot);
+ assertThat(actualLogEvents).hasSize(logRecordsCount);
+
+ for (int i = 0; i < logRecordsCount; i++) {
+ if (expectedLog.get(i) instanceof SchemaChangeEvent) {
+ assertThat(actualLogEvents.get(i)).isEqualTo(expectedLog.get(i));
+ } else {
+ DataChangeEvent expectedEvent = (DataChangeEvent) expectedLog.get(i);
+ DataChangeEvent actualEvent = (DataChangeEvent) actualLogEvents.get(i);
+ assertThat(actualEvent.op()).isEqualTo(expectedEvent.op());
+ assertThat(actualEvent.before()).isEqualTo(expectedEvent.before());
+ assertThat(actualEvent.after()).isEqualTo(expectedEvent.after());
+ assertThat(actualEvent.meta().get("database_name")).isEqualTo(DATABASE_NAME);
+ assertThat(actualEvent.meta().get("schema_name")).isEqualTo("dbo");
+ assertThat(actualEvent.meta().get("table_name")).isEqualTo("products");
+ assertThat(actualEvent.meta().get("op_ts")).isGreaterThanOrEqualTo(startTime);
+ }
+ }
+ }
+
+ @Test
+ public void testSnapshotOnlyMode() throws Exception {
+ SqlServerSourceConfigFactory configFactory =
+ (SqlServerSourceConfigFactory)
+ new SqlServerSourceConfigFactory()
+ .hostname(MSSQL_SERVER_CONTAINER.getHost())
+ .port(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT))
+ .username(MSSQL_SERVER_CONTAINER.getUsername())
+ .password(MSSQL_SERVER_CONTAINER.getPassword())
+ .databaseList(DATABASE_NAME)
+ .tableList("dbo.products")
+ .startupOptions(StartupOptions.snapshot())
+ .skipSnapshotBackfill(false)
+ .serverTimeZone("UTC");
+
+ FlinkSourceProvider sourceProvider =
+ (FlinkSourceProvider)
+ new SqlServerDataSource(configFactory).getEventSourceProvider();
+ CloseableIterator events =
+ env.fromSource(
+ sourceProvider.getSource(),
+ WatermarkStrategy.noWatermarks(),
+ SqlServerDataSourceFactory.IDENTIFIER,
+ new EventTypeInfo())
+ .executeAndCollect();
+
+ TableId tableId = TableId.tableId(DATABASE_NAME, "dbo", "products");
+ CreateTableEvent createTableEvent = getProductsCreateTableEvent(tableId);
+
+ // generate snapshot data
+ List expectedSnapshot = getSnapshotExpected(tableId);
+
+ // In this configuration, several subtasks might emit their corresponding CreateTableEvent
+ // to downstream. Since it is not possible to predict how many CreateTableEvents should we
+ // expect, we simply filter them out from expected sets, and assert there's at least one.
+ List actual = fetchResultsExcept(events, expectedSnapshot.size(), createTableEvent);
+ assertThat(actual.subList(0, expectedSnapshot.size()))
+ .containsExactlyInAnyOrder(expectedSnapshot.toArray(new Event[0]));
+ }
+
+ private static List fetchResultsExcept(Iterator iter, int size, T sideEvent) {
+ List result = new ArrayList<>(size);
+ List sideResults = new ArrayList<>();
+ while (size > 0 && iter.hasNext()) {
+ T event = iter.next();
+ if (sideEvent.getClass().isInstance(event)) {
+ sideResults.add(event);
+ } else {
+ result.add(event);
+ size--;
+ }
+ }
+ // Also ensure we've received at least one or many side events.
+ assertThat(sideResults).isNotEmpty();
+ return result;
+ }
+
+ private List getSnapshotExpected(TableId tableId) {
+ RowType rowType =
+ RowType.of(
+ new DataType[] {
+ DataTypes.INT().notNull(),
+ DataTypes.VARCHAR(255).notNull(),
+ DataTypes.VARCHAR(512),
+ DataTypes.DOUBLE()
+ },
+ new String[] {"id", "name", "description", "weight"});
+ BinaryRecordDataGenerator generator = new BinaryRecordDataGenerator(rowType);
+ List snapshotExpected = new ArrayList<>();
+ snapshotExpected.add(
+ DataChangeEvent.insertEvent(
+ tableId,
+ generator.generate(
+ new Object[] {
+ 101,
+ BinaryStringData.fromString("scooter"),
+ BinaryStringData.fromString("Small 2-wheel scooter"),
+ 3.14d
+ })));
+ snapshotExpected.add(
+ DataChangeEvent.insertEvent(
+ tableId,
+ generator.generate(
+ new Object[] {
+ 102,
+ BinaryStringData.fromString("car battery"),
+ BinaryStringData.fromString("12V car battery"),
+ 8.1d
+ })));
+ snapshotExpected.add(
+ DataChangeEvent.insertEvent(
+ tableId,
+ generator.generate(
+ new Object[] {
+ 103,
+ BinaryStringData.fromString("12-pack drill bits"),
+ BinaryStringData.fromString(
+ "12-pack of drill bits with sizes ranging from #40 to #3"),
+ 0.8d
+ })));
+ snapshotExpected.add(
+ DataChangeEvent.insertEvent(
+ tableId,
+ generator.generate(
+ new Object[] {
+ 104,
+ BinaryStringData.fromString("hammer"),
+ BinaryStringData.fromString("12oz carpenter's hammer"),
+ 0.75d
+ })));
+ snapshotExpected.add(
+ DataChangeEvent.insertEvent(
+ tableId,
+ generator.generate(
+ new Object[] {
+ 105,
+ BinaryStringData.fromString("hammer"),
+ BinaryStringData.fromString("14oz carpenter's hammer"),
+ 0.875d
+ })));
+ snapshotExpected.add(
+ DataChangeEvent.insertEvent(
+ tableId,
+ generator.generate(
+ new Object[] {
+ 106,
+ BinaryStringData.fromString("hammer"),
+ BinaryStringData.fromString("16oz carpenter's hammer"),
+ 1.0d
+ })));
+ snapshotExpected.add(
+ DataChangeEvent.insertEvent(
+ tableId,
+ generator.generate(
+ new Object[] {
+ 107,
+ BinaryStringData.fromString("rocks"),
+ BinaryStringData.fromString("box of assorted rocks"),
+ 5.3d
+ })));
+ snapshotExpected.add(
+ DataChangeEvent.insertEvent(
+ tableId,
+ generator.generate(
+ new Object[] {
+ 108,
+ BinaryStringData.fromString("jacket"),
+ BinaryStringData.fromString(
+ "water resistant black wind breaker"),
+ 0.1d
+ })));
+ snapshotExpected.add(
+ DataChangeEvent.insertEvent(
+ tableId,
+ generator.generate(
+ new Object[] {
+ 109,
+ BinaryStringData.fromString("spare tire"),
+ BinaryStringData.fromString("24 inch spare tire"),
+ 22.2d
+ })));
+ return snapshotExpected;
+ }
+
+ private CreateTableEvent getProductsCreateTableEvent(TableId tableId) {
+ return new CreateTableEvent(
+ tableId,
+ Schema.newBuilder()
+ .physicalColumn("id", DataTypes.INT().notNull())
+ .physicalColumn("name", DataTypes.VARCHAR(255).notNull())
+ .physicalColumn("description", DataTypes.VARCHAR(512))
+ .physicalColumn("weight", DataTypes.FLOAT())
+ .primaryKey(Collections.singletonList("id"))
+ .build());
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerPipelineNewlyAddedTableITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerPipelineNewlyAddedTableITCase.java
new file mode 100644
index 00000000000..6ecd9fc2999
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerPipelineNewlyAddedTableITCase.java
@@ -0,0 +1,666 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.sqlserver.source;
+
+import org.apache.flink.api.common.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.cdc.common.data.binary.BinaryStringData;
+import org.apache.flink.cdc.common.event.CreateTableEvent;
+import org.apache.flink.cdc.common.event.DataChangeEvent;
+import org.apache.flink.cdc.common.event.Event;
+import org.apache.flink.cdc.common.event.SchemaChangeEvent;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.factories.Factory;
+import org.apache.flink.cdc.common.factories.FactoryHelper;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.common.source.FlinkSourceProvider;
+import org.apache.flink.cdc.common.types.DataType;
+import org.apache.flink.cdc.common.types.DataTypes;
+import org.apache.flink.cdc.common.types.RowType;
+import org.apache.flink.cdc.connectors.sqlserver.SqlServerTestBase;
+import org.apache.flink.cdc.connectors.sqlserver.factory.SqlServerDataSourceFactory;
+import org.apache.flink.cdc.connectors.sqlserver.testutils.SqlServerSourceTestUtils;
+import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
+import org.apache.flink.cdc.runtime.typeutils.EventTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.runtime.checkpoint.CheckpointException;
+import org.apache.flink.runtime.jobgraph.SavepointConfigOptions;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.operators.collect.AbstractCollectResultBuffer;
+import org.apache.flink.streaming.api.operators.collect.CheckpointedCollectResultBuffer;
+import org.apache.flink.streaming.api.operators.collect.CollectResultIterator;
+import org.apache.flink.streaming.api.operators.collect.CollectSinkOperator;
+import org.apache.flink.streaming.api.operators.collect.CollectSinkOperatorFactory;
+import org.apache.flink.streaming.api.operators.collect.CollectStreamSink;
+import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.HOSTNAME;
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.PASSWORD;
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.PORT;
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.SCAN_NEWLY_ADDED_TABLE_ENABLED;
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.SERVER_TIME_ZONE;
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.TABLES;
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.TABLES_EXCLUDE;
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.USERNAME;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.testcontainers.containers.MSSQLServerContainer.MS_SQL_SERVER_PORT;
+
+/** IT tests to cover various newly added tables during capture process in pipeline mode. */
+class SqlServerPipelineNewlyAddedTableITCase extends SqlServerTestBase {
+
+ private static final String DATABASE_NAME = "newly_added_table_test";
+ private static final String SCHEMA_NAME = "dbo";
+ private static final int DEFAULT_PARALLELISM = 4;
+
+ private final ScheduledExecutorService mockCdcExecutor = Executors.newScheduledThreadPool(1);
+
+ @BeforeEach
+ void before() throws SQLException {
+ TestValuesTableFactory.clearAllData();
+ initializeDatabase();
+ }
+
+ @AfterEach
+ void after() {
+ mockCdcExecutor.shutdown();
+ }
+
+ private void initializeDatabase() throws SQLException {
+ try (Connection connection = getJdbcConnection();
+ Statement statement = connection.createStatement()) {
+ // Drop database if exists
+ statement.execute(
+ String.format(
+ "IF EXISTS(select 1 from sys.databases where name = '%s') "
+ + "BEGIN ALTER DATABASE [%s] SET SINGLE_USER WITH ROLLBACK IMMEDIATE; "
+ + "DROP DATABASE [%s]; END",
+ DATABASE_NAME, DATABASE_NAME, DATABASE_NAME));
+
+ // Create database
+ statement.execute(String.format("CREATE DATABASE %s;", DATABASE_NAME));
+ statement.execute(String.format("USE %s;", DATABASE_NAME));
+
+ // Wait for SQL Server Agent
+ statement.execute("WAITFOR DELAY '00:00:03';");
+ statement.execute("EXEC sys.sp_cdc_enable_db;");
+
+ // Create produce_cdc_table for background CDC activity
+ statement.execute(
+ "CREATE TABLE dbo.produce_cdc_table (id BIGINT NOT NULL PRIMARY KEY, cnt BIGINT);");
+ statement.execute(
+ "INSERT INTO dbo.produce_cdc_table VALUES (0, 100), (1, 101), (2, 102);");
+ statement.execute(
+ "EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', "
+ + "@source_name = 'produce_cdc_table', @role_name = NULL, @supports_net_changes = 0;");
+
+ // Mock continuous CDC during the newly added table capturing process
+ mockCdcExecutor.schedule(
+ () -> {
+ try (Connection conn = getJdbcConnection();
+ Statement stmt = conn.createStatement()) {
+ stmt.execute(String.format("USE %s;", DATABASE_NAME));
+ stmt.execute(
+ "UPDATE dbo.produce_cdc_table SET cnt = cnt + 1 WHERE id < 2;");
+ } catch (SQLException e) {
+ e.printStackTrace();
+ }
+ },
+ 500,
+ TimeUnit.MILLISECONDS);
+ }
+ }
+
+ @Test
+ void testAddNewTableOneByOneSingleParallelism() throws Exception {
+ TestParam testParam =
+ TestParam.newBuilder(
+ Collections.singletonList("address_hangzhou"),
+ 4,
+ Arrays.asList("address_hangzhou", "address_beijing"),
+ 4)
+ .setFirstRoundInitTables(
+ Arrays.asList("address_hangzhou", "address_beijing"))
+ .build();
+
+ testAddNewTable(testParam, 1);
+ }
+
+ @Test
+ void testAddNewTableOneByOne() throws Exception {
+ TestParam testParam =
+ TestParam.newBuilder(
+ Collections.singletonList("address_hangzhou"),
+ 4,
+ Arrays.asList("address_hangzhou", "address_beijing"),
+ 4)
+ .setFirstRoundInitTables(
+ Arrays.asList("address_hangzhou", "address_beijing"))
+ .build();
+
+ testAddNewTable(testParam, DEFAULT_PARALLELISM);
+ }
+
+ @Test
+ void testAddNewTableByPatternSingleParallelism() throws Exception {
+ TestParam testParam =
+ TestParam.newBuilder(
+ Collections.singletonList("address_\\.*"),
+ 8,
+ Collections.singletonList("address_\\.*"),
+ 8)
+ .setFirstRoundInitTables(
+ Arrays.asList("address_hangzhou", "address_beijing"))
+ .setSecondRoundInitTables(
+ Arrays.asList("address_shanghai", "address_suzhou"))
+ .build();
+
+ testAddNewTable(testParam, 1);
+ }
+
+ @Test
+ void testAddNewTableByPattern() throws Exception {
+ TestParam testParam =
+ TestParam.newBuilder(
+ Collections.singletonList("address_\\.*"),
+ 8,
+ Collections.singletonList("address_\\.*"),
+ 12)
+ .setFirstRoundInitTables(
+ Arrays.asList("address_hangzhou", "address_beijing"))
+ .setSecondRoundInitTables(
+ Arrays.asList(
+ "address_shanghai", "address_suzhou", "address_shenzhen"))
+ .build();
+
+ testAddNewTable(testParam, DEFAULT_PARALLELISM);
+ }
+
+ @Test
+ void testAddNewTableWithExclude() throws Exception {
+ // Initialize all tables first
+ initialAddressTables(
+ Arrays.asList("address_hangzhou", "address_beijing", "address_excluded"));
+
+ Map options = new HashMap<>();
+ options.put(TABLES_EXCLUDE.key(), DATABASE_NAME + "." + SCHEMA_NAME + ".address_excluded");
+
+ FlinkSourceProvider sourceProvider =
+ getFlinkSourceProvider(
+ Collections.singletonList("address_\\.*"),
+ DEFAULT_PARALLELISM,
+ options,
+ true);
+ StreamExecutionEnvironment env =
+ StreamExecutionEnvironment.getExecutionEnvironment(new Configuration());
+ env.enableCheckpointing(200);
+ DataStreamSource source =
+ env.fromSource(
+ sourceProvider.getSource(),
+ WatermarkStrategy.noWatermarks(),
+ SqlServerDataSourceFactory.IDENTIFIER,
+ new EventTypeInfo());
+
+ TypeSerializer serializer =
+ source.getTransformation().getOutputType().createSerializer(env.getConfig());
+ CheckpointedCollectResultBuffer resultBuffer =
+ new CheckpointedCollectResultBuffer<>(serializer);
+ String accumulatorName = "dataStreamCollect_" + UUID.randomUUID();
+ CollectResultIterator iterator =
+ addCollector(env, source, resultBuffer, serializer, accumulatorName);
+ JobClient jobClient = env.executeAsync("TestExcludeTables");
+ iterator.setJobClient(jobClient);
+
+ // Expect 2 tables (hangzhou, beijing), each with CreateTableEvent + 3 DataChangeEvents = 8
+ // total
+ List actual = SqlServerSourceTestUtils.fetchResults(iterator, 8);
+
+ List tableNames =
+ actual.stream()
+ .filter(event -> event instanceof CreateTableEvent)
+ .map(event -> ((SchemaChangeEvent) event).tableId().getTableName())
+ .collect(Collectors.toList());
+
+ assertThat(tableNames).hasSize(2);
+ assertThat(tableNames).containsExactlyInAnyOrder("address_hangzhou", "address_beijing");
+ assertThat(tableNames).doesNotContain("address_excluded");
+
+ jobClient.cancel().get();
+ iterator.close();
+ }
+
+ private void testAddNewTable(TestParam testParam, int parallelism) throws Exception {
+ // step 1: create sqlserver tables
+ if (CollectionUtils.isNotEmpty(testParam.getFirstRoundInitTables())) {
+ initialAddressTables(testParam.getFirstRoundInitTables());
+ }
+ Path savepointDir = Files.createTempDirectory("add-new-table-test");
+ final String savepointDirectory = savepointDir.toAbsolutePath().toString();
+ String finishedSavePointPath = null;
+ StreamExecutionEnvironment env =
+ getStreamExecutionEnvironment(finishedSavePointPath, parallelism);
+
+ // step 2: listen tables first time
+ List listenTablesFirstRound = testParam.getFirstRoundListenTables();
+
+ FlinkSourceProvider sourceProvider =
+ getFlinkSourceProvider(listenTablesFirstRound, parallelism, new HashMap<>(), true);
+ DataStreamSource source =
+ env.fromSource(
+ sourceProvider.getSource(),
+ WatermarkStrategy.noWatermarks(),
+ SqlServerDataSourceFactory.IDENTIFIER,
+ new EventTypeInfo());
+
+ TypeSerializer serializer =
+ source.getTransformation().getOutputType().createSerializer(env.getConfig());
+ CheckpointedCollectResultBuffer resultBuffer =
+ new CheckpointedCollectResultBuffer<>(serializer);
+ String accumulatorName = "dataStreamCollect_" + UUID.randomUUID();
+ CollectResultIterator iterator =
+ addCollector(env, source, resultBuffer, serializer, accumulatorName);
+ JobClient jobClient = env.executeAsync("beforeAddNewTable");
+ iterator.setJobClient(jobClient);
+
+ List actual =
+ SqlServerSourceTestUtils.fetchResults(iterator, testParam.getFirstRoundFetchSize());
+ Optional listenByPattern =
+ listenTablesFirstRound.stream()
+ .filter(table -> StringUtils.contains(table, "\\.*"))
+ .findAny();
+ multiAssert(
+ actual,
+ listenByPattern.isPresent()
+ ? testParam.getFirstRoundInitTables()
+ : listenTablesFirstRound);
+
+ // step 3: create new tables if needed
+ if (CollectionUtils.isNotEmpty(testParam.getSecondRoundInitTables())) {
+ initialAddressTables(testParam.getSecondRoundInitTables());
+ }
+
+ // step 4: trigger a savepoint and cancel the job
+ finishedSavePointPath = triggerSavepointWithRetry(jobClient, savepointDirectory);
+ jobClient.cancel().get();
+ iterator.close();
+
+ // step 5: restore from savepoint
+ StreamExecutionEnvironment restoredEnv =
+ getStreamExecutionEnvironment(finishedSavePointPath, parallelism);
+ List listenTablesSecondRound = testParam.getSecondRoundListenTables();
+ FlinkSourceProvider restoredSourceProvider =
+ getFlinkSourceProvider(listenTablesSecondRound, parallelism, new HashMap<>(), true);
+ DataStreamSource restoreSource =
+ restoredEnv.fromSource(
+ restoredSourceProvider.getSource(),
+ WatermarkStrategy.noWatermarks(),
+ SqlServerDataSourceFactory.IDENTIFIER,
+ new EventTypeInfo());
+ CollectResultIterator restoredIterator =
+ addCollector(restoredEnv, restoreSource, resultBuffer, serializer, accumulatorName);
+ JobClient restoreClient = restoredEnv.executeAsync("AfterAddNewTable");
+
+ List newlyAddTables =
+ listenTablesSecondRound.stream()
+ .filter(table -> !listenTablesFirstRound.contains(table))
+ .collect(Collectors.toList());
+ // it means listen by pattern when newlyAddTables is empty
+ if (CollectionUtils.isEmpty(newlyAddTables)) {
+ newlyAddTables = testParam.getSecondRoundInitTables();
+ }
+ List newlyTableEvent =
+ SqlServerSourceTestUtils.fetchResults(
+ restoredIterator, testParam.getSecondRoundFetchSize());
+ multiAssert(newlyTableEvent, newlyAddTables);
+ restoreClient.cancel().get();
+ restoredIterator.close();
+ }
+
+ private void multiAssert(List actualEvents, List listenTables) {
+ List expectedCreateTableEvents = new ArrayList<>();
+ List expectedDataChangeEvents = new ArrayList<>();
+ for (String table : listenTables) {
+ expectedCreateTableEvents.add(
+ getCreateTableEvent(TableId.tableId(DATABASE_NAME, SCHEMA_NAME, table)));
+ expectedDataChangeEvents.addAll(
+ getSnapshotExpected(TableId.tableId(DATABASE_NAME, SCHEMA_NAME, table)));
+ }
+ // compare create table events
+ List actualCreateTableEvents =
+ actualEvents.stream()
+ .filter(event -> event instanceof CreateTableEvent)
+ .collect(Collectors.toList());
+ assertThat(actualCreateTableEvents)
+ .containsExactlyInAnyOrder(expectedCreateTableEvents.toArray(new Event[0]));
+
+ // compare data change events
+ List actualDataChangeEvents =
+ actualEvents.stream()
+ .filter(event -> event instanceof DataChangeEvent)
+ .collect(Collectors.toList());
+ assertThat(actualDataChangeEvents)
+ .containsExactlyInAnyOrder(expectedDataChangeEvents.toArray(new Event[0]));
+ }
+
+ private CreateTableEvent getCreateTableEvent(TableId tableId) {
+ Schema schema =
+ Schema.newBuilder()
+ .physicalColumn("id", DataTypes.BIGINT().notNull())
+ .physicalColumn("country", DataTypes.VARCHAR(255).notNull())
+ .physicalColumn("city", DataTypes.VARCHAR(255).notNull())
+ .physicalColumn("detail_address", DataTypes.VARCHAR(1024))
+ .primaryKey(Collections.singletonList("id"))
+ .build();
+ return new CreateTableEvent(tableId, schema);
+ }
+
+ private List getSnapshotExpected(TableId tableId) {
+ RowType rowType =
+ RowType.of(
+ new DataType[] {
+ DataTypes.BIGINT().notNull(),
+ DataTypes.VARCHAR(255).notNull(),
+ DataTypes.VARCHAR(255).notNull(),
+ DataTypes.VARCHAR(1024)
+ },
+ new String[] {"id", "country", "city", "detail_address"});
+ BinaryRecordDataGenerator generator = new BinaryRecordDataGenerator(rowType);
+ String cityName = tableId.getTableName().split("_")[1];
+ return Arrays.asList(
+ DataChangeEvent.insertEvent(
+ tableId,
+ generator.generate(
+ new Object[] {
+ 416874195632735147L,
+ BinaryStringData.fromString("China"),
+ BinaryStringData.fromString(cityName),
+ BinaryStringData.fromString(cityName + " West Town address 1")
+ })),
+ DataChangeEvent.insertEvent(
+ tableId,
+ generator.generate(
+ new Object[] {
+ 416927583791428523L,
+ BinaryStringData.fromString("China"),
+ BinaryStringData.fromString(cityName),
+ BinaryStringData.fromString(cityName + " West Town address 2")
+ })),
+ DataChangeEvent.insertEvent(
+ tableId,
+ generator.generate(
+ new Object[] {
+ 417022095255614379L,
+ BinaryStringData.fromString("China"),
+ BinaryStringData.fromString(cityName),
+ BinaryStringData.fromString(cityName + " West Town address 3")
+ })));
+ }
+
+ private String triggerSavepointWithRetry(JobClient jobClient, String savepointDirectory)
+ throws ExecutionException, InterruptedException {
+ int retryTimes = 0;
+ // retry 600 times, it takes 100 milliseconds per time, at most retry 1 minute
+ while (retryTimes < 600) {
+ try {
+ return jobClient.triggerSavepoint(savepointDirectory).get();
+ } catch (Exception e) {
+ Optional exception =
+ ExceptionUtils.findThrowable(e, CheckpointException.class);
+ if (exception.isPresent()
+ && exception.get().getMessage().contains("Checkpoint triggering task")) {
+ Thread.sleep(100);
+ retryTimes++;
+ } else {
+ throw e;
+ }
+ }
+ }
+ throw new AssertionError(
+ String.format(
+ "Failed to trigger savepoint in directory '%s' after %d retries.",
+ savepointDirectory, retryTimes));
+ }
+
+ private void initialAddressTables(List addressTables) throws SQLException {
+ try (Connection connection = getJdbcConnection();
+ Statement statement = connection.createStatement()) {
+ statement.execute(String.format("USE %s;", DATABASE_NAME));
+ for (String tableName : addressTables) {
+ String cityName = tableName.split("_")[1];
+ // Create table
+ statement.execute(
+ String.format(
+ "IF NOT EXISTS (SELECT * FROM sys.tables WHERE name = '%s') "
+ + "CREATE TABLE %s.%s ("
+ + " id BIGINT NOT NULL PRIMARY KEY,"
+ + " country VARCHAR(255) NOT NULL,"
+ + " city VARCHAR(255) NOT NULL,"
+ + " detail_address VARCHAR(1024)"
+ + ");",
+ tableName, SCHEMA_NAME, tableName));
+
+ // Insert data
+ statement.execute(
+ String.format(
+ "INSERT INTO %s.%s "
+ + "VALUES (416874195632735147, 'China', '%s', '%s West Town address 1'),"
+ + " (416927583791428523, 'China', '%s', '%s West Town address 2'),"
+ + " (417022095255614379, 'China', '%s', '%s West Town address 3');",
+ SCHEMA_NAME,
+ tableName,
+ cityName,
+ cityName,
+ cityName,
+ cityName,
+ cityName,
+ cityName));
+
+ // Enable CDC for the table
+ statement.execute(
+ String.format(
+ "EXEC sys.sp_cdc_enable_table @source_schema = '%s', "
+ + "@source_name = '%s', @role_name = NULL, @supports_net_changes = 0;",
+ SCHEMA_NAME, tableName));
+ }
+ }
+ }
+
+ private FlinkSourceProvider getFlinkSourceProvider(
+ List tables,
+ int parallelism,
+ Map additionalOptions,
+ boolean enableScanNewlyAddedTable) {
+ List fullTableNames =
+ tables.stream()
+ .map(table -> DATABASE_NAME + "." + SCHEMA_NAME + "." + table)
+ .collect(Collectors.toList());
+ Map options = new HashMap<>();
+ options.put(HOSTNAME.key(), MSSQL_SERVER_CONTAINER.getHost());
+ options.put(
+ PORT.key(),
+ String.valueOf(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT)));
+ options.put(USERNAME.key(), MSSQL_SERVER_CONTAINER.getUsername());
+ options.put(PASSWORD.key(), MSSQL_SERVER_CONTAINER.getPassword());
+ options.put(SERVER_TIME_ZONE.key(), "UTC");
+ options.put(TABLES.key(), StringUtils.join(fullTableNames, ","));
+ if (enableScanNewlyAddedTable) {
+ options.put(SCAN_NEWLY_ADDED_TABLE_ENABLED.key(), "true");
+ }
+ options.putAll(additionalOptions);
+ Factory.Context context =
+ new FactoryHelper.DefaultContext(
+ org.apache.flink.cdc.common.configuration.Configuration.fromMap(options),
+ null,
+ this.getClass().getClassLoader());
+
+ SqlServerDataSourceFactory factory = new SqlServerDataSourceFactory();
+ SqlServerDataSource dataSource = (SqlServerDataSource) factory.createDataSource(context);
+
+ return (FlinkSourceProvider) dataSource.getEventSourceProvider();
+ }
+
+ private CollectResultIterator addCollector(
+ StreamExecutionEnvironment env,
+ DataStreamSource source,
+ AbstractCollectResultBuffer buffer,
+ TypeSerializer serializer,
+ String accumulatorName) {
+ CollectSinkOperatorFactory sinkFactory =
+ new CollectSinkOperatorFactory<>(serializer, accumulatorName);
+ CollectSinkOperator operator = (CollectSinkOperator) sinkFactory.getOperator();
+ CollectResultIterator iterator =
+ new CollectResultIterator<>(
+ buffer, operator.getOperatorIdFuture(), accumulatorName, 0);
+ CollectStreamSink sink = new CollectStreamSink<>(source, sinkFactory);
+ sink.name("Data stream collect sink");
+ env.addOperator(sink.getTransformation());
+ env.registerCollectIterator(iterator);
+ return iterator;
+ }
+
+ private StreamExecutionEnvironment getStreamExecutionEnvironment(
+ String finishedSavePointPath, int parallelism) {
+ Configuration configuration = new Configuration();
+ if (finishedSavePointPath != null) {
+ configuration.setString(SavepointConfigOptions.SAVEPOINT_PATH, finishedSavePointPath);
+ }
+ StreamExecutionEnvironment env =
+ StreamExecutionEnvironment.getExecutionEnvironment(configuration);
+ env.setParallelism(parallelism);
+ env.enableCheckpointing(500L);
+ env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 1000L));
+ return env;
+ }
+
+ private static class TestParam {
+ private final List firstRoundInitTables;
+ private final List firstRoundListenTables;
+ private final Integer firstRoundFetchSize;
+ private final List secondRoundInitTables;
+ private final List secondRoundListenTables;
+ private final Integer secondRoundFetchSize;
+
+ private TestParam(Builder builder) {
+ this.firstRoundInitTables = builder.firstRoundInitTables;
+ this.firstRoundListenTables = builder.firstRoundListenTables;
+ this.firstRoundFetchSize = builder.firstRoundFetchSize;
+ this.secondRoundInitTables = builder.secondRoundInitTables;
+ this.secondRoundListenTables = builder.secondRoundListenTables;
+ this.secondRoundFetchSize = builder.secondRoundFetchSize;
+ }
+
+ public static Builder newBuilder(
+ List firstRoundListenTables,
+ Integer firstRoundFetchSize,
+ List secondRoundListenTables,
+ Integer secondRoundFetchSize) {
+ return new Builder(
+ firstRoundListenTables,
+ firstRoundFetchSize,
+ secondRoundListenTables,
+ secondRoundFetchSize);
+ }
+
+ public static class Builder {
+ private List firstRoundInitTables;
+ private final List firstRoundListenTables;
+ private final Integer firstRoundFetchSize;
+
+ private List secondRoundInitTables;
+ private final List secondRoundListenTables;
+ private final Integer secondRoundFetchSize;
+
+ public Builder(
+ List firstRoundListenTables,
+ Integer firstRoundFetchSize,
+ List secondRoundListenTables,
+ Integer secondRoundFetchSize) {
+ this.firstRoundListenTables = firstRoundListenTables;
+ this.firstRoundFetchSize = firstRoundFetchSize;
+ this.secondRoundListenTables = secondRoundListenTables;
+ this.secondRoundFetchSize = secondRoundFetchSize;
+ }
+
+ public TestParam build() {
+ return new TestParam(this);
+ }
+
+ public Builder setFirstRoundInitTables(List firstRoundInitTables) {
+ this.firstRoundInitTables = firstRoundInitTables;
+ return this;
+ }
+
+ public Builder setSecondRoundInitTables(List secondRoundInitTables) {
+ this.secondRoundInitTables = secondRoundInitTables;
+ return this;
+ }
+ }
+
+ public List getFirstRoundInitTables() {
+ return firstRoundInitTables;
+ }
+
+ public List getFirstRoundListenTables() {
+ return firstRoundListenTables;
+ }
+
+ public Integer getFirstRoundFetchSize() {
+ return firstRoundFetchSize;
+ }
+
+ public List getSecondRoundInitTables() {
+ return secondRoundInitTables;
+ }
+
+ public List getSecondRoundListenTables() {
+ return secondRoundListenTables;
+ }
+
+ public Integer getSecondRoundFetchSize() {
+ return secondRoundFetchSize;
+ }
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerTablePatternMatchingTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerTablePatternMatchingTest.java
new file mode 100644
index 00000000000..66c101e6969
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerTablePatternMatchingTest.java
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.sqlserver.source;
+
+import org.apache.flink.cdc.common.configuration.Configuration;
+import org.apache.flink.cdc.common.factories.Factory;
+import org.apache.flink.cdc.connectors.sqlserver.SqlServerTestBase;
+import org.apache.flink.cdc.connectors.sqlserver.factory.SqlServerDataSourceFactory;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import javax.annotation.Nullable;
+
+import java.sql.Connection;
+import java.sql.Statement;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.HOSTNAME;
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.PASSWORD;
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.PORT;
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.TABLES;
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.TABLES_EXCLUDE;
+import static org.apache.flink.cdc.connectors.sqlserver.source.SqlServerDataSourceOptions.USERNAME;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.testcontainers.containers.MSSQLServerContainer.MS_SQL_SERVER_PORT;
+
+/** Test cases for matching SQL Server source tables. */
+public class SqlServerTablePatternMatchingTest extends SqlServerTestBase {
+
+ private static final String DATABASE_NAME = "pattern_test";
+
+ @BeforeEach
+ public void before() {
+ initializePatternTestDatabase();
+ }
+
+ private void initializePatternTestDatabase() {
+ try (Connection connection = getJdbcConnection();
+ Statement statement = connection.createStatement()) {
+ dropTestDatabase(connection, DATABASE_NAME);
+
+ // Create database
+ statement.execute(String.format("CREATE DATABASE [%s];", DATABASE_NAME));
+ statement.execute(String.format("USE [%s];", DATABASE_NAME));
+
+ // Wait for SQL Server Agent
+ statement.execute("WAITFOR DELAY '00:00:03';");
+ statement.execute("EXEC sys.sp_cdc_enable_db;");
+
+ // Create test tables
+ String[] tableNames = {"tbl1", "tbl2", "tbl3", "tbl_special"};
+ for (String tableName : tableNames) {
+ statement.execute(
+ String.format(
+ "CREATE TABLE dbo.%s (id INT NOT NULL PRIMARY KEY);", tableName));
+ statement.execute(String.format("INSERT INTO dbo.%s VALUES (1);", tableName));
+ statement.execute(
+ String.format(
+ "EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', "
+ + "@source_name = '%s', @role_name = NULL, @supports_net_changes = 0;",
+ tableName));
+ }
+ } catch (Exception e) {
+ throw new RuntimeException("Failed to initialize pattern test database", e);
+ }
+ }
+
+ @Test
+ public void testWildcardMatchingAllTables() {
+ List tables = testTableMatching(DATABASE_NAME + ".dbo.\\.*", null);
+ assertThat(tables)
+ .containsExactlyInAnyOrder("dbo.tbl1", "dbo.tbl2", "dbo.tbl3", "dbo.tbl_special");
+ }
+
+ @Test
+ public void testWildcardMatchingPartialTables() {
+ List tables = testTableMatching(DATABASE_NAME + ".dbo.tbl[1-2]", null);
+ assertThat(tables).containsExactlyInAnyOrder("dbo.tbl1", "dbo.tbl2");
+ }
+
+ @Test
+ public void testExactTableMatching() {
+ List tables = testTableMatching(DATABASE_NAME + ".dbo.tbl1", null);
+ assertThat(tables).containsExactly("dbo.tbl1");
+ }
+
+ @Test
+ public void testMultipleTableMatching() {
+ List tables =
+ testTableMatching(DATABASE_NAME + ".dbo.tbl1," + DATABASE_NAME + ".dbo.tbl3", null);
+ assertThat(tables).containsExactlyInAnyOrder("dbo.tbl1", "dbo.tbl3");
+ }
+
+ @Test
+ public void testWildcardMatchingWithExclusion() {
+ List tables =
+ testTableMatching(DATABASE_NAME + ".dbo.\\.*", DATABASE_NAME + ".dbo.tbl1");
+ assertThat(tables).containsExactlyInAnyOrder("dbo.tbl2", "dbo.tbl3", "dbo.tbl_special");
+ }
+
+ @Test
+ public void testWildcardMatchingWithPatternExclusion() {
+ List tables =
+ testTableMatching(DATABASE_NAME + ".dbo.\\.*", DATABASE_NAME + ".dbo.tbl[1-2]");
+ assertThat(tables).containsExactlyInAnyOrder("dbo.tbl3", "dbo.tbl_special");
+ }
+
+ @Test
+ public void testWildcardMatchingWithMultipleExclusions() {
+ List tables =
+ testTableMatching(
+ DATABASE_NAME + ".dbo.\\.*",
+ DATABASE_NAME + ".dbo.tbl1," + DATABASE_NAME + ".dbo.tbl_special");
+ assertThat(tables).containsExactlyInAnyOrder("dbo.tbl2", "dbo.tbl3");
+ }
+
+ @Test
+ public void testPatternMatchingWithUnderscore() {
+ List tables = testTableMatching(DATABASE_NAME + ".dbo.tbl_\\.*", null);
+ assertThat(tables).containsExactly("dbo.tbl_special");
+ }
+
+ @Test
+ public void testMatchingWithSpacedRules() {
+ // Test with spaces around commas
+ List tables =
+ testTableMatching(
+ DATABASE_NAME + ".dbo.tbl1 , " + DATABASE_NAME + ".dbo.tbl2", null);
+ assertThat(tables).containsExactlyInAnyOrder("dbo.tbl1", "dbo.tbl2");
+ }
+
+ private List testTableMatching(String tablesConfig, @Nullable String tablesExclude) {
+ Map options = new HashMap<>();
+ options.put(HOSTNAME.key(), MSSQL_SERVER_CONTAINER.getHost());
+ options.put(
+ PORT.key(),
+ String.valueOf(MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT)));
+ options.put(USERNAME.key(), MSSQL_SERVER_CONTAINER.getUsername());
+ options.put(PASSWORD.key(), MSSQL_SERVER_CONTAINER.getPassword());
+ options.put(TABLES.key(), tablesConfig);
+ if (tablesExclude != null) {
+ options.put(TABLES_EXCLUDE.key(), tablesExclude);
+ }
+
+ Factory.Context context = new MockContext(Configuration.fromMap(options));
+ SqlServerDataSourceFactory factory = new SqlServerDataSourceFactory();
+ SqlServerDataSource dataSource = (SqlServerDataSource) factory.createDataSource(context);
+ return dataSource.getSqlServerSourceConfig().getTableList();
+ }
+
+ static class MockContext implements Factory.Context {
+
+ Configuration factoryConfiguration;
+
+ public MockContext(Configuration factoryConfiguration) {
+ this.factoryConfiguration = factoryConfiguration;
+ }
+
+ @Override
+ public Configuration getFactoryConfiguration() {
+ return factoryConfiguration;
+ }
+
+ @Override
+ public Configuration getPipelineConfiguration() {
+ return null;
+ }
+
+ @Override
+ public ClassLoader getClassLoader() {
+ return this.getClass().getClassLoader();
+ }
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/testutils/RecordDataTestUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/testutils/RecordDataTestUtils.java
new file mode 100644
index 00000000000..7a3885fbe85
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/testutils/RecordDataTestUtils.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.sqlserver.testutils;
+
+import org.apache.flink.cdc.common.data.RecordData;
+import org.apache.flink.cdc.common.types.DataType;
+import org.apache.flink.cdc.common.types.RowType;
+
+import java.util.List;
+
+/** Utility for {@link RecordData}. */
+public class RecordDataTestUtils {
+
+ public static Object[] recordFields(RecordData record, RowType rowType) {
+ int fieldNum = record.getArity();
+ List fieldTypes = rowType.getChildren();
+ Object[] fields = new Object[fieldNum];
+ for (int i = 0; i < fieldNum; i++) {
+ if (record.isNullAt(i)) {
+ fields[i] = null;
+ } else {
+ DataType type = fieldTypes.get(i);
+ try {
+ RecordData.FieldGetter fieldGetter = RecordData.createFieldGetter(type, i);
+ Object o = fieldGetter.getFieldOrNull(record);
+ fields[i] = o;
+ } catch (Exception e) {
+ throw new RuntimeException(
+ String.format(
+ "Failed to get field at index %d, type: %s. Field names: %s",
+ i, type, rowType.getFieldNames()),
+ e);
+ }
+ }
+ }
+ return fields;
+ }
+
+ private RecordDataTestUtils() {}
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/testutils/SqlServerSourceTestUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/testutils/SqlServerSourceTestUtils.java
new file mode 100644
index 00000000000..9642e9d134c
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/testutils/SqlServerSourceTestUtils.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.sqlserver.testutils;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.cdc.common.event.CreateTableEvent;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+
+/** Test utilities for SQL Server event source. */
+public class SqlServerSourceTestUtils {
+
+ public static List fetchResults(Iterator iter, int size) {
+ List result = new ArrayList<>(size);
+ while (size > 0 && iter.hasNext()) {
+ T event = iter.next();
+ result.add(event);
+ size--;
+ }
+ return result;
+ }
+
+ public static Tuple2, List> fetchResultsAndCreateTableEvent(
+ Iterator iter, int size) {
+ List result = new ArrayList<>(size);
+ List createTableEvents = new ArrayList<>();
+ while (size > 0 && iter.hasNext()) {
+ T event = iter.next();
+ if (event instanceof CreateTableEvent) {
+ createTableEvents.add((CreateTableEvent) event);
+ } else {
+ result.add(event);
+ size--;
+ }
+ }
+ return Tuple2.of(result, createTableEvents);
+ }
+
+ public static void loopCheck(
+ Supplier runnable, String description, Duration timeout, Duration interval)
+ throws Exception {
+ long deadline = System.currentTimeMillis() + timeout.toMillis();
+ while (System.currentTimeMillis() < deadline) {
+ if (runnable.get()) {
+ return;
+ }
+ TimeUnit.MILLISECONDS.sleep(interval.toMillis());
+ }
+ throw new TimeoutException(
+ "Ran out of time when waiting for " + description + " to succeed.");
+ }
+
+ private SqlServerSourceTestUtils() {}
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/utils/SqlServerSchemaUtilsTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/utils/SqlServerSchemaUtilsTest.java
new file mode 100644
index 00000000000..0dbdd64b6fb
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/utils/SqlServerSchemaUtilsTest.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.sqlserver.utils;
+
+import org.junit.jupiter.api.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+class SqlServerSchemaUtilsTest {
+
+ @Test
+ void testQuoteEscapesClosingBracket() {
+ assertThat(SqlServerSchemaUtils.quote("db]name")).isEqualTo("[db]]name]");
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/resources/ddl/column_type_test.sql b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/resources/ddl/column_type_test.sql
new file mode 100644
index 00000000000..b12f1065dfb
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/resources/ddl/column_type_test.sql
@@ -0,0 +1,136 @@
+ -- Licensed to the Apache Software Foundation (ASF) under one or more
+-- contributor license agreements. See the NOTICE file distributed with
+-- this work for additional information regarding copyright ownership.
+-- The ASF licenses this file to You under the Apache License, Version 2.0
+-- (the "License"); you may not use this file except in compliance with
+-- the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+-- ----------------------------------------------------------------------------------------------------------------
+-- DATABASE: column_type_test
+-- ----------------------------------------------------------------------------------------------------------------
+-- Create the column_type_test database
+CREATE DATABASE column_type_test;
+
+USE column_type_test;
+
+-- Avoid SqlServer error com.microsoft.sqlserver.jdbc.SQLServerException: Could not update the metadata that indicates,
+-- the root cause is 14258: 'Cannot perform this operation while SQLServerAgent is starting. Try again later.'. We simply
+-- wait for 3 seconds to improve the test stabilises.
+WAITFOR DELAY '00:00:03';
+EXEC sys.sp_cdc_enable_db;
+
+-- Create full_types table for testing all supported data types
+CREATE TABLE full_types (
+ id int NOT NULL PRIMARY KEY,
+ -- Character types
+ val_char char(3),
+ val_varchar varchar(1000),
+ val_text text,
+ val_nchar nchar(3),
+ val_nvarchar nvarchar(1000),
+ val_ntext ntext,
+ -- Numeric types
+ val_decimal decimal(6,3),
+ val_numeric numeric(10,2),
+ val_float float,
+ val_real real,
+ val_smallmoney smallmoney,
+ val_money money,
+ -- Boolean and integer types
+ val_bit bit,
+ val_tinyint tinyint,
+ val_smallint smallint,
+ val_int int,
+ val_bigint bigint,
+ -- Date and time types
+ val_date date,
+ val_time_p2 time(2),
+ val_time time(4),
+ val_datetime2 datetime2,
+ val_datetimeoffset datetimeoffset,
+ val_datetime datetime,
+ val_smalldatetime smalldatetime,
+ -- Other types
+ val_xml xml,
+ val_binary binary(8),
+ val_varbinary varbinary(100),
+ val_uniqueidentifier uniqueidentifier
+);
+
+INSERT INTO full_types VALUES (
+ 1,
+ -- Character types
+ 'abc', 'varchar value', 'text value', N'中文', N'nvarchar value', N'ntext value',
+ -- Numeric types
+ 123.456, 9876543.21, 3.14159265358979, 2.71828, 214748.3647, 5477.5807,
+ -- Boolean and integer types
+ 1, 255, 32767, 2147483647, 9223372036854775807,
+ -- Date and time types
+ '2020-07-17', '18:00:22.12', '18:00:22.1234', '2020-07-17 18:00:22.123456',
+ '2020-07-17 18:00:22.1234567 +00:00', '2020-07-17 18:00:22.123', '2020-07-17 18:00:00',
+ -- Other types
+ 'value', 0x0102030405060708, 0x48656C6C6F, 'a0eebc99-9c0b-4ef8-bb6d-6bb9bd380a11'
+);
+
+EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'full_types', @role_name = NULL, @supports_net_changes = 0;
+
+-- Create time_types table for testing time-related types
+CREATE TABLE time_types (
+ id int NOT NULL PRIMARY KEY,
+ val_date date,
+ val_time_0 time(0),
+ val_time_3 time(3),
+ val_time_6 time(6),
+ val_datetime2_0 datetime2(0),
+ val_datetime2_3 datetime2(3),
+ val_datetime2_6 datetime2(6),
+ val_datetimeoffset_0 datetimeoffset(0),
+ val_datetimeoffset_3 datetimeoffset(3),
+ val_datetimeoffset_6 datetimeoffset(6),
+ val_datetime datetime,
+ val_smalldatetime smalldatetime
+);
+
+INSERT INTO time_types VALUES (
+ 1,
+ '2020-07-17',
+ '18:00:22', '18:00:22.123', '18:00:22.123456',
+ '2020-07-17 18:00:22', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456',
+ '2020-07-17 18:00:22 +00:00', '2020-07-17 18:00:22.123 +00:00', '2020-07-17 18:00:22.123456 +00:00',
+ '2020-07-17 18:00:22.123', '2020-07-17 18:00:00'
+);
+
+EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'time_types', @role_name = NULL, @supports_net_changes = 0;
+
+-- Create precision_types table for testing decimal precision
+CREATE TABLE precision_types (
+ id int NOT NULL PRIMARY KEY,
+ val_decimal_6_2 decimal(6,2),
+ val_decimal_10_4 decimal(10,4),
+ val_decimal_20_6 decimal(20,6),
+ val_decimal_38_10 decimal(38,10),
+ val_numeric_6_2 numeric(6,2),
+ val_numeric_10_4 numeric(10,4),
+ val_float float,
+ val_real real,
+ val_money money,
+ val_smallmoney smallmoney
+);
+
+INSERT INTO precision_types VALUES (
+ 1,
+ 1234.56, 123456.7890, 12345678901234.567890, 1234567890123456789012345678.9012345678,
+ 1234.56, 123456.7890,
+ 3.141592653589793, 2.7182818,
+ 922337203685477.5807, 214748.3647
+);
+
+EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'precision_types', @role_name = NULL, @supports_net_changes = 0;
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/resources/ddl/inventory.sql b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/resources/ddl/inventory.sql
new file mode 100644
index 00000000000..9f4b39c09d2
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/resources/ddl/inventory.sql
@@ -0,0 +1,106 @@
+-- Licensed to the Apache Software Foundation (ASF) under one or more
+-- contributor license agreements. See the NOTICE file distributed with
+-- this work for additional information regarding copyright ownership.
+-- The ASF licenses this file to You under the Apache License, Version 2.0
+-- (the "License"); you may not use this file except in compliance with
+-- the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+-- ----------------------------------------------------------------------------------------------------------------
+-- DATABASE: inventory
+-- ----------------------------------------------------------------------------------------------------------------
+-- Create the inventory database
+CREATE DATABASE inventory;
+
+USE inventory;
+
+-- Avoid SqlServer error com.microsoft.sqlserver.jdbc.SQLServerException: Could not update the metadata that indicates,
+-- the root cause is 14258: 'Cannot perform this operation while SQLServerAgent is starting. Try again later.'. We simply
+-- wait for 3 seconds to improve the test stabilises.
+WAITFOR DELAY '00:00:03';
+EXEC sys.sp_cdc_enable_db;
+
+-- Create and populate our products using a single insert with many rows
+CREATE TABLE products (
+ id INTEGER IDENTITY(101,1) NOT NULL PRIMARY KEY,
+ name VARCHAR(255) NOT NULL,
+ description VARCHAR(512),
+ weight FLOAT
+);
+INSERT INTO products(name,description,weight)
+VALUES ('scooter','Small 2-wheel scooter',3.14);
+INSERT INTO products(name,description,weight)
+VALUES ('car battery','12V car battery',8.1);
+INSERT INTO products(name,description,weight)
+VALUES ('12-pack drill bits','12-pack of drill bits with sizes ranging from #40 to #3',0.8);
+INSERT INTO products(name,description,weight)
+VALUES ('hammer','12oz carpenter''s hammer',0.75);
+INSERT INTO products(name,description,weight)
+VALUES ('hammer','14oz carpenter''s hammer',0.875);
+INSERT INTO products(name,description,weight)
+VALUES ('hammer','16oz carpenter''s hammer',1.0);
+INSERT INTO products(name,description,weight)
+VALUES ('rocks','box of assorted rocks',5.3);
+INSERT INTO products(name,description,weight)
+VALUES ('jacket','water resistant black wind breaker',0.1);
+INSERT INTO products(name,description,weight)
+VALUES ('spare tire','24 inch spare tire',22.2);
+EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'products', @role_name = NULL, @supports_net_changes = 0;
+-- Create and populate the products on hand using multiple inserts
+CREATE TABLE products_on_hand (
+ product_id INTEGER NOT NULL PRIMARY KEY,
+ quantity INTEGER NOT NULL,
+ FOREIGN KEY (product_id) REFERENCES products(id)
+);
+INSERT INTO products_on_hand VALUES (101,3);
+INSERT INTO products_on_hand VALUES (102,8);
+INSERT INTO products_on_hand VALUES (103,18);
+INSERT INTO products_on_hand VALUES (104,4);
+INSERT INTO products_on_hand VALUES (105,5);
+INSERT INTO products_on_hand VALUES (106,0);
+INSERT INTO products_on_hand VALUES (107,44);
+INSERT INTO products_on_hand VALUES (108,2);
+INSERT INTO products_on_hand VALUES (109,5);
+EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'products_on_hand', @role_name = NULL, @supports_net_changes = 0;
+-- Create some customers ...
+CREATE TABLE customers (
+ id INTEGER IDENTITY(1001,1) NOT NULL PRIMARY KEY,
+ first_name VARCHAR(255) NOT NULL,
+ last_name VARCHAR(255) NOT NULL,
+ email VARCHAR(255) NOT NULL UNIQUE
+);
+INSERT INTO customers(first_name,last_name,email)
+VALUES ('Sally','Thomas','sally.thomas@acme.com');
+INSERT INTO customers(first_name,last_name,email)
+VALUES ('George','Bailey','gbailey@foobar.com');
+INSERT INTO customers(first_name,last_name,email)
+VALUES ('Edward','Walker','ed@walker.com');
+INSERT INTO customers(first_name,last_name,email)
+VALUES ('Anne','Kretchmar','annek@noanswer.org');
+EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'customers', @role_name = NULL, @supports_net_changes = 0;
+-- Create some very simple orders
+CREATE TABLE orders (
+ id INTEGER IDENTITY(10001,1) NOT NULL PRIMARY KEY,
+ order_date DATE NOT NULL,
+ purchaser INTEGER NOT NULL,
+ quantity INTEGER NOT NULL,
+ product_id INTEGER NOT NULL,
+ FOREIGN KEY (purchaser) REFERENCES customers(id),
+ FOREIGN KEY (product_id) REFERENCES products(id)
+);
+INSERT INTO orders(order_date,purchaser,quantity,product_id)
+VALUES ('16-JAN-2016', 1001, 1, 102);
+INSERT INTO orders(order_date,purchaser,quantity,product_id)
+VALUES ('17-JAN-2016', 1002, 2, 105);
+INSERT INTO orders(order_date,purchaser,quantity,product_id)
+VALUES ('19-FEB-2016', 1002, 2, 106);
+INSERT INTO orders(order_date,purchaser,quantity,product_id)
+VALUES ('21-FEB-2016', 1003, 1, 107);
+EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'orders', @role_name = NULL, @supports_net_changes = 0;
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/resources/log4j2-test.properties b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/resources/log4j2-test.properties
new file mode 100644
index 00000000000..931478c6c76
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/resources/log4j2-test.properties
@@ -0,0 +1,34 @@
+################################################################################
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+# http://www.apache.org/licenses/LICENSE-2.0
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+################################################################################
+
+# Set root logger level to INFO to not flood build logs
+# set manually to DEBUG for debugging purposes
+rootLogger.level=INFO
+rootLogger.appenderRef.test.ref = TestLogger
+
+appender.testlogger.name = TestLogger
+appender.testlogger.type = CONSOLE
+appender.testlogger.target = SYSTEM_ERR
+appender.testlogger.layout.type = PatternLayout
+appender.testlogger.layout.pattern = %-4r [%t] %-5p %c - %m%n
+
+logger.debezium.name = io.debezium
+logger.debezium.level = INFO
+
+logger.testcontainers.name = org.testcontainers
+logger.testcontainers.level = WARN
+
+logger.docker.name = com.github.dockerjava
+logger.docker.level = WARN
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/pom.xml
index f7afcf9803b..490a0d33a6c 100644
--- a/flink-cdc-connect/flink-cdc-pipeline-connectors/pom.xml
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/pom.xml
@@ -41,6 +41,7 @@ limitations under the License.
flink-cdc-pipeline-connector-maxcompute
flink-cdc-pipeline-connector-iceberg
flink-cdc-pipeline-connector-fluss
+ flink-cdc-pipeline-connector-sqlserver
flink-cdc-pipeline-connector-hudi
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSource.java
index 80b4cf235bb..f7c62fb3d58 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSource.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSource.java
@@ -62,6 +62,12 @@
public class SqlServerStreamingChangeEventSource
implements StreamingChangeEventSource {
+ /**
+ * SQL Server Msg 313 is misleading for CDC table-valued functions. In this path it usually
+ * means the requested LSN interval fell behind the cleanup window.
+ */
+ private static final int INVALID_CDC_LSN_RANGE_ERROR_CODE = 313;
+
private static final Pattern MISSING_CDC_FUNCTION_CHANGES_ERROR =
Pattern.compile("Invalid object name '(.*)\\.cdc.fn_cdc_get_all_changes_(.*)'\\.");
@@ -452,7 +458,8 @@ public boolean executeIteration(
afterHandleLsn(partition, toLsn);
} catch (SQLException e) {
tablesSlot.set(
- processErrorFromChangeTableQuery(databaseName, e, tablesSlot.get()));
+ processErrorFromChangeTableQuery(
+ databaseName, lastProcessedPosition, e, tablesSlot.get()));
}
}
} catch (Exception e) {
@@ -501,7 +508,10 @@ private void migrateTable(
}
private SqlServerChangeTable[] processErrorFromChangeTableQuery(
- String databaseName, SQLException exception, SqlServerChangeTable[] currentChangeTables)
+ String databaseName,
+ TxLogPosition lastProcessedPosition,
+ SQLException exception,
+ SqlServerChangeTable[] currentChangeTables)
throws Exception {
final Matcher m = MISSING_CDC_FUNCTION_CHANGES_ERROR.matcher(exception.getMessage());
if (m.matches() && m.group(1).equals(databaseName)) {
@@ -511,9 +521,43 @@ private SqlServerChangeTable[] processErrorFromChangeTableQuery(
.filter(x -> !x.getCaptureInstance().equals(captureName))
.toArray(SqlServerChangeTable[]::new);
}
+ if (exception.getErrorCode() == INVALID_CDC_LSN_RANGE_ERROR_CODE) {
+ throw buildInvalidCdcLsnRangeException(
+ databaseName,
+ lastProcessedPosition,
+ Arrays.stream(currentChangeTables)
+ .map(SqlServerChangeTable::getCaptureInstance)
+ .distinct()
+ .collect(Collectors.toList()),
+ exception);
+ }
throw exception;
}
+ static IllegalStateException buildInvalidCdcLsnRangeException(
+ String databaseName,
+ TxLogPosition lastProcessedPosition,
+ List captureInstances,
+ SQLException exception) {
+ final String captureInstancesMessage =
+ captureInstances.isEmpty() ? "unknown" : String.join(", ", captureInstances);
+ return new IllegalStateException(
+ String.format(
+ "SQL Server rejected the CDC query window for database '%s' at offset '%s' "
+ + "(error %d). This usually means the CDC data for the current LSN "
+ + "has already been purged and the saved offset is no longer valid. "
+ + "Affected capture instances: [%s]. Please restart with a new "
+ + "snapshot or manually advance the offset to the minimum available "
+ + "LSN for the affected capture instance(s), for example with "
+ + "sys.fn_cdc_get_min_lsn(''). Original message: %s",
+ databaseName,
+ lastProcessedPosition,
+ exception.getErrorCode(),
+ captureInstancesMessage,
+ exception.getMessage()),
+ exception);
+ }
+
private SqlServerChangeTable[] getChangeTablesToQuery(
SqlServerPartition partition, SqlServerOffsetContext offsetContext, Lsn toLsn)
throws SQLException, InterruptedException {
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerSourceBuilder.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerSourceBuilder.java
index 64141f05b51..080535e15bb 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerSourceBuilder.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerSourceBuilder.java
@@ -19,6 +19,7 @@
import org.apache.flink.cdc.connectors.base.options.StartupOptions;
import org.apache.flink.cdc.connectors.base.source.jdbc.JdbcIncrementalSource;
+import org.apache.flink.cdc.connectors.sqlserver.source.config.SqlServerSourceConfig;
import org.apache.flink.cdc.connectors.sqlserver.source.config.SqlServerSourceConfigFactory;
import org.apache.flink.cdc.connectors.sqlserver.source.dialect.SqlServerDialect;
import org.apache.flink.cdc.connectors.sqlserver.source.offset.LsnFactory;
@@ -40,10 +41,6 @@ public class SqlServerSourceBuilder {
private final SqlServerSourceConfigFactory configFactory = new SqlServerSourceConfigFactory();
- private LsnFactory offsetFactory;
-
- private SqlServerDialect dialect;
-
private DebeziumDeserializationSchema deserializer;
/** Hostname of the SQL Server database server. */
@@ -244,9 +241,10 @@ public SqlServerSourceBuilder assignUnboundedChunkFirst(boolean assignUnbound
* @return a SqlSeverParallelSource with the settings made for this builder.
*/
public SqlServerIncrementalSource build() {
- this.offsetFactory = new LsnFactory();
- this.dialect = new SqlServerDialect(configFactory.create(0));
- return new SqlServerIncrementalSource(
+ SqlServerSourceConfig sourceConfig = configFactory.create(0);
+ LsnFactory offsetFactory = new LsnFactory(sourceConfig);
+ SqlServerDialect dialect = new SqlServerDialect(sourceConfig);
+ return new SqlServerIncrementalSource<>(
configFactory, checkNotNull(deserializer), offsetFactory, dialect);
}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/config/SqlServerSourceConfig.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/config/SqlServerSourceConfig.java
index 356b6a0135a..5ff78c307c7 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/config/SqlServerSourceConfig.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/config/SqlServerSourceConfig.java
@@ -57,6 +57,7 @@ public SqlServerSourceConfig(
int connectionPoolSize,
String chunkKeyColumn,
boolean skipSnapshotBackfill,
+ boolean scanNewlyAddedTableEnabled,
boolean assignUnboundedChunkFirst) {
super(
startupOptions,
@@ -83,7 +84,7 @@ public SqlServerSourceConfig(
connectionPoolSize,
chunkKeyColumn,
skipSnapshotBackfill,
- false,
+ scanNewlyAddedTableEnabled,
assignUnboundedChunkFirst);
}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/config/SqlServerSourceConfigFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/config/SqlServerSourceConfigFactory.java
index 661094a8942..f485019ac71 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/config/SqlServerSourceConfigFactory.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/config/SqlServerSourceConfigFactory.java
@@ -18,6 +18,8 @@
package org.apache.flink.cdc.connectors.sqlserver.source.config;
import org.apache.flink.cdc.connectors.base.config.JdbcSourceConfigFactory;
+import org.apache.flink.cdc.connectors.base.options.StartupMode;
+import org.apache.flink.cdc.connectors.base.options.StartupOptions;
import org.apache.flink.cdc.connectors.base.source.EmbeddedFlinkDatabaseHistory;
import io.debezium.config.Configuration;
@@ -35,6 +37,19 @@ public class SqlServerSourceConfigFactory extends JdbcSourceConfigFactory {
private static final String DATABASE_SERVER_NAME = "sqlserver_transaction_log_source";
private static final String DRIVER_ClASS_NAME = "com.microsoft.sqlserver.jdbc.SQLServerDriver";
+ @Override
+ public SqlServerSourceConfigFactory startupOptions(StartupOptions startupOptions) {
+ if (startupOptions.startupMode != StartupMode.INITIAL
+ && startupOptions.startupMode != StartupMode.SNAPSHOT
+ && startupOptions.startupMode != StartupMode.LATEST_OFFSET
+ && startupOptions.startupMode != StartupMode.TIMESTAMP) {
+ throw new UnsupportedOperationException(
+ "Unsupported startup mode: " + startupOptions.startupMode);
+ }
+ this.startupOptions = startupOptions;
+ return this;
+ }
+
@Override
public SqlServerSourceConfig create(int subtask) {
checkSupportCheckpointsAfterTasksFinished(closeIdleReaders);
@@ -68,9 +83,18 @@ public SqlServerSourceConfig create(int subtask) {
case INITIAL:
props.setProperty("snapshot.mode", "initial");
break;
+ case SNAPSHOT:
+ // Debezium snapshot only; do not start the streaming phase
+ props.setProperty("snapshot.mode", "initial_only");
+ break;
case LATEST_OFFSET:
props.setProperty("snapshot.mode", "schema_only");
break;
+ case TIMESTAMP:
+ // We recover stream reading from timestamp-mapped LSN and only need schema
+ // snapshot.
+ props.setProperty("snapshot.mode", "schema_only");
+ break;
default:
throw new UnsupportedOperationException();
}
@@ -104,6 +128,7 @@ public SqlServerSourceConfig create(int subtask) {
connectionPoolSize,
chunkKeyColumn,
skipSnapshotBackfill,
+ scanNewlyAddedTableEnabled,
assignUnboundedChunkFirst);
}
}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/offset/LsnFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/offset/LsnFactory.java
index 23c56c4bd81..35cf75c7eec 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/offset/LsnFactory.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/offset/LsnFactory.java
@@ -19,14 +19,42 @@
import org.apache.flink.cdc.connectors.base.source.meta.offset.Offset;
import org.apache.flink.cdc.connectors.base.source.meta.offset.OffsetFactory;
+import org.apache.flink.cdc.connectors.sqlserver.source.config.SqlServerSourceConfig;
+import org.apache.flink.cdc.connectors.sqlserver.source.utils.SqlServerConnectionUtils;
+import org.apache.flink.cdc.connectors.sqlserver.source.utils.SqlServerUtils;
import io.debezium.connector.sqlserver.Lsn;
import io.debezium.connector.sqlserver.SourceInfo;
+import io.debezium.connector.sqlserver.SqlServerConnection;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.util.Calendar;
+import java.util.List;
import java.util.Map;
+import java.util.TimeZone;
/** A factory to create {@link LsnOffset}. */
public class LsnFactory extends OffsetFactory {
+ private static final Logger LOG = LoggerFactory.getLogger(LsnFactory.class);
+ private static final String MAP_TIME_TO_LSN_QUERY =
+ "SELECT %s.sys.fn_cdc_map_time_to_lsn('smallest greater than or equal', ?)";
+ private static final String MAP_LSN_TO_TIME_QUERY = "SELECT %s.sys.fn_cdc_map_lsn_to_time(?)";
+
+ private final SqlServerSourceConfig sourceConfig;
+
+ public LsnFactory() {
+ this(null);
+ }
+
+ public LsnFactory(SqlServerSourceConfig sourceConfig) {
+ this.sourceConfig = sourceConfig;
+ }
+
@Override
public Offset newOffset(Map offset) {
Lsn changeLsn = Lsn.valueOf(offset.get(SourceInfo.CHANGE_LSN_KEY));
@@ -52,7 +80,70 @@ public Offset newOffset(Long position) {
@Override
public Offset createTimestampOffset(long timestampMillis) {
- throw new UnsupportedOperationException("not supported create new Offset by timestamp.");
+ if (sourceConfig == null) {
+ throw new UnsupportedOperationException(
+ "Timestamp startup mode requires SqlServerSourceConfig in LsnFactory.");
+ }
+ List databaseList = sourceConfig.getDatabaseList();
+ if (databaseList == null || databaseList.isEmpty()) {
+ throw new IllegalStateException(
+ "Cannot create timestamp offset without configured SQL Server database.");
+ }
+ final String databaseName = databaseList.get(0);
+ final ZoneId serverZoneId = ZoneId.of(sourceConfig.getServerTimeZone());
+ final Timestamp startupTimestamp =
+ Timestamp.valueOf(
+ Instant.ofEpochMilli(timestampMillis)
+ .atZone(serverZoneId)
+ .toLocalDateTime());
+ final String mapTimeToLsnQuery =
+ String.format(MAP_TIME_TO_LSN_QUERY, SqlServerUtils.quote(databaseName));
+ final Calendar serverCalendar = Calendar.getInstance(TimeZone.getTimeZone(serverZoneId));
+ try (SqlServerConnection connection =
+ SqlServerConnectionUtils.createSqlServerConnection(
+ sourceConfig.getDbzConnectorConfig())) {
+ Lsn mappedLsn =
+ connection.prepareQueryAndMap(
+ mapTimeToLsnQuery,
+ statement ->
+ statement.setTimestamp(1, startupTimestamp, serverCalendar),
+ rs -> {
+ if (!rs.next()) {
+ throw new SQLException(
+ "No result returned when mapping timestamp to LSN.");
+ }
+ return Lsn.valueOf(rs.getBytes(1));
+ });
+ if (!mappedLsn.isAvailable()) {
+ Lsn latestLsn = connection.getMaxTransactionLsn(databaseName);
+ if (!latestLsn.isAvailable()) {
+ throw new IllegalStateException(
+ String.format(
+ "Cannot create timestamp offset for %s because SQL Server returns no available LSN.",
+ databaseName));
+ }
+ LOG.warn(
+ "No LSN is mapped for startup timestamp {} on database {}, fallback to latest LSN {}.",
+ timestampMillis,
+ databaseName,
+ latestLsn);
+ mappedLsn = latestLsn;
+ }
+ logResolvedStartupLsn(
+ connection,
+ databaseName,
+ timestampMillis,
+ startupTimestamp,
+ serverZoneId,
+ mappedLsn);
+ return new LsnOffset(mappedLsn, mappedLsn, null);
+ } catch (Exception e) {
+ throw new IllegalStateException(
+ String.format(
+ "Failed to create SQL Server offset from timestamp %s for database %s.",
+ timestampMillis, databaseName),
+ e);
+ }
}
@Override
@@ -64,4 +155,48 @@ public Offset createInitialOffset() {
public Offset createNoStoppingOffset() {
return LsnOffset.NO_STOPPING_OFFSET;
}
+
+ private void logResolvedStartupLsn(
+ SqlServerConnection connection,
+ String databaseName,
+ long timestampMillis,
+ Timestamp startupTimestamp,
+ ZoneId serverZoneId,
+ Lsn restartLsn) {
+ try {
+ Timestamp restartTimestamp =
+ connection.prepareQueryAndMap(
+ String.format(
+ MAP_LSN_TO_TIME_QUERY, SqlServerUtils.quote(databaseName)),
+ statement -> statement.setBytes(1, restartLsn.getBinary()),
+ rs -> {
+ if (!rs.next()) {
+ throw new SQLException(
+ "No result returned when mapping LSN to timestamp.");
+ }
+ return rs.getTimestamp(1);
+ });
+ LOG.info(
+ "Resolved SQL Server startup timestamp {} (server time: {}) to restart LSN {} with commit time {} on database {}.",
+ timestampMillis,
+ startupTimestamp.toLocalDateTime().atZone(serverZoneId),
+ restartLsn,
+ restartTimestamp == null
+ ? "unknown"
+ : restartTimestamp.toLocalDateTime().atZone(serverZoneId),
+ databaseName);
+ } catch (Exception e) {
+ LOG.info(
+ "Resolved SQL Server startup timestamp {} (server time: {}) to restart LSN {} on database {}.",
+ timestampMillis,
+ startupTimestamp.toLocalDateTime().atZone(serverZoneId),
+ restartLsn,
+ databaseName);
+ LOG.warn(
+ "Failed to query commit time for restart LSN {} on database {}, but startup continues.",
+ restartLsn,
+ databaseName,
+ e);
+ }
+ }
}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableFactory.java
index a841366b20c..5ca26f31620 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableFactory.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableFactory.java
@@ -17,11 +17,11 @@
package org.apache.flink.cdc.connectors.sqlserver.table;
+import org.apache.flink.cdc.connectors.base.options.StartupMode;
import org.apache.flink.cdc.connectors.base.options.StartupOptions;
import org.apache.flink.cdc.connectors.base.utils.OptionUtils;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ConfigOptions;
-import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.catalog.ResolvedSchema;
@@ -107,7 +107,14 @@ public class SqlServerTableFactory implements DynamicTableSourceFactory {
.defaultValue("initial")
.withDescription(
"Optional startup mode for SqlServer CDC consumer, valid enumerations are "
- + "\"initial\" and \"latest-offset\"");
+ + "\"initial\", \"latest-offset\" and \"timestamp\"");
+
+ public static final ConfigOption SCAN_STARTUP_TIMESTAMP_MILLIS =
+ ConfigOptions.key("scan.startup.timestamp-millis")
+ .longType()
+ .noDefaultValue()
+ .withDescription(
+ "Optional timestamp used in case of \"timestamp\" startup mode");
@Override
public DynamicTableSource createDynamicTableSource(Context context) {
@@ -124,10 +131,11 @@ public DynamicTableSource createDynamicTableSource(Context context) {
ZoneId serverTimeZone = ZoneId.of(config.get(SERVER_TIME_ZONE));
int port = config.get(PORT);
StartupOptions startupOptions = getStartupOptions(config);
+ boolean enableParallelRead = config.get(SCAN_INCREMENTAL_SNAPSHOT_ENABLED);
+ validateStartupOptions(startupOptions, enableParallelRead);
ResolvedSchema physicalSchema =
getPhysicalSchema(context.getCatalogTable().getResolvedSchema());
- boolean enableParallelRead = config.get(SCAN_INCREMENTAL_SNAPSHOT_ENABLED);
int splitSize = config.get(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE);
int splitMetaGroupSize = config.get(CHUNK_META_GROUP_SIZE);
int fetchSize = config.get(SCAN_SNAPSHOT_FETCH_SIZE);
@@ -153,7 +161,7 @@ public DynamicTableSource createDynamicTableSource(Context context) {
validateDistributionFactorLower(distributionFactorLower);
}
- OptionUtils.printOptions(IDENTIFIER, ((Configuration) config).toMap());
+ OptionUtils.printOptions(IDENTIFIER, config.toMap());
return new SqlServerTableSource(
physicalSchema,
@@ -203,6 +211,7 @@ public Set> optionalOptions() {
options.add(PORT);
options.add(SERVER_TIME_ZONE);
options.add(SCAN_STARTUP_MODE);
+ options.add(SCAN_STARTUP_TIMESTAMP_MILLIS);
options.add(SCAN_INCREMENTAL_SNAPSHOT_ENABLED);
options.add(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE);
options.add(CHUNK_META_GROUP_SIZE);
@@ -221,6 +230,7 @@ public Set> optionalOptions() {
private static final String SCAN_STARTUP_MODE_VALUE_INITIAL = "initial";
private static final String SCAN_STARTUP_MODE_VALUE_LATEST = "latest-offset";
+ private static final String SCAN_STARTUP_MODE_VALUE_TIMESTAMP = "timestamp";
private static StartupOptions getStartupOptions(ReadableConfig config) {
String modeString = config.get(SCAN_STARTUP_MODE);
@@ -232,17 +242,42 @@ private static StartupOptions getStartupOptions(ReadableConfig config) {
case SCAN_STARTUP_MODE_VALUE_LATEST:
return StartupOptions.latest();
+ case SCAN_STARTUP_MODE_VALUE_TIMESTAMP:
+ Long startupTimestampMillis = config.get(SCAN_STARTUP_TIMESTAMP_MILLIS);
+ if (startupTimestampMillis == null) {
+ throw new ValidationException(
+ String.format(
+ "To use timestamp startup mode, the startup timestamp millis '%s' must be set.",
+ SCAN_STARTUP_TIMESTAMP_MILLIS.key()));
+ }
+ return StartupOptions.timestamp(startupTimestampMillis);
+
default:
throw new ValidationException(
String.format(
- "Invalid value for option '%s'. Supported values are [%s, %s], but was: %s",
+ "Invalid value for option '%s'. Supported values are [%s, %s, %s], but was: %s",
SCAN_STARTUP_MODE.key(),
SCAN_STARTUP_MODE_VALUE_INITIAL,
SCAN_STARTUP_MODE_VALUE_LATEST,
+ SCAN_STARTUP_MODE_VALUE_TIMESTAMP,
modeString));
}
}
+ private static void validateStartupOptions(
+ StartupOptions startupOptions, boolean enableParallelRead) {
+ if (startupOptions.startupMode == StartupMode.TIMESTAMP && !enableParallelRead) {
+ throw new ValidationException(
+ String.format(
+ "Option '%s'='%s' requires '%s'='true' because timestamp startup "
+ + "is only supported by the incremental SQL Server source; "
+ + "the legacy non-parallel source does not support it.",
+ SCAN_STARTUP_MODE.key(),
+ SCAN_STARTUP_MODE_VALUE_TIMESTAMP,
+ SCAN_INCREMENTAL_SNAPSHOT_ENABLED.key()));
+ }
+ }
+
/** Checks the value of given integer option is valid. */
private void validateIntegerOption(
ConfigOption option, int optionValue, int exclusiveMin) {
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSourceTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSourceTest.java
new file mode 100644
index 00000000000..46370a6ac61
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSourceTest.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed under the Apache Software License version 2.0, available at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ */
+
+package io.debezium.connector.sqlserver;
+
+import org.junit.jupiter.api.Test;
+
+import java.sql.SQLException;
+import java.util.Arrays;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+class SqlServerStreamingChangeEventSourceTest {
+
+ @Test
+ void shouldFailFastWhenCdcLsnRangeIsInvalid() {
+ SQLException sqlException =
+ new SQLException("LSN range is no longer available", "S0001", 313);
+
+ IllegalStateException exception =
+ SqlServerStreamingChangeEventSource.buildInvalidCdcLsnRangeException(
+ "inventory",
+ TxLogPosition.valueOf(Lsn.valueOf("00000027:00000758:0005")),
+ Arrays.asList("dbo_products", "dbo_orders"),
+ sqlException);
+
+ assertThat(exception)
+ .hasMessageContaining("database 'inventory'")
+ .hasMessageContaining("error 313")
+ .hasMessageContaining("saved offset is no longer valid")
+ .hasMessageContaining("new snapshot")
+ .hasMessageContaining("minimum available LSN")
+ .hasMessageContaining("sys.fn_cdc_get_min_lsn")
+ .hasMessageContaining("dbo_products, dbo_orders")
+ .hasCause(sqlException);
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/SqlServerTestBase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/SqlServerTestBase.java
index 5469eb70604..59ba390ce93 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/SqlServerTestBase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/SqlServerTestBase.java
@@ -33,9 +33,10 @@
import org.testcontainers.containers.output.Slf4jLogConsumer;
import org.testcontainers.lifecycle.Startables;
-import java.net.URL;
-import java.nio.file.Files;
-import java.nio.file.Paths;
+import java.io.BufferedReader;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.SQLException;
@@ -92,7 +93,7 @@ protected Connection getJdbcConnection() throws SQLException {
MSSQL_SERVER_CONTAINER.getPassword());
}
- private static void dropTestDatabase(Connection connection, String databaseName)
+ protected static void dropTestDatabase(Connection connection, String databaseName)
throws SQLException {
try {
Awaitility.await("Disabling CDC")
@@ -175,14 +176,19 @@ protected static void disableDbCdc(Connection connection, String name) throws SQ
*/
protected void initializeSqlServerTable(String sqlFile) {
final String ddlFile = String.format("ddl/%s.sql", sqlFile);
- final URL ddlTestFile = SqlServerTestBase.class.getClassLoader().getResource(ddlFile);
+ final InputStream ddlTestFile =
+ SqlServerTestBase.class.getClassLoader().getResourceAsStream(ddlFile);
Assertions.assertThat(ddlTestFile).withFailMessage("Cannot locate " + ddlFile).isNotNull();
- try (Connection connection = getJdbcConnection();
+ try (InputStream inputStream = ddlTestFile;
+ BufferedReader reader =
+ new BufferedReader(
+ new InputStreamReader(inputStream, StandardCharsets.UTF_8));
+ Connection connection = getJdbcConnection();
Statement statement = connection.createStatement()) {
dropTestDatabase(connection, sqlFile);
final List statements =
Arrays.stream(
- Files.readAllLines(Paths.get(ddlTestFile.toURI())).stream()
+ reader.lines()
.map(String::trim)
.filter(x -> !x.startsWith("--") && !x.isEmpty())
.map(
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/config/SqlServerSourceConfigFactoryTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/config/SqlServerSourceConfigFactoryTest.java
new file mode 100644
index 00000000000..36b1b5f6b38
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/config/SqlServerSourceConfigFactoryTest.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.sqlserver.source.config;
+
+import org.apache.flink.cdc.connectors.base.options.StartupOptions;
+
+import org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+/** Unit tests for {@link SqlServerSourceConfigFactory}. */
+class SqlServerSourceConfigFactoryTest {
+
+ @Test
+ void testTimestampStartupOption() {
+ long startupTimestampMillis = 1667232000000L;
+
+ SqlServerSourceConfigFactory factory = new SqlServerSourceConfigFactory();
+ factory.hostname("localhost")
+ .port(1433)
+ .databaseList("inventory")
+ .tableList("inventory.dbo.products")
+ .username("flinkuser")
+ .password("flinkpw")
+ .serverTimeZone("UTC");
+ factory.startupOptions(StartupOptions.timestamp(startupTimestampMillis));
+
+ SqlServerSourceConfig sourceConfig = factory.create(0);
+
+ Assertions.assertThat(sourceConfig.getStartupOptions())
+ .isEqualTo(StartupOptions.timestamp(startupTimestampMillis));
+ Assertions.assertThat(sourceConfig.getDbzProperties().getProperty("snapshot.mode"))
+ .isEqualTo("schema_only");
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerConnectorITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerConnectorITCase.java
index 25cb6ba59e2..ae1515f0c22 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerConnectorITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerConnectorITCase.java
@@ -27,6 +27,7 @@
import org.apache.flink.table.utils.LegacyRowResource;
import org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.RegisterExtension;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
@@ -36,6 +37,7 @@
import java.sql.Statement;
import java.util.Arrays;
import java.util.List;
+import java.util.TimeZone;
import java.util.concurrent.ExecutionException;
import static org.apache.flink.api.common.JobStatus.RUNNING;
@@ -275,6 +277,84 @@ void testStartupFromLatestOffset(boolean parallelismSnapshot) throws Exception {
result.getJobClient().get().cancel().get();
}
+ @Test
+ void testStartupFromTimestampWithServerTimeZone() throws Exception {
+ TimeZone defaultTimeZone = TimeZone.getDefault();
+ try {
+ TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"));
+ setup(true);
+ initializeSqlServerTable("inventory");
+
+ long startupTimestampMillis = System.currentTimeMillis();
+
+ String sourceDDL =
+ String.format(
+ "CREATE TABLE debezium_source ("
+ + " id INT NOT NULL,"
+ + " name STRING,"
+ + " description STRING,"
+ + " weight DECIMAL(10,3)"
+ + ") WITH ("
+ + " 'connector' = 'sqlserver-cdc',"
+ + " 'hostname' = '%s',"
+ + " 'port' = '%s',"
+ + " 'username' = '%s',"
+ + " 'password' = '%s',"
+ + " 'database-name' = '%s',"
+ + " 'table-name' = '%s',"
+ + " 'server-time-zone' = 'UTC',"
+ + " 'scan.startup.mode' = 'timestamp',"
+ + " 'scan.startup.timestamp-millis' = '%s',"
+ + " 'scan.incremental.snapshot.enabled' = 'true'"
+ + ")",
+ MSSQL_SERVER_CONTAINER.getHost(),
+ MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT),
+ MSSQL_SERVER_CONTAINER.getUsername(),
+ MSSQL_SERVER_CONTAINER.getPassword(),
+ "inventory",
+ "dbo.products",
+ startupTimestampMillis);
+ String sinkDDL =
+ "CREATE TABLE sink "
+ + " WITH ("
+ + " 'connector' = 'values',"
+ + " 'sink-insert-only' = 'false'"
+ + ") LIKE debezium_source (EXCLUDING OPTIONS)";
+ tEnv.executeSql(sourceDDL);
+ tEnv.executeSql(sinkDDL);
+
+ TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM debezium_source");
+
+ do {
+ Thread.sleep(5000L);
+ } while (result.getJobClient().get().getJobStatus().get() != RUNNING);
+ Thread.sleep(5000L);
+
+ try (Connection connection = getJdbcConnection();
+ Statement statement = connection.createStatement()) {
+ statement.execute(
+ "INSERT INTO inventory.dbo.products (name,description,weight) VALUES ('hammer','18oz carpenters hammer',1.2);");
+ statement.execute(
+ "INSERT INTO inventory.dbo.products (name,description,weight) VALUES ('scooter','Big 3-wheel scooter',5.20);");
+ }
+
+ waitForSinkSize("sink", 2);
+
+ String[] expected =
+ new String[] {
+ "110,hammer,18oz carpenters hammer,1.200",
+ "111,scooter,Big 3-wheel scooter,5.200"
+ };
+
+ List actual = TestValuesTableFactory.getResultsAsStrings("sink");
+ Assertions.assertThat(actual).containsExactlyInAnyOrder(expected);
+
+ result.getJobClient().get().cancel().get();
+ } finally {
+ TimeZone.setDefault(defaultTimeZone);
+ }
+ }
+
@ParameterizedTest
@ValueSource(booleans = {true, false})
void testAllTypes(boolean parallelismSnapshot) throws Throwable {
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableFactoryTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableFactoryTest.java
index 492b2795bb9..c286e7418bb 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableFactoryTest.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableFactoryTest.java
@@ -24,6 +24,7 @@
import org.apache.flink.configuration.Configuration;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.catalog.CatalogTableAdapter;
import org.apache.flink.table.catalog.Column;
import org.apache.flink.table.catalog.ObjectIdentifier;
@@ -251,6 +252,73 @@ void testMetadataColumns() {
Assertions.assertThat(actualSource).isEqualTo(expectedSource);
}
+ @Test
+ void testStartupFromTimestamp() {
+ Map properties = getAllOptions();
+ properties.put("scan.startup.mode", "timestamp");
+ properties.put("scan.startup.timestamp-millis", "1667232000000");
+ properties.put("scan.incremental.snapshot.enabled", "true");
+
+ DynamicTableSource actualSource = createTableSource(SCHEMA, properties);
+ SqlServerTableSource expectedSource =
+ new SqlServerTableSource(
+ SCHEMA,
+ 1433,
+ MY_LOCALHOST,
+ MY_DATABASE,
+ MY_TABLE,
+ ZoneId.of("UTC"),
+ MY_USERNAME,
+ MY_PASSWORD,
+ PROPERTIES,
+ StartupOptions.timestamp(1667232000000L),
+ true,
+ SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE.defaultValue(),
+ SourceOptions.CHUNK_META_GROUP_SIZE.defaultValue(),
+ SourceOptions.SCAN_SNAPSHOT_FETCH_SIZE.defaultValue(),
+ JdbcSourceOptions.CONNECT_TIMEOUT.defaultValue(),
+ JdbcSourceOptions.CONNECT_MAX_RETRIES.defaultValue(),
+ JdbcSourceOptions.CONNECTION_POOL_SIZE.defaultValue(),
+ JdbcSourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND
+ .defaultValue(),
+ JdbcSourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND
+ .defaultValue(),
+ null,
+ false,
+ JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP.defaultValue(),
+ JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED
+ .defaultValue());
+ Assertions.assertThat(actualSource).isEqualTo(expectedSource);
+ }
+
+ @Test
+ void testTimestampStartupRequiresIncrementalSnapshot() {
+ Map properties = getAllOptions();
+ properties.put("scan.startup.mode", "timestamp");
+ properties.put("scan.startup.timestamp-millis", "1667232000000");
+ properties.put("scan.incremental.snapshot.enabled", "false");
+
+ Assertions.assertThatThrownBy(() -> createTableSource(SCHEMA, properties))
+ .isInstanceOf(ValidationException.class)
+ .hasMessageContaining("scan.startup.mode")
+ .hasMessageContaining("timestamp")
+ .hasMessageContaining("scan.incremental.snapshot.enabled");
+ }
+
+ @Test
+ void testTimestampStartupRequiresTimestampMillis() {
+ Map properties = getAllOptions();
+ properties.put("scan.startup.mode", "timestamp");
+ properties.put("scan.incremental.snapshot.enabled", "true");
+
+ Assertions.assertThatThrownBy(() -> createTableSource(SCHEMA, properties))
+ .isInstanceOf(ValidationException.class)
+ .hasRootCauseInstanceOf(ValidationException.class)
+ .rootCause()
+ .hasMessageContaining("scan.startup.timestamp-millis")
+ .hasMessageContaining("must be set");
+ }
+
private Map getAllOptions() {
Map options = new HashMap<>();
options.put("connector", "sqlserver-cdc");