[sqlserver] Sqlserver incremental source. (#1732)

* [sqlserver] Sqlserver incremental source

---------

Co-authored-by: gongzhongqiang <gongzhongqiang@gigacloudtech.com>
pull/2176/head
gongzhongqiang 2 years ago committed by GitHub
parent c8c8107bf0
commit 8cef4af544
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23

@ -81,8 +81,7 @@ CREATE TABLE orders (
'username' = 'sa',
'password' = 'Password!',
'database-name' = 'inventory',
'schema-name' = 'dbo',
'table-name' = 'orders'
'table-name' = 'dob.orders'
);
-- read snapshot and binlogs from orders table
@ -139,19 +138,12 @@ Connector Options
<td>String</td>
<td>Database name of the SQLServer database to monitor.</td>
</tr>
<tr>
<td>schema-name</td>
<td>required</td>
<td style="word-wrap: break-word;">(none)</td>
<td>String</td>
<td>Schema name of the SQLServer database to monitor.</td>
</tr>
<tr>
<td>table-name</td>
<td>required</td>
<td style="word-wrap: break-word;">(none)</td>
<td>String</td>
<td>Table name of the SQLServer database to monitor.</td>
<td>Table name of the SQLServer database to monitor, e.g.: "db1.table1"</td>
</tr>
<tr>
<td>port</td>
@ -167,6 +159,38 @@ Connector Options
<td>String</td>
<td>The session time zone in database server, e.g. "Asia/Shanghai".</td>
</tr>
<tr>
<td>scan.incremental.snapshot.enabled</td>
<td>optional</td>
<td style="word-wrap: break-word;">true</td>
<td>Boolean</td>
<td>Whether enable parallelism snapshot.</td>
</tr>
<tr>
<td>chunk-meta.group.size</td>
<td>optional</td>
<td style="word-wrap: break-word;">1000</td>
<td>Integer</td>
<td>The group size of chunk meta, if the meta size exceeds the group size, the meta will be divided into multiple groups.</td>
</tr>
<tr>
<td>chunk-key.even-distribution.factor.lower-bound</td>
<td>optional</td>
<td style="word-wrap: break-word;">0.05d</td>
<td>Double</td>
<td>The lower bound of chunk key distribution factor. The distribution factor is used to determine whether the table is evenly distribution or not.
The table chunks would use evenly calculation optimization when the data distribution is even, and the query for splitting would happen when it is uneven.
The distribution factor could be calculated by (MAX(id) - MIN(id) + 1) / rowCount.</td>
</tr>
<tr>
<td>chunk-key.even-distribution.factor.upper-bound</td>
<td>optional</td>
<td style="word-wrap: break-word;">1000.0d</td>
<td>Double</td>
<td>The upper bound of chunk key distribution factor. The distribution factor is used to determine whether the table is evenly distribution or not.
The table chunks would use evenly calculation optimization when the data distribution is even, and the query for splitting would happen when it is uneven.
The distribution factor could be calculated by (MAX(id) - MIN(id) + 1) / rowCount.</td>
</tr>
<tr>
<td>debezium.*</td>
<td>optional</td>
@ -248,8 +272,7 @@ CREATE TABLE products (
'username' = 'sa',
'password' = 'Password!',
'database-name' = 'inventory',
'schema-name' = 'dbo',
'table-name' = 'products'
'table-name' = 'dbo.products'
);
```
@ -306,6 +329,47 @@ public class SqlServerSourceExample {
}
}
```
The SQLServer CDC incremental connector (after 2.4.0) can be used as the following shows:
```java
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import com.ververica.cdc.connectors.base.options.StartupOptions;
import com.ververica.cdc.connectors.sqlserver.source.SqlServerSourceBuilder;
import com.ververica.cdc.connectors.sqlserver.source.SqlServerSourceBuilder.SqlServerIncrementalSource;
import com.ververica.cdc.debezium.JsonDebeziumDeserializationSchema;
public class SqlServerIncrementalSourceExample {
public static void main(String[] args) throws Exception {
SqlServerIncrementalSource<String> sqlServerSource =
new SqlServerSourceBuilder()
.hostname("localhost")
.port(1433)
.databaseList("inventory")
.tableList("dbo.products")
.username("sa")
.password("Password!")
.deserializer(new JsonDebeziumDeserializationSchema())
.startupOptions(StartupOptions.initial())
.build();
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
// enable checkpoint
env.enableCheckpointing(3000);
// set the source parallelism to 2
env.fromSource(
sqlServerSource,
WatermarkStrategy.noWatermarks(),
"SqlServerIncrementalSource")
.setParallelism(2)
.print()
.setParallelism(1);
env.execute("Print SqlServer Snapshot + Change Stream");
}
}
```
**Note:** Please refer [Deserialization](../about.html#deserialization) for more details about the JSON deserialization.
Data Type Mapping

@ -143,8 +143,7 @@ Flink SQL> CREATE TABLE products (
'username' = 'sa',
'password' = 'Password!',
'database-name' = 'inventory',
'schema-name' = 'dbo',
'table-name' = 'products'
'table-name' = 'dbo.products'
);
Flink SQL> CREATE TABLE orders (
@ -161,8 +160,7 @@ Flink SQL> CREATE TABLE orders (
'username' = 'sa',
'password' = 'Password!',
'database-name' = 'inventory',
'schema-name' = 'dbo',
'table-name' = 'orders'
'table-name' = 'dbo.orders'
);
Flink SQL> CREATE TABLE enriched_orders (

@ -143,8 +143,7 @@ Flink SQL> CREATE TABLE products (
'username' = 'sa',
'password' = 'Password!',
'database-name' = 'inventory',
'schema-name' = 'dbo',
'table-name' = 'products'
'table-name' = 'dbo.products'
);
Flink SQL> CREATE TABLE orders (
@ -161,8 +160,7 @@ Flink SQL> CREATE TABLE orders (
'username' = 'sa',
'password' = 'Password!',
'database-name' = 'inventory',
'schema-name' = 'dbo',
'table-name' = 'orders'
'table-name' = 'dbo.orders'
);
Flink SQL> CREATE TABLE enriched_orders (

@ -99,7 +99,8 @@ public class SourceRecordUtils {
public static boolean isDataChangeRecord(SourceRecord record) {
Schema valueSchema = record.valueSchema();
Struct value = (Struct) record.value();
return valueSchema.field(Envelope.FieldName.OPERATION) != null
return valueSchema != null
&& valueSchema.field(Envelope.FieldName.OPERATION) != null
&& value.getString(Envelope.FieldName.OPERATION) != null;
}

@ -23,6 +23,7 @@ import org.junit.After;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.runners.Parameterized;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testcontainers.containers.MSSQLServerContainer;
@ -38,6 +39,7 @@ import java.sql.DriverManager;
import java.sql.SQLException;
import java.sql.Statement;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.regex.Matcher;
@ -57,6 +59,20 @@ public class SqlServerE2eITCase extends FlinkContainerTestEnvironment {
TestUtils.getResource("sqlserver-cdc-connector.jar");
private static final Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar");
@Parameterized.Parameter(1)
public boolean parallelismSnapshot;
@Parameterized.Parameters(name = "flinkVersion: {0}, parallelismSnapshot: {1}")
public static List<Object[]> parameters() {
final List<String> flinkVersions = getFlinkVersion();
List<Object[]> params = new ArrayList<>();
for (String flinkVersion : flinkVersions) {
params.add(new Object[] {flinkVersion, true});
params.add(new Object[] {flinkVersion, false});
}
return params;
}
@Rule
public MSSQLServerContainer sqlServer =
new MSSQLServerContainer<>("mcr.microsoft.com/mssql/server:2019-latest")
@ -101,8 +117,9 @@ public class SqlServerE2eITCase extends FlinkContainerTestEnvironment {
" 'username' = '" + sqlServer.getUsername() + "',",
" 'password' = '" + sqlServer.getPassword() + "',",
" 'database-name' = 'inventory',",
" 'schema-name' = 'dbo',",
" 'table-name' = 'products'",
" 'table-name' = 'dbo.products',",
" 'scan.incremental.snapshot.enabled' = '" + parallelismSnapshot + "',",
" 'scan.incremental.snapshot.chunk.size' = '4'",
");",
"CREATE TABLE products_sink (",
" `id` INT NOT NULL,",
@ -126,7 +143,7 @@ public class SqlServerE2eITCase extends FlinkContainerTestEnvironment {
submitSQLJob(sqlLines, sqlServerCdcJar, jdbcJar, mysqlDriverJar);
waitUntilJobRunning(Duration.ofSeconds(30));
// generate binlogs
// generate change stream
try (Connection conn = getSqlServerJdbcConnection();
Statement statement = conn.createStatement()) {
@ -171,7 +188,7 @@ public class SqlServerE2eITCase extends FlinkContainerTestEnvironment {
expectResult,
"products_sink",
new String[] {"id", "name", "description", "weight"},
60000L);
80000L);
}
private void initializeSqlServerTable(String sqlFile) {

@ -30,6 +30,12 @@ under the License.
<dependencies>
<dependency>
<groupId>com.ververica</groupId>
<artifactId>flink-cdc-base</artifactId>
<version>${project.version}</version>
</dependency>
<!-- Debezium dependencies -->
<dependency>
<groupId>com.ververica</groupId>
@ -138,6 +144,7 @@ under the License.
<scope>test</scope>
</dependency>
</dependencies>
</project>

@ -16,7 +16,7 @@
package com.ververica.cdc.connectors.sqlserver;
import com.ververica.cdc.connectors.sqlserver.table.StartupOptions;
import com.ververica.cdc.connectors.base.options.StartupOptions;
import com.ververica.cdc.debezium.DebeziumDeserializationSchema;
import com.ververica.cdc.debezium.DebeziumSourceFunction;
import io.debezium.connector.sqlserver.SqlServerConnector;
@ -137,9 +137,6 @@ public class SqlServerSource {
case INITIAL:
props.setProperty("snapshot.mode", "initial");
break;
case INITIAL_ONLY:
props.setProperty("snapshot.mode", "initial_only");
break;
case LATEST_OFFSET:
props.setProperty("snapshot.mode", "schema_only");
break;

@ -0,0 +1,226 @@
/*
* Copyright 2022 Ververica Inc.
*
* Licensed 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 com.ververica.cdc.connectors.sqlserver.source;
import com.ververica.cdc.connectors.base.options.StartupOptions;
import com.ververica.cdc.connectors.base.source.jdbc.JdbcIncrementalSource;
import com.ververica.cdc.connectors.sqlserver.source.config.SqlServerSourceConfigFactory;
import com.ververica.cdc.connectors.sqlserver.source.dialect.SqlServerDialect;
import com.ververica.cdc.connectors.sqlserver.source.offset.LsnFactory;
import com.ververica.cdc.debezium.DebeziumDeserializationSchema;
import java.time.Duration;
import java.util.Properties;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
* The builder class for {@link SqlServerIncrementalSource} to make it easier for the users to
* construct a {@link SqlServerIncrementalSource}.
*
* <p>Check the Java docs of each individual method to learn more about the settings to build a
* {@link SqlServerIncrementalSource}.
*/
public class SqlServerSourceBuilder<T> {
private final SqlServerSourceConfigFactory configFactory = new SqlServerSourceConfigFactory();
private LsnFactory offsetFactory;
private SqlServerDialect dialect;
private DebeziumDeserializationSchema<T> deserializer;
/** Hostname of the SQL Server database server. */
public SqlServerSourceBuilder<T> hostname(String hostname) {
this.configFactory.hostname(hostname);
return this;
}
/** Integer port number of the SqlServer database server. */
public SqlServerSourceBuilder<T> port(int port) {
this.configFactory.port(port);
return this;
}
/**
* A required list of regular expressions that match database names to be monitored; any
* database name not included in the whitelist will be excluded from monitoring.
*/
public SqlServerSourceBuilder<T> databaseList(String... databaseList) {
this.configFactory.databaseList(databaseList);
return this;
}
/**
* A required list of regular expressions that match fully-qualified table identifiers for
* tables to be monitored; any table not included in the list will be excluded from monitoring.
* Each identifier is of the form {@code <schemaName>.<tableName>}.
*/
public SqlServerSourceBuilder<T> tableList(String... tableList) {
this.configFactory.tableList(tableList);
return this;
}
/** Name of the SqlSever database to use when connecting to the SqlSever database server. */
public SqlServerSourceBuilder<T> username(String username) {
this.configFactory.username(username);
return this;
}
/** Password to use when connecting to the SqlSever database server. */
public SqlServerSourceBuilder<T> password(String password) {
this.configFactory.password(password);
return this;
}
/**
* The session time zone in database server, e.g. "America/Los_Angeles". It controls how the
* TIMESTAMP type in SqlSever converted to STRING. See more
*/
public SqlServerSourceBuilder<T> serverTimeZone(String timeZone) {
this.configFactory.serverTimeZone(timeZone);
return this;
}
/**
* The split size (number of rows) of table snapshot, captured tables are split into multiple
* splits when read the snapshot of table.
*/
public SqlServerSourceBuilder<T> splitSize(int splitSize) {
this.configFactory.splitSize(splitSize);
return this;
}
/**
* The group size of split meta, if the meta size exceeds the group size, the meta will be will
* be divided into multiple groups.
*/
public SqlServerSourceBuilder<T> splitMetaGroupSize(int splitMetaGroupSize) {
this.configFactory.splitMetaGroupSize(splitMetaGroupSize);
return this;
}
/**
* The upper bound of split key evenly distribution factor, the factor is used to determine
* whether the table is evenly distribution or not.
*/
public SqlServerSourceBuilder<T> distributionFactorUpper(double distributionFactorUpper) {
this.configFactory.distributionFactorUpper(distributionFactorUpper);
return this;
}
/**
* The lower bound of split key evenly distribution factor, the factor is used to determine
* whether the table is evenly distribution or not.
*/
public SqlServerSourceBuilder<T> distributionFactorLower(double distributionFactorLower) {
this.configFactory.distributionFactorLower(distributionFactorLower);
return this;
}
/** The maximum fetch size for per poll when read table snapshot. */
public SqlServerSourceBuilder<T> fetchSize(int fetchSize) {
this.configFactory.fetchSize(fetchSize);
return this;
}
/**
* The maximum time that the connector should wait after trying to connect to the SqlSever
* database server before timing out.
*/
public SqlServerSourceBuilder<T> connectTimeout(Duration connectTimeout) {
this.configFactory.connectTimeout(connectTimeout);
return this;
}
/** The max retry times to get connection. */
public SqlServerSourceBuilder<T> connectMaxRetries(int connectMaxRetries) {
this.configFactory.connectMaxRetries(connectMaxRetries);
return this;
}
/** The connection pool size. */
public SqlServerSourceBuilder<T> connectionPoolSize(int connectionPoolSize) {
this.configFactory.connectionPoolSize(connectionPoolSize);
return this;
}
/** Whether the {@link SqlServerIncrementalSource} should output the schema changes or not. */
public SqlServerSourceBuilder<T> includeSchemaChanges(boolean includeSchemaChanges) {
this.configFactory.includeSchemaChanges(includeSchemaChanges);
return this;
}
/** Specifies the startup options. */
public SqlServerSourceBuilder<T> startupOptions(StartupOptions startupOptions) {
this.configFactory.startupOptions(startupOptions);
return this;
}
/**
* The chunk key of table snapshot, captured tables are split into multiple chunks by the chunk
* key column when read the snapshot of table.
*/
public SqlServerSourceBuilder<T> chunkKeyColumn(String chunkKeyColumn) {
this.configFactory.chunkKeyColumn(chunkKeyColumn);
return this;
}
/** The Debezium SqlSever connector properties. For example, "snapshot.mode". */
public SqlServerSourceBuilder<T> debeziumProperties(Properties properties) {
this.configFactory.debeziumProperties(properties);
return this;
}
/**
* The deserializer used to convert from consumed {@link
* org.apache.kafka.connect.source.SourceRecord}.
*/
public SqlServerSourceBuilder<T> deserializer(DebeziumDeserializationSchema<T> deserializer) {
this.deserializer = deserializer;
return this;
}
/**
* Build the {@link SqlServerIncrementalSource}.
*
* @return a SqlSeverParallelSource with the settings made for this builder.
*/
public SqlServerIncrementalSource<T> build() {
this.offsetFactory = new LsnFactory();
this.dialect = new SqlServerDialect(configFactory);
return new SqlServerIncrementalSource<T>(
configFactory, checkNotNull(deserializer), offsetFactory, dialect);
}
/** The {@link JdbcIncrementalSource} implementation for SqlServer. */
public static class SqlServerIncrementalSource<T> extends JdbcIncrementalSource<T> {
public SqlServerIncrementalSource(
SqlServerSourceConfigFactory configFactory,
DebeziumDeserializationSchema<T> deserializationSchema,
LsnFactory offsetFactory,
SqlServerDialect dataSourceDialect) {
super(configFactory, deserializationSchema, offsetFactory, dataSourceDialect);
}
public static <T> SqlServerSourceBuilder<T> builder() {
return new SqlServerSourceBuilder<>();
}
}
}

@ -0,0 +1,93 @@
/*
* Copyright 2022 Ververica Inc.
*
* Licensed 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 com.ververica.cdc.connectors.sqlserver.source.config;
import com.ververica.cdc.connectors.base.config.JdbcSourceConfig;
import com.ververica.cdc.connectors.base.options.StartupOptions;
import io.debezium.config.Configuration;
import io.debezium.connector.sqlserver.SqlServerConnectorConfig;
import io.debezium.relational.RelationalTableFilters;
import java.time.Duration;
import java.util.List;
import java.util.Properties;
/**
* Describes the connection information of the SQL Server database and the configuration information
* for performing snapshotting and streaming reading, such as splitSize.
*/
public class SqlServerSourceConfig extends JdbcSourceConfig {
public SqlServerSourceConfig(
StartupOptions startupOptions,
List<String> databaseList,
List<String> tableList,
int splitSize,
int splitMetaGroupSize,
double distributionFactorUpper,
double distributionFactorLower,
boolean includeSchemaChanges,
Properties dbzProperties,
Configuration dbzConfiguration,
String driverClassName,
String hostname,
int port,
String username,
String password,
int fetchSize,
String serverTimeZone,
Duration connectTimeout,
int connectMaxRetries,
int connectionPoolSize,
String chunkKeyColumn) {
super(
startupOptions,
databaseList,
tableList,
splitSize,
splitMetaGroupSize,
distributionFactorUpper,
distributionFactorLower,
includeSchemaChanges,
dbzProperties,
dbzConfiguration,
driverClassName,
hostname,
port,
username,
password,
fetchSize,
serverTimeZone,
connectTimeout,
connectMaxRetries,
connectionPoolSize,
chunkKeyColumn);
}
@Override
public SqlServerConnectorConfig getDbzConnectorConfig() {
return new SqlServerConnectorConfig(getDbzConfiguration());
}
public Configuration getOriginDbzConnectorConfig() {
return super.getDbzConfiguration();
}
public RelationalTableFilters getTableFilters() {
return getDbzConnectorConfig().getTableFilters();
}
}

@ -0,0 +1,115 @@
/*
* Copyright 2022 Ververica Inc.
*
* Licensed 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 com.ververica.cdc.connectors.sqlserver.source.config;
import com.ververica.cdc.connectors.base.config.JdbcSourceConfigFactory;
import com.ververica.cdc.connectors.base.source.EmbeddedFlinkDatabaseHistory;
import io.debezium.config.Configuration;
import io.debezium.connector.sqlserver.SqlServerConnector;
import java.util.Arrays;
import java.util.List;
import java.util.Properties;
import java.util.UUID;
import static org.apache.flink.util.Preconditions.checkNotNull;
/** Factory for creating {@link SqlServerSourceConfig}. */
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";
private List<String> schemaList;
/**
* An optional list of regular expressions that match schema names to be monitored; any schema
* name not included in the whitelist will be excluded from monitoring. By default, all
* non-system schemas will be monitored.
*/
public JdbcSourceConfigFactory schemaList(String... schemaList) {
this.schemaList = Arrays.asList(schemaList);
return this;
}
@Override
public SqlServerSourceConfig create(int subtask) {
Properties props = new Properties();
props.setProperty("connector.class", SqlServerConnector.class.getCanonicalName());
// set database history impl to flink database history
props.setProperty(
"database.history", EmbeddedFlinkDatabaseHistory.class.getCanonicalName());
props.setProperty("database.history.instance.name", UUID.randomUUID() + "_" + subtask);
// hard code server name, because we don't need to distinguish it, docs:
// Logical name that identifies and provides a namespace for the SQL Server database
// server that you want Debezium to capture. The logical name should be unique across
// all other connectors, since it is used as a prefix for all Kafka topic names
// emanating from this connector. Only alphanumeric characters and underscores should be
// used.
props.setProperty("database.server.name", DATABASE_SERVER_NAME);
props.setProperty("database.hostname", checkNotNull(hostname));
props.setProperty("database.user", checkNotNull(username));
props.setProperty("database.password", checkNotNull(password));
props.setProperty("database.port", String.valueOf(port));
props.setProperty("database.history.skip.unparseable.ddl", String.valueOf(true));
props.setProperty("database.dbname", checkNotNull(databaseList.get(0)));
if (tableList != null) {
props.setProperty("table.include.list", String.join(",", tableList));
}
switch (startupOptions.startupMode) {
case INITIAL:
props.setProperty("snapshot.mode", "initial");
break;
case LATEST_OFFSET:
props.setProperty("snapshot.mode", "schema_only");
break;
default:
throw new UnsupportedOperationException();
}
if (dbzProperties != null) {
props.putAll(dbzProperties);
}
Configuration dbzConfiguration = Configuration.from(props);
return new SqlServerSourceConfig(
startupOptions,
databaseList,
tableList,
splitSize,
splitMetaGroupSize,
distributionFactorUpper,
distributionFactorLower,
includeSchemaChanges,
props,
dbzConfiguration,
DRIVER_ClASS_NAME,
hostname,
port,
username,
password,
fetchSize,
serverTimeZone,
connectTimeout,
connectMaxRetries,
connectionPoolSize,
chunkKeyColumn);
}
}

@ -0,0 +1,361 @@
/*
* Copyright 2022 Ververica Inc.
*
* Licensed 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 com.ververica.cdc.connectors.sqlserver.source.dialect;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.util.FlinkRuntimeException;
import com.ververica.cdc.connectors.base.config.JdbcSourceConfig;
import com.ververica.cdc.connectors.base.dialect.JdbcDataSourceDialect;
import com.ververica.cdc.connectors.base.source.assigner.splitter.ChunkRange;
import com.ververica.cdc.connectors.base.source.assigner.splitter.JdbcSourceChunkSplitter;
import com.ververica.cdc.connectors.base.source.meta.split.SnapshotSplit;
import com.ververica.cdc.connectors.base.utils.ObjectUtils;
import com.ververica.cdc.connectors.sqlserver.source.utils.SqlServerTypeUtils;
import com.ververica.cdc.connectors.sqlserver.source.utils.SqlServerUtils;
import io.debezium.jdbc.JdbcConnection;
import io.debezium.relational.Column;
import io.debezium.relational.Table;
import io.debezium.relational.TableId;
import io.debezium.relational.history.TableChanges.TableChange;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.math.BigDecimal;
import java.math.RoundingMode;
import java.sql.SQLException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import static com.ververica.cdc.connectors.base.utils.ObjectUtils.doubleCompare;
/**
* The {@code ChunkSplitter} used to split SqlServer table into a set of chunks for JDBC data
* source.
*/
public class SqlServerChunkSplitter implements JdbcSourceChunkSplitter {
private static final Logger LOG = LoggerFactory.getLogger(SqlServerChunkSplitter.class);
private final JdbcSourceConfig sourceConfig;
private final JdbcDataSourceDialect dialect;
public SqlServerChunkSplitter(JdbcSourceConfig sourceConfig, JdbcDataSourceDialect dialect) {
this.sourceConfig = sourceConfig;
this.dialect = dialect;
}
private static String splitId(TableId tableId, int chunkId) {
return tableId.toString() + ":" + chunkId;
}
private static void maySleep(int count, TableId tableId) {
// every 100 queries to sleep 1s
if (count % 10 == 0) {
try {
Thread.sleep(100);
} catch (InterruptedException e) {
// nothing to do
}
LOG.info("JdbcSourceChunkSplitter has split {} chunks for table {}", count, tableId);
}
}
@Override
public Collection<SnapshotSplit> generateSplits(TableId tableId) {
try (JdbcConnection jdbc = dialect.openJdbcConnection(sourceConfig)) {
LOG.info("Start splitting table {} into chunks...", tableId);
long start = System.currentTimeMillis();
Table table = dialect.queryTableSchema(jdbc, tableId).getTable();
Column splitColumn = SqlServerUtils.getSplitColumn(table);
final List<ChunkRange> chunks;
try {
chunks = splitTableIntoChunks(jdbc, tableId, splitColumn);
} catch (SQLException e) {
throw new FlinkRuntimeException("Failed to split chunks for table " + tableId, e);
}
// convert chunks into splits
List<SnapshotSplit> splits = new ArrayList<>();
RowType splitType = getSplitType(splitColumn);
for (int i = 0; i < chunks.size(); i++) {
ChunkRange chunk = chunks.get(i);
SnapshotSplit split =
createSnapshotSplit(
jdbc,
tableId,
i,
splitType,
chunk.getChunkStart(),
chunk.getChunkEnd());
splits.add(split);
}
long end = System.currentTimeMillis();
LOG.info(
"Split table {} into {} chunks, time cost: {}ms.",
tableId,
splits.size(),
end - start);
return splits;
} catch (Exception e) {
throw new FlinkRuntimeException(
String.format("Generate Splits for table %s error", tableId), e);
}
}
@Override
public Object[] queryMinMax(JdbcConnection jdbc, TableId tableId, String columnName)
throws SQLException {
return SqlServerUtils.queryMinMax(jdbc, tableId, columnName);
}
@Override
public Object queryMin(
JdbcConnection jdbc, TableId tableId, String columnName, Object excludedLowerBound)
throws SQLException {
return SqlServerUtils.queryMin(jdbc, tableId, columnName, excludedLowerBound);
}
@Override
public DataType fromDbzColumn(Column splitColumn) {
return SqlServerTypeUtils.fromDbzColumn(splitColumn);
}
// --------------------------------------------------------------------------------------------
// Utilities
// --------------------------------------------------------------------------------------------
@Override
public Object queryNextChunkMax(
JdbcConnection jdbc,
TableId tableId,
String columnName,
int chunkSize,
Object includedLowerBound)
throws SQLException {
return SqlServerUtils.queryNextChunkMax(
jdbc, tableId, columnName, chunkSize, includedLowerBound);
}
@Override
public Long queryApproximateRowCnt(JdbcConnection jdbc, TableId tableId) throws SQLException {
return SqlServerUtils.queryApproximateRowCnt(jdbc, tableId);
}
@Override
public String buildSplitScanQuery(
TableId tableId, RowType splitKeyType, boolean isFirstSplit, boolean isLastSplit) {
return SqlServerUtils.buildSplitScanQuery(tableId, splitKeyType, isFirstSplit, isLastSplit);
}
/**
* We can use evenly-sized chunks or unevenly-sized chunks when split table into chunks, using
* evenly-sized chunks which is much efficient, using unevenly-sized chunks which will request
* many queries and is not efficient.
*/
private List<ChunkRange> splitTableIntoChunks(
JdbcConnection jdbc, TableId tableId, Column splitColumn) throws SQLException {
final String splitColumnName = splitColumn.name();
final Object[] minMax = queryMinMax(jdbc, tableId, splitColumnName);
final Object min = minMax[0];
final Object max = minMax[1];
if (min == null || max == null || min.equals(max)) {
// empty table, or only one row, return full table scan as a chunk
return Collections.singletonList(ChunkRange.all());
}
final int chunkSize = sourceConfig.getSplitSize();
final double distributionFactorUpper = sourceConfig.getDistributionFactorUpper();
final double distributionFactorLower = sourceConfig.getDistributionFactorLower();
if (isEvenlySplitColumn(splitColumn)) {
long approximateRowCnt = queryApproximateRowCnt(jdbc, tableId);
double distributionFactor =
calculateDistributionFactor(tableId, min, max, approximateRowCnt);
boolean dataIsEvenlyDistributed =
doubleCompare(distributionFactor, distributionFactorLower) >= 0
&& doubleCompare(distributionFactor, distributionFactorUpper) <= 0;
if (dataIsEvenlyDistributed) {
// the minimum dynamic chunk size is at least 1
final int dynamicChunkSize = Math.max((int) (distributionFactor * chunkSize), 1);
return splitEvenlySizedChunks(
tableId, min, max, approximateRowCnt, chunkSize, dynamicChunkSize);
} else {
return splitUnevenlySizedChunks(
jdbc, tableId, splitColumnName, min, max, chunkSize);
}
} else {
return splitUnevenlySizedChunks(jdbc, tableId, splitColumnName, min, max, chunkSize);
}
}
/**
* Split table into evenly sized chunks based on the numeric min and max value of split column,
* and tumble chunks in step size.
*/
private List<ChunkRange> splitEvenlySizedChunks(
TableId tableId,
Object min,
Object max,
long approximateRowCnt,
int chunkSize,
int dynamicChunkSize) {
LOG.info(
"Use evenly-sized chunk optimization for table {}, the approximate row count is {}, the chunk size is {}, the dynamic chunk size is {}",
tableId,
approximateRowCnt,
chunkSize,
dynamicChunkSize);
if (approximateRowCnt <= chunkSize) {
// there is no more than one chunk, return full table as a chunk
return Collections.singletonList(ChunkRange.all());
}
final List<ChunkRange> splits = new ArrayList<>();
Object chunkStart = null;
Object chunkEnd = ObjectUtils.plus(min, dynamicChunkSize);
while (ObjectUtils.compare(chunkEnd, max) <= 0) {
splits.add(ChunkRange.of(chunkStart, chunkEnd));
chunkStart = chunkEnd;
try {
chunkEnd = ObjectUtils.plus(chunkEnd, dynamicChunkSize);
} catch (ArithmeticException e) {
// Stop chunk split to avoid dead loop when number overflows.
break;
}
}
// add the ending split
splits.add(ChunkRange.of(chunkStart, null));
return splits;
}
// ------------------------------------------------------------------------------------------
/** Split table into unevenly sized chunks by continuously calculating next chunk max value. */
private List<ChunkRange> splitUnevenlySizedChunks(
JdbcConnection jdbc,
TableId tableId,
String splitColumnName,
Object min,
Object max,
int chunkSize)
throws SQLException {
LOG.info(
"Use unevenly-sized chunks for table {}, the chunk size is {}", tableId, chunkSize);
final List<ChunkRange> splits = new ArrayList<>();
Object chunkStart = null;
Object chunkEnd = nextChunkEnd(jdbc, min, tableId, splitColumnName, max, chunkSize);
int count = 0;
while (chunkEnd != null && ObjectUtils.compare(chunkEnd, max) <= 0) {
// we start from [null, min + chunk_size) and avoid [null, min)
splits.add(ChunkRange.of(chunkStart, chunkEnd));
// may sleep a while to avoid DDOS on SqlServer server
maySleep(count++, tableId);
chunkStart = chunkEnd;
chunkEnd = nextChunkEnd(jdbc, chunkEnd, tableId, splitColumnName, max, chunkSize);
}
// add the ending split
splits.add(ChunkRange.of(chunkStart, null));
return splits;
}
private Object nextChunkEnd(
JdbcConnection jdbc,
Object previousChunkEnd,
TableId tableId,
String splitColumnName,
Object max,
int chunkSize)
throws SQLException {
// chunk end might be null when max values are removed
Object chunkEnd =
queryNextChunkMax(jdbc, tableId, splitColumnName, chunkSize, previousChunkEnd);
if (Objects.equals(previousChunkEnd, chunkEnd)) {
// we don't allow equal chunk start and end,
// should query the next one larger than chunkEnd
chunkEnd = queryMin(jdbc, tableId, splitColumnName, chunkEnd);
}
if (ObjectUtils.compare(chunkEnd, max) >= 0) {
return null;
} else {
return chunkEnd;
}
}
private SnapshotSplit createSnapshotSplit(
JdbcConnection jdbc,
TableId tableId,
int chunkId,
RowType splitKeyType,
Object chunkStart,
Object chunkEnd) {
// currently, we only support single split column
Object[] splitStart = chunkStart == null ? null : new Object[] {chunkStart};
Object[] splitEnd = chunkEnd == null ? null : new Object[] {chunkEnd};
Map<TableId, TableChange> schema = new HashMap<>();
schema.put(tableId, dialect.queryTableSchema(jdbc, tableId));
return new SnapshotSplit(
tableId,
splitId(tableId, chunkId),
splitKeyType,
splitStart,
splitEnd,
null,
schema);
}
/** Returns the distribution factor of the given table. */
private double calculateDistributionFactor(
TableId tableId, Object min, Object max, long approximateRowCnt) {
if (!min.getClass().equals(max.getClass())) {
throw new IllegalStateException(
String.format(
"Unsupported operation type, the MIN value type %s is different with MAX value type %s.",
min.getClass().getSimpleName(), max.getClass().getSimpleName()));
}
if (approximateRowCnt == 0) {
return Double.MAX_VALUE;
}
BigDecimal difference = ObjectUtils.minus(max, min);
// factor = (max - min + 1) / rowCount
final BigDecimal subRowCnt = difference.add(BigDecimal.valueOf(1));
double distributionFactor =
subRowCnt
.divide(new BigDecimal(approximateRowCnt), 4, RoundingMode.CEILING)
.doubleValue();
LOG.info(
"The distribution factor of table {} is {} according to the min split key {}, max split key {} and approximate row count {}",
tableId,
distributionFactor,
min,
max,
approximateRowCnt);
return distributionFactor;
}
}

@ -0,0 +1,156 @@
/*
* 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 com.ververica.cdc.connectors.sqlserver.source.dialect;
import org.apache.flink.annotation.Experimental;
import org.apache.flink.util.FlinkRuntimeException;
import com.ververica.cdc.connectors.base.config.JdbcSourceConfig;
import com.ververica.cdc.connectors.base.dialect.JdbcDataSourceDialect;
import com.ververica.cdc.connectors.base.relational.connection.JdbcConnectionPoolFactory;
import com.ververica.cdc.connectors.base.source.assigner.splitter.ChunkSplitter;
import com.ververica.cdc.connectors.base.source.meta.offset.Offset;
import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase;
import com.ververica.cdc.connectors.base.source.reader.external.FetchTask;
import com.ververica.cdc.connectors.sqlserver.source.config.SqlServerSourceConfig;
import com.ververica.cdc.connectors.sqlserver.source.config.SqlServerSourceConfigFactory;
import com.ververica.cdc.connectors.sqlserver.source.reader.fetch.SqlServerScanFetchTask;
import com.ververica.cdc.connectors.sqlserver.source.reader.fetch.SqlServerSourceFetchTaskContext;
import com.ververica.cdc.connectors.sqlserver.source.reader.fetch.SqlServerStreamFetchTask;
import com.ververica.cdc.connectors.sqlserver.source.utils.SqlServerConnectionUtils;
import io.debezium.connector.sqlserver.SqlServerConnection;
import io.debezium.jdbc.JdbcConnection;
import io.debezium.relational.TableId;
import io.debezium.relational.history.TableChanges.TableChange;
import java.sql.SQLException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static com.ververica.cdc.connectors.sqlserver.source.utils.SqlServerConnectionUtils.createSqlServerConnection;
import static com.ververica.cdc.connectors.sqlserver.source.utils.SqlServerUtils.currentLsn;
/** The {@link JdbcDataSourceDialect} implementation for SqlServer datasource. */
@Experimental
public class SqlServerDialect implements JdbcDataSourceDialect {
private static final long serialVersionUID = 1L;
private final SqlServerSourceConfigFactory configFactory;
private final SqlServerSourceConfig sourceConfig;
private transient SqlServerSchema sqlserverSchema;
public SqlServerDialect(SqlServerSourceConfigFactory configFactory) {
this.configFactory = configFactory;
this.sourceConfig = configFactory.create(0);
}
@Override
public String getName() {
return "SqlServer";
}
@Override
public Offset displayCurrentOffset(JdbcSourceConfig sourceConfig) {
try (JdbcConnection jdbcConnection = openJdbcConnection(sourceConfig)) {
return currentLsn((SqlServerConnection) jdbcConnection);
} catch (Exception e) {
throw new FlinkRuntimeException("Read the redoLog offset error", e);
}
}
@Override
public boolean isDataCollectionIdCaseSensitive(JdbcSourceConfig sourceConfig) {
// todo: need to check the case sensitive of the database
return true;
}
@Override
public JdbcConnection openJdbcConnection(JdbcSourceConfig sourceConfig) {
return createSqlServerConnection(sourceConfig.getDbzConnectorConfig().getJdbcConfig());
}
@Override
public ChunkSplitter createChunkSplitter(JdbcSourceConfig sourceConfig) {
return new SqlServerChunkSplitter(sourceConfig, this);
}
@Override
public JdbcConnectionPoolFactory getPooledDataSourceFactory() {
return new SqlServerPooledDataSourceFactory();
}
@Override
public List<TableId> discoverDataCollections(JdbcSourceConfig sourceConfig) {
SqlServerSourceConfig sqlserverSourceConfig = (SqlServerSourceConfig) sourceConfig;
try (JdbcConnection jdbcConnection = openJdbcConnection(sourceConfig)) {
return SqlServerConnectionUtils.listTables(
jdbcConnection, sqlserverSourceConfig.getTableFilters());
} catch (SQLException e) {
throw new FlinkRuntimeException("Error to discover tables: " + e.getMessage(), e);
}
}
@Override
public Map<TableId, TableChange> discoverDataCollectionSchemas(JdbcSourceConfig sourceConfig) {
final List<TableId> capturedTableIds = discoverDataCollections(sourceConfig);
try (SqlServerConnection jdbc =
createSqlServerConnection(sourceConfig.getDbzConnectorConfig().getJdbcConfig())) {
// fetch table schemas
Map<TableId, TableChange> tableSchemas = new HashMap<>();
for (TableId tableId : capturedTableIds) {
TableChange tableSchema = queryTableSchema(jdbc, tableId);
tableSchemas.put(tableId, tableSchema);
}
return tableSchemas;
} catch (Exception e) {
throw new FlinkRuntimeException(
"Error to discover table schemas: " + e.getMessage(), e);
}
}
@Override
public TableChange queryTableSchema(JdbcConnection jdbc, TableId tableId) {
if (sqlserverSchema == null) {
sqlserverSchema = new SqlServerSchema();
}
return sqlserverSchema.getTableSchema(jdbc, tableId);
}
@Override
public SqlServerSourceFetchTaskContext createFetchTaskContext(
SourceSplitBase sourceSplitBase, JdbcSourceConfig taskSourceConfig) {
final SqlServerConnection jdbcConnection =
createSqlServerConnection(sourceConfig.getDbzConnectorConfig().getJdbcConfig());
final SqlServerConnection metaDataConnection =
createSqlServerConnection(sourceConfig.getDbzConnectorConfig().getJdbcConfig());
return new SqlServerSourceFetchTaskContext(
taskSourceConfig, this, jdbcConnection, metaDataConnection);
}
@Override
public FetchTask<SourceSplitBase> createFetchTask(SourceSplitBase sourceSplitBase) {
if (sourceSplitBase.isSnapshotSplit()) {
return new SqlServerScanFetchTask(sourceSplitBase.asSnapshotSplit());
} else {
return new SqlServerStreamFetchTask(sourceSplitBase.asStreamSplit());
}
}
}

@ -0,0 +1,34 @@
/*
* Copyright 2022 Ververica Inc.
*
* Licensed 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 com.ververica.cdc.connectors.sqlserver.source.dialect;
import com.ververica.cdc.connectors.base.config.JdbcSourceConfig;
import com.ververica.cdc.connectors.base.relational.connection.JdbcConnectionPoolFactory;
/** Factory to create {@link JdbcConnectionPoolFactory} for SQL Server. */
public class SqlServerPooledDataSourceFactory extends JdbcConnectionPoolFactory {
private static final String URL_PATTERN = "jdbc:sqlserver://%s:%s;databaseName=%s";
@Override
public String getJdbcUrl(JdbcSourceConfig sourceConfig) {
String hostName = sourceConfig.getHostname();
int port = sourceConfig.getPort();
String database = sourceConfig.getDatabaseList().get(0);
return String.format(URL_PATTERN, hostName, port, database);
}
}

@ -0,0 +1,82 @@
/*
* Copyright 2022 Ververica Inc.
*
* Licensed 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 com.ververica.cdc.connectors.sqlserver.source.dialect;
import org.apache.flink.util.FlinkRuntimeException;
import io.debezium.connector.sqlserver.SqlServerConnection;
import io.debezium.jdbc.JdbcConnection;
import io.debezium.relational.Table;
import io.debezium.relational.TableId;
import io.debezium.relational.Tables;
import io.debezium.relational.history.TableChanges;
import io.debezium.relational.history.TableChanges.TableChange;
import java.sql.SQLException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
/** A component used to get schema by table path. */
public class SqlServerSchema {
private final Map<TableId, TableChange> schemasByTableId;
public SqlServerSchema() {
this.schemasByTableId = new ConcurrentHashMap<>();
}
public TableChange getTableSchema(JdbcConnection jdbc, TableId tableId) {
// read schema from cache first
TableChange schema = schemasByTableId.get(tableId);
if (schema == null) {
schema = readTableSchema(jdbc, tableId);
schemasByTableId.put(tableId, schema);
}
return schema;
}
private TableChange readTableSchema(JdbcConnection jdbc, TableId tableId) {
SqlServerConnection sqlServerConnection = (SqlServerConnection) jdbc;
Set<TableId> tableIdSet = new HashSet<>();
tableIdSet.add(tableId);
final Map<TableId, TableChange> tableChangeMap = new HashMap<>();
Tables tables = new Tables();
tables.overwriteTable(tables.editOrCreateTable(tableId).create());
try {
sqlServerConnection.readSchema(
tables, tableId.catalog(), tableId.schema(), null, null, false);
Table table = tables.forTable(tableId);
TableChange tableChange = new TableChange(TableChanges.TableChangeType.CREATE, table);
tableChangeMap.put(tableId, tableChange);
} catch (SQLException e) {
throw new FlinkRuntimeException(
String.format("Failed to read schema for table %s ", tableId), e);
}
if (!tableChangeMap.containsKey(tableId)) {
throw new FlinkRuntimeException(
String.format("Can't obtain schema for table %s ", tableId));
}
return tableChangeMap.get(tableId);
}
}

@ -0,0 +1,54 @@
/*
* Copyright 2022 Ververica Inc.
*
* Licensed 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 com.ververica.cdc.connectors.sqlserver.source.offset;
import com.ververica.cdc.connectors.base.source.meta.offset.Offset;
import com.ververica.cdc.connectors.base.source.meta.offset.OffsetFactory;
import io.debezium.connector.sqlserver.Lsn;
import io.debezium.connector.sqlserver.SourceInfo;
import java.util.Map;
/** A factory to create {@link LsnOffset}. */
public class LsnFactory extends OffsetFactory {
@Override
public Offset newOffset(Map<String, String> offset) {
return new LsnOffset(Lsn.valueOf(offset.get(SourceInfo.CHANGE_LSN_KEY)));
}
@Override
public Offset newOffset(String filename, Long position) {
throw new UnsupportedOperationException(
"not supported create new Offset by filename and position.");
}
@Override
public Offset newOffset(Long position) {
throw new UnsupportedOperationException(
"not supported create new Offset by filename and position.");
}
@Override
public Offset createInitialOffset() {
return LsnOffset.INITIAL_OFFSET;
}
@Override
public Offset createNoStoppingOffset() {
return LsnOffset.NO_STOPPING_OFFSET;
}
}

@ -0,0 +1,96 @@
/*
* Copyright 2022 Ververica Inc.
*
* Licensed 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 com.ververica.cdc.connectors.sqlserver.source.offset;
import com.ververica.cdc.connectors.base.source.meta.offset.Offset;
import io.debezium.connector.sqlserver.Lsn;
import io.debezium.connector.sqlserver.SourceInfo;
import java.util.HashMap;
import java.util.Map;
/** A structure describes an offset in a Lsn event. */
public class LsnOffset extends Offset {
public static final LsnOffset INITIAL_OFFSET =
new LsnOffset(Lsn.valueOf(new byte[] {Byte.MIN_VALUE}));
public static final LsnOffset NO_STOPPING_OFFSET =
new LsnOffset(Lsn.valueOf(new byte[] {Byte.MAX_VALUE}));
public LsnOffset(Lsn scn, Lsn commitScn, Long eventSerialNo) {
Map<String, String> offsetMap = new HashMap<>();
if (scn != null && scn.isAvailable()) {
offsetMap.put(SourceInfo.CHANGE_LSN_KEY, scn.toString());
}
if (commitScn != null && commitScn.isAvailable()) {
offsetMap.put(SourceInfo.COMMIT_LSN_KEY, commitScn.toString());
}
if (eventSerialNo != null) {
offsetMap.put(SourceInfo.EVENT_SERIAL_NO_KEY, String.valueOf(eventSerialNo));
}
this.offset = offsetMap;
}
public LsnOffset(Lsn lsn) {
this(lsn, null, null);
}
public Lsn getLcn() {
return Lsn.valueOf(offset.get(SourceInfo.CHANGE_LSN_KEY));
}
@Override
public int compareTo(Offset offset) {
LsnOffset that = (LsnOffset) offset;
// the NO_STOPPING_OFFSET is the max offset
if (NO_STOPPING_OFFSET.equals(that) && NO_STOPPING_OFFSET.equals(this)) {
return 0;
}
if (NO_STOPPING_OFFSET.equals(this)) {
return 1;
}
if (NO_STOPPING_OFFSET.equals(that)) {
return -1;
}
Lsn lsn = this.getLcn();
Lsn targetLsn = that.getLcn();
if (targetLsn.isAvailable()) {
if (lsn.isAvailable()) {
return lsn.compareTo(targetLsn);
}
return -1;
} else if (lsn.isAvailable()) {
return 1;
}
return 0;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (!(o instanceof LsnOffset)) {
return false;
}
LsnOffset that = (LsnOffset) o;
return offset.equals(that.offset);
}
}

@ -0,0 +1,459 @@
/*
* Copyright 2022 Ververica Inc.
*
* Licensed 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 com.ververica.cdc.connectors.sqlserver.source.reader.fetch;
import com.ververica.cdc.connectors.base.relational.JdbcSourceEventDispatcher;
import com.ververica.cdc.connectors.base.source.meta.split.SnapshotSplit;
import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase;
import com.ververica.cdc.connectors.base.source.meta.split.StreamSplit;
import com.ververica.cdc.connectors.base.source.meta.wartermark.WatermarkKind;
import com.ververica.cdc.connectors.base.source.reader.external.FetchTask;
import com.ververica.cdc.connectors.sqlserver.source.offset.LsnOffset;
import com.ververica.cdc.connectors.sqlserver.source.reader.fetch.SqlServerStreamFetchTask.LsnSplitReadTask;
import io.debezium.DebeziumException;
import io.debezium.config.Configuration;
import io.debezium.connector.sqlserver.SqlServerConnection;
import io.debezium.connector.sqlserver.SqlServerConnectorConfig;
import io.debezium.connector.sqlserver.SqlServerDatabaseSchema;
import io.debezium.connector.sqlserver.SqlServerOffsetContext;
import io.debezium.heartbeat.Heartbeat;
import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.source.AbstractSnapshotChangeEventSource;
import io.debezium.pipeline.source.spi.ChangeEventSource;
import io.debezium.pipeline.source.spi.SnapshotProgressListener;
import io.debezium.pipeline.spi.ChangeRecordEmitter;
import io.debezium.pipeline.spi.OffsetContext;
import io.debezium.pipeline.spi.SnapshotResult;
import io.debezium.relational.Column;
import io.debezium.relational.RelationalSnapshotChangeEventSource;
import io.debezium.relational.SnapshotChangeRecordEmitter;
import io.debezium.relational.Table;
import io.debezium.relational.TableId;
import io.debezium.util.Clock;
import io.debezium.util.ColumnUtils;
import io.debezium.util.Strings;
import io.debezium.util.Threads;
import org.apache.kafka.connect.errors.ConnectException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.sql.PreparedStatement;
import java.sql.ResultSet;
import java.sql.ResultSetMetaData;
import java.sql.SQLException;
import java.sql.Types;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Map;
import static com.ververica.cdc.connectors.sqlserver.source.utils.SqlServerConnectionUtils.createSqlServerConnection;
import static com.ververica.cdc.connectors.sqlserver.source.utils.SqlServerUtils.buildSplitScanQuery;
import static com.ververica.cdc.connectors.sqlserver.source.utils.SqlServerUtils.currentLsn;
import static com.ververica.cdc.connectors.sqlserver.source.utils.SqlServerUtils.readTableSplitDataStatement;
/** The task to work for fetching data of SqlServer table snapshot split. */
public class SqlServerScanFetchTask implements FetchTask<SourceSplitBase> {
private final SnapshotSplit split;
private volatile boolean taskRunning = false;
private SqlServerSnapshotSplitReadTask snapshotSplitReadTask;
public SqlServerScanFetchTask(SnapshotSplit split) {
this.split = split;
}
@Override
public void execute(Context context) throws Exception {
SqlServerSourceFetchTaskContext sourceFetchContext =
(SqlServerSourceFetchTaskContext) context;
taskRunning = true;
snapshotSplitReadTask =
new SqlServerSnapshotSplitReadTask(
sourceFetchContext.getDbzConnectorConfig(),
sourceFetchContext.getOffsetContext(),
sourceFetchContext.getSnapshotChangeEventSourceMetrics(),
sourceFetchContext.getDatabaseSchema(),
sourceFetchContext.getConnection(),
sourceFetchContext.getDispatcher(),
split);
SnapshotSplitChangeEventSourceContext changeEventSourceContext =
new SnapshotSplitChangeEventSourceContext();
SnapshotResult snapshotResult =
snapshotSplitReadTask.execute(
changeEventSourceContext, sourceFetchContext.getOffsetContext());
final StreamSplit backfillBinlogSplit = createBackFillLsnSplit(changeEventSourceContext);
// optimization that skip the binlog read when the low watermark equals high
// watermark
final boolean binlogBackfillRequired =
backfillBinlogSplit
.getEndingOffset()
.isAfter(backfillBinlogSplit.getStartingOffset());
if (!binlogBackfillRequired) {
dispatchLsnEndEvent(
backfillBinlogSplit,
((SqlServerSourceFetchTaskContext) context).getOffsetContext().getPartition(),
((SqlServerSourceFetchTaskContext) context).getDispatcher());
taskRunning = false;
return;
}
// execute stream read task
if (snapshotResult.isCompletedOrSkipped()) {
final LsnSplitReadTask backfillBinlogReadTask =
createBackFillLsnSplitReadTask(backfillBinlogSplit, sourceFetchContext);
backfillBinlogReadTask.execute(
new SnapshotBinlogSplitChangeEventSourceContext(),
sourceFetchContext.getOffsetContext());
} else {
taskRunning = false;
throw new IllegalStateException(
String.format("Read snapshot for SqlServer split %s fail", split));
}
}
private StreamSplit createBackFillLsnSplit(
SnapshotSplitChangeEventSourceContext sourceContext) {
return new StreamSplit(
split.splitId(),
sourceContext.getLowWatermark(),
sourceContext.getHighWatermark(),
new ArrayList<>(),
split.getTableSchemas(),
0);
}
private void dispatchLsnEndEvent(
StreamSplit backFillBinlogSplit,
Map<String, ?> sourcePartition,
JdbcSourceEventDispatcher eventDispatcher)
throws InterruptedException {
eventDispatcher.dispatchWatermarkEvent(
sourcePartition,
backFillBinlogSplit,
backFillBinlogSplit.getEndingOffset(),
WatermarkKind.END);
}
private LsnSplitReadTask createBackFillLsnSplitReadTask(
StreamSplit backfillBinlogSplit, SqlServerSourceFetchTaskContext context) {
// we should only capture events for the current table,
// otherwise, we may can't find corresponding schema
Configuration dezConf =
context.getSourceConfig()
.getDbzConfiguration()
.edit()
.with("table.include.list", split.getTableId().toString())
// Disable heartbeat event in snapshot split fetcher
.with(Heartbeat.HEARTBEAT_INTERVAL, 0)
.build();
// task to read binlog and backfill for current split
return new LsnSplitReadTask(
new SqlServerConnectorConfig(dezConf),
createSqlServerConnection(context.getSourceConfig().getDbzConfiguration()),
context.getMetaDataConnection(),
context.getDispatcher(),
context.getErrorHandler(),
context.getDatabaseSchema(),
backfillBinlogSplit);
}
@Override
public boolean isRunning() {
return taskRunning;
}
@Override
public SourceSplitBase getSplit() {
return split;
}
/** A wrapped task to fetch snapshot split of table. */
public static class SqlServerSnapshotSplitReadTask extends AbstractSnapshotChangeEventSource {
private static final Logger LOG =
LoggerFactory.getLogger(SqlServerSnapshotSplitReadTask.class);
/** Interval for showing a log statement with the progress while scanning a single table. */
private static final Duration LOG_INTERVAL = Duration.ofMillis(10_000);
private final SqlServerConnectorConfig connectorConfig;
private final SqlServerDatabaseSchema databaseSchema;
private final SqlServerConnection jdbcConnection;
private final JdbcSourceEventDispatcher dispatcher;
private final Clock clock;
private final SnapshotSplit snapshotSplit;
private final SqlServerOffsetContext offsetContext;
private final SnapshotProgressListener snapshotProgressListener;
public SqlServerSnapshotSplitReadTask(
SqlServerConnectorConfig connectorConfig,
SqlServerOffsetContext previousOffset,
SnapshotProgressListener snapshotProgressListener,
SqlServerDatabaseSchema databaseSchema,
SqlServerConnection jdbcConnection,
JdbcSourceEventDispatcher dispatcher,
SnapshotSplit snapshotSplit) {
super(connectorConfig, snapshotProgressListener);
this.offsetContext = previousOffset;
this.connectorConfig = connectorConfig;
this.databaseSchema = databaseSchema;
this.jdbcConnection = jdbcConnection;
this.dispatcher = dispatcher;
this.clock = Clock.SYSTEM;
this.snapshotSplit = snapshotSplit;
this.snapshotProgressListener = snapshotProgressListener;
}
@Override
public SnapshotResult execute(
ChangeEventSourceContext context, OffsetContext previousOffset)
throws InterruptedException {
SnapshottingTask snapshottingTask = getSnapshottingTask(previousOffset);
final SnapshotContext ctx;
try {
ctx = prepare(context);
} catch (Exception e) {
LOG.error("Failed to initialize snapshot context.", e);
throw new RuntimeException(e);
}
try {
return doExecute(context, previousOffset, ctx, snapshottingTask);
} catch (InterruptedException e) {
LOG.warn("Snapshot was interrupted before completion");
throw e;
} catch (Exception t) {
throw new DebeziumException(t);
}
}
@Override
protected SnapshotResult doExecute(
ChangeEventSourceContext context,
OffsetContext previousOffset,
SnapshotContext snapshotContext,
SnapshottingTask snapshottingTask)
throws Exception {
final RelationalSnapshotChangeEventSource.RelationalSnapshotContext ctx =
(RelationalSnapshotChangeEventSource.RelationalSnapshotContext) snapshotContext;
ctx.offset = offsetContext;
final LsnOffset lowWatermark = currentLsn(jdbcConnection);
LOG.info(
"Snapshot step 1 - Determining low watermark {} for split {}",
lowWatermark,
snapshotSplit);
((SnapshotSplitChangeEventSourceContext) (context)).setLowWatermark(lowWatermark);
dispatcher.dispatchWatermarkEvent(
offsetContext.getPartition(), snapshotSplit, lowWatermark, WatermarkKind.LOW);
LOG.info("Snapshot step 2 - Snapshotting data");
createDataEvents(ctx, snapshotSplit.getTableId());
final LsnOffset highWatermark = currentLsn(jdbcConnection);
LOG.info(
"Snapshot step 3 - Determining high watermark {} for split {}",
highWatermark,
snapshotSplit);
((SnapshotSplitChangeEventSourceContext) (context)).setHighWatermark(lowWatermark);
dispatcher.dispatchWatermarkEvent(
offsetContext.getPartition(), snapshotSplit, highWatermark, WatermarkKind.HIGH);
return SnapshotResult.completed(ctx.offset);
}
@Override
protected SnapshottingTask getSnapshottingTask(OffsetContext previousOffset) {
return new SnapshottingTask(false, true);
}
@Override
protected SnapshotContext prepare(ChangeEventSourceContext changeEventSourceContext)
throws Exception {
return new SqlSeverSnapshotContext();
}
private void createDataEvents(
RelationalSnapshotChangeEventSource.RelationalSnapshotContext snapshotContext,
TableId tableId)
throws Exception {
EventDispatcher.SnapshotReceiver snapshotReceiver =
dispatcher.getSnapshotChangeEventReceiver();
LOG.debug("Snapshotting table {}", tableId);
createDataEventsForTable(
snapshotContext, snapshotReceiver, databaseSchema.tableFor(tableId));
snapshotReceiver.completeSnapshot();
}
/** Dispatches the data change events for the records of a single table. */
private void createDataEventsForTable(
RelationalSnapshotChangeEventSource.RelationalSnapshotContext snapshotContext,
EventDispatcher.SnapshotReceiver snapshotReceiver,
Table table)
throws InterruptedException {
long exportStart = clock.currentTimeInMillis();
LOG.info(
"Exporting data from split '{}' of table {}",
snapshotSplit.splitId(),
table.id());
final String selectSql =
buildSplitScanQuery(
snapshotSplit.getTableId(),
snapshotSplit.getSplitKeyType(),
snapshotSplit.getSplitStart() == null,
snapshotSplit.getSplitEnd() == null);
LOG.info(
"For split '{}' of table {} using select statement: '{}'",
snapshotSplit.splitId(),
table.id(),
selectSql);
try (PreparedStatement selectStatement =
readTableSplitDataStatement(
jdbcConnection,
selectSql,
snapshotSplit.getSplitStart() == null,
snapshotSplit.getSplitEnd() == null,
snapshotSplit.getSplitStart(),
snapshotSplit.getSplitEnd(),
snapshotSplit.getSplitKeyType().getFieldCount(),
connectorConfig.getQueryFetchSize());
ResultSet rs = selectStatement.executeQuery()) {
ColumnUtils.ColumnArray columnArray = ColumnUtils.toArray(rs, table);
long rows = 0;
Threads.Timer logTimer = getTableScanLogTimer();
while (rs.next()) {
rows++;
final Object[] row = new Object[columnArray.getGreatestColumnPosition()];
for (int i = 0; i < columnArray.getColumns().length; i++) {
Column actualColumn = table.columns().get(i);
row[columnArray.getColumns()[i].position() - 1] =
readField(rs, i + 1, actualColumn, table);
}
if (logTimer.expired()) {
long stop = clock.currentTimeInMillis();
LOG.info(
"Exported {} records for split '{}' after {}",
rows,
snapshotSplit.splitId(),
Strings.duration(stop - exportStart));
snapshotProgressListener.rowsScanned(table.id(), rows);
logTimer = getTableScanLogTimer();
}
dispatcher.dispatchSnapshotEvent(
table.id(),
getChangeRecordEmitter(snapshotContext, table.id(), row),
snapshotReceiver);
}
LOG.info(
"Finished exporting {} records for split '{}', total duration '{}'",
rows,
snapshotSplit.splitId(),
Strings.duration(clock.currentTimeInMillis() - exportStart));
} catch (SQLException e) {
throw new ConnectException("Snapshotting of table " + table.id() + " failed", e);
}
}
protected ChangeRecordEmitter getChangeRecordEmitter(
SnapshotContext snapshotContext, TableId tableId, Object[] row) {
snapshotContext.offset.event(tableId, clock.currentTime());
return new SnapshotChangeRecordEmitter(snapshotContext.offset, row, clock);
}
private Threads.Timer getTableScanLogTimer() {
return Threads.timer(clock, LOG_INTERVAL);
}
/**
* copied from
* io.debezium.connector.SqlServer.antlr.listener.ParserUtils#convertValueToSchemaType.
*/
private Object readField(
ResultSet rs, int columnIndex, Column actualColumn, Table actualTable)
throws SQLException {
final ResultSetMetaData metaData = rs.getMetaData();
final int columnType = metaData.getColumnType(columnIndex);
if (columnType == Types.TIME) {
return rs.getTimestamp(columnIndex);
} else {
return rs.getObject(columnIndex);
}
}
private static class SqlSeverSnapshotContext
extends RelationalSnapshotChangeEventSource.RelationalSnapshotContext {
public SqlSeverSnapshotContext() throws SQLException {
super("");
}
}
}
/**
* {@link ChangeEventSource.ChangeEventSourceContext} implementation that keeps low/high
* watermark for each {@link SnapshotSplit}.
*/
public class SnapshotSplitChangeEventSourceContext
implements ChangeEventSource.ChangeEventSourceContext {
private LsnOffset lowWatermark;
private LsnOffset highWatermark;
public LsnOffset getLowWatermark() {
return lowWatermark;
}
public void setLowWatermark(LsnOffset lowWatermark) {
this.lowWatermark = lowWatermark;
}
public LsnOffset getHighWatermark() {
return highWatermark;
}
public void setHighWatermark(LsnOffset highWatermark) {
this.highWatermark = highWatermark;
}
@Override
public boolean isRunning() {
return lowWatermark != null && highWatermark != null;
}
}
/**
* The {@link ChangeEventSource.ChangeEventSourceContext} implementation for bounded stream task
* of a snapshot split task.
*/
public class SnapshotBinlogSplitChangeEventSourceContext
implements ChangeEventSource.ChangeEventSourceContext {
public void finished() {
taskRunning = false;
}
@Override
public boolean isRunning() {
return taskRunning;
}
}
}

@ -0,0 +1,301 @@
/*
* Copyright 2022 Ververica Inc.
*
* Licensed 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 com.ververica.cdc.connectors.sqlserver.source.reader.fetch;
import org.apache.flink.table.types.logical.RowType;
import com.ververica.cdc.connectors.base.config.JdbcSourceConfig;
import com.ververica.cdc.connectors.base.relational.JdbcSourceEventDispatcher;
import com.ververica.cdc.connectors.base.source.EmbeddedFlinkDatabaseHistory;
import com.ververica.cdc.connectors.base.source.meta.offset.Offset;
import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase;
import com.ververica.cdc.connectors.base.source.reader.external.JdbcSourceFetchTaskContext;
import com.ververica.cdc.connectors.base.utils.SourceRecordUtils;
import com.ververica.cdc.connectors.sqlserver.source.config.SqlServerSourceConfig;
import com.ververica.cdc.connectors.sqlserver.source.dialect.SqlServerDialect;
import com.ververica.cdc.connectors.sqlserver.source.offset.LsnOffset;
import com.ververica.cdc.connectors.sqlserver.source.utils.SqlServerUtils;
import io.debezium.connector.base.ChangeEventQueue;
import io.debezium.connector.base.ChangeEventQueue.Builder;
import io.debezium.connector.sqlserver.SourceInfo;
import io.debezium.connector.sqlserver.SqlServerConnection;
import io.debezium.connector.sqlserver.SqlServerConnectorConfig;
import io.debezium.connector.sqlserver.SqlServerDatabaseSchema;
import io.debezium.connector.sqlserver.SqlServerErrorHandler;
import io.debezium.connector.sqlserver.SqlServerOffsetContext;
import io.debezium.connector.sqlserver.SqlServerOffsetContext.Loader;
import io.debezium.connector.sqlserver.SqlServerTaskContext;
import io.debezium.connector.sqlserver.SqlServerTopicSelector;
import io.debezium.data.Envelope.FieldName;
import io.debezium.pipeline.DataChangeEvent;
import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.metrics.DefaultChangeEventSourceMetricsFactory;
import io.debezium.pipeline.metrics.SnapshotChangeEventSourceMetrics;
import io.debezium.pipeline.metrics.StreamingChangeEventSourceMetrics;
import io.debezium.pipeline.source.spi.EventMetadataProvider;
import io.debezium.pipeline.spi.OffsetContext;
import io.debezium.relational.Table;
import io.debezium.relational.TableId;
import io.debezium.relational.Tables.TableFilter;
import io.debezium.schema.DataCollectionId;
import io.debezium.schema.TopicSelector;
import io.debezium.util.Collect;
import io.debezium.util.SchemaNameAdjuster;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.source.SourceRecord;
import java.time.Instant;
import java.util.Map;
/** The context for fetch task that fetching data of snapshot split from SqlServer data source. */
public class SqlServerSourceFetchTaskContext extends JdbcSourceFetchTaskContext {
/** Connection used for reading CDC tables. */
private final SqlServerConnection connection;
/**
* A separate connection for retrieving details of the schema changes; without it, adaptive
* buffering will not work.
*
* @link
* https://docs.microsoft.com/en-us/sql/connect/jdbc/using-adaptive-buffering?view=sql-server-2017#guidelines-for-using-adaptive-buffering
*/
private final SqlServerConnection metaDataConnection;
private final SqlServerEventMetadataProvider metadataProvider;
private SqlServerOffsetContext offsetContext;
private SqlServerDatabaseSchema databaseSchema;
private JdbcSourceEventDispatcher dispatcher;
private SqlServerErrorHandler errorHandler;
private ChangeEventQueue<DataChangeEvent> queue;
private SqlServerTaskContext taskContext;
private TopicSelector<TableId> topicSelector;
private SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics;
private StreamingChangeEventSourceMetrics streamingChangeEventSourceMetrics;
public SqlServerSourceFetchTaskContext(
JdbcSourceConfig sourceConfig,
SqlServerDialect dataSourceDialect,
SqlServerConnection connection,
SqlServerConnection metaDataConnection) {
super(sourceConfig, dataSourceDialect);
this.connection = connection;
this.metadataProvider = new SqlServerEventMetadataProvider();
this.metaDataConnection = metaDataConnection;
}
@Override
public void configure(SourceSplitBase sourceSplitBase) {
// initial stateful objects
final SqlServerConnectorConfig connectorConfig = getDbzConnectorConfig();
this.topicSelector = SqlServerTopicSelector.defaultSelector(connectorConfig);
EmbeddedFlinkDatabaseHistory.registerHistory(
sourceConfig
.getDbzConfiguration()
.getString(EmbeddedFlinkDatabaseHistory.DATABASE_HISTORY_INSTANCE_NAME),
sourceSplitBase.getTableSchemas().values());
this.databaseSchema = SqlServerUtils.createSqlServerDatabaseSchema(connectorConfig);
this.offsetContext = loadStartingOffsetState(new Loader(connectorConfig), sourceSplitBase);
validateAndLoadDatabaseHistory(offsetContext, databaseSchema);
this.taskContext = new SqlServerTaskContext(connectorConfig, databaseSchema);
final int queueSize =
sourceSplitBase.isSnapshotSplit()
? Integer.MAX_VALUE
: getSourceConfig().getDbzConnectorConfig().getMaxQueueSize();
this.queue =
new Builder<DataChangeEvent>()
.pollInterval(connectorConfig.getPollInterval())
.maxBatchSize(connectorConfig.getMaxBatchSize())
.maxQueueSize(queueSize)
.maxQueueSizeInBytes(connectorConfig.getMaxQueueSizeInBytes())
.loggingContextSupplier(
() ->
taskContext.configureLoggingContext(
"sqlserver-cdc-connector-task"))
// do not buffer any element, we use signal event
// .buffering()
.build();
this.dispatcher =
new JdbcSourceEventDispatcher(
connectorConfig,
topicSelector,
databaseSchema,
queue,
connectorConfig.getTableFilters().dataCollectionFilter(),
DataChangeEvent::new,
metadataProvider,
schemaNameAdjuster);
final DefaultChangeEventSourceMetricsFactory changeEventSourceMetricsFactory =
new DefaultChangeEventSourceMetricsFactory();
this.snapshotChangeEventSourceMetrics =
changeEventSourceMetricsFactory.getSnapshotMetrics(
taskContext, queue, metadataProvider);
this.streamingChangeEventSourceMetrics =
changeEventSourceMetricsFactory.getStreamingMetrics(
taskContext, queue, metadataProvider);
this.errorHandler = new SqlServerErrorHandler(connectorConfig.getLogicalName(), queue);
}
/** Loads the connector's persistent offset (if present) via the given loader. */
private SqlServerOffsetContext loadStartingOffsetState(
SqlServerOffsetContext.Loader loader, SourceSplitBase sourceSplitBase) {
Offset offset =
sourceSplitBase.isSnapshotSplit()
? LsnOffset.INITIAL_OFFSET
: sourceSplitBase.asStreamSplit().getStartingOffset();
SqlServerOffsetContext sqlServerOffsetContext = loader.load(offset.getOffset());
return sqlServerOffsetContext;
}
private void validateAndLoadDatabaseHistory(
SqlServerOffsetContext offset, SqlServerDatabaseSchema schema) {
schema.initializeStorage();
schema.recover(offset);
}
@Override
public ChangeEventQueue<DataChangeEvent> getQueue() {
return queue;
}
@Override
public TableFilter getTableFilter() {
return getDbzConnectorConfig().getTableFilters().dataCollectionFilter();
}
@Override
public Offset getStreamOffset(SourceRecord record) {
return SqlServerUtils.getLsn(record);
}
@Override
public SqlServerDatabaseSchema getDatabaseSchema() {
return databaseSchema;
}
@Override
public RowType getSplitType(Table table) {
return SqlServerUtils.getSplitType(table);
}
@Override
public ErrorHandler getErrorHandler() {
return errorHandler;
}
@Override
public SqlServerConnectorConfig getDbzConnectorConfig() {
return (SqlServerConnectorConfig) super.getDbzConnectorConfig();
}
@Override
public SqlServerSourceConfig getSourceConfig() {
return (SqlServerSourceConfig) sourceConfig;
}
@Override
public JdbcSourceEventDispatcher getDispatcher() {
return dispatcher;
}
@Override
public SqlServerOffsetContext getOffsetContext() {
return offsetContext;
}
public SqlServerConnection getConnection() {
return connection;
}
public SqlServerConnection getMetaDataConnection() {
return metaDataConnection;
}
public SnapshotChangeEventSourceMetrics getSnapshotChangeEventSourceMetrics() {
return snapshotChangeEventSourceMetrics;
}
public StreamingChangeEventSourceMetrics getStreamingChangeEventSourceMetrics() {
return streamingChangeEventSourceMetrics;
}
@Override
public TableId getTableId(SourceRecord record) {
return SourceRecordUtils.getTableId(record);
}
@Override
public boolean isDataChangeRecord(SourceRecord record) {
return SourceRecordUtils.isDataChangeRecord(record);
}
@Override
public SchemaNameAdjuster getSchemaNameAdjuster() {
return schemaNameAdjuster;
}
/** Copied from debezium for accessing here. */
public static class SqlServerEventMetadataProvider implements EventMetadataProvider {
@Override
public Instant getEventTimestamp(
DataCollectionId source, OffsetContext offset, Object key, Struct value) {
if (value == null) {
return null;
}
final Struct sourceInfo = value.getStruct(FieldName.SOURCE);
if (source == null) {
return null;
}
final Long timestamp = sourceInfo.getInt64(SourceInfo.TIMESTAMP_KEY);
return timestamp == null ? null : Instant.ofEpochMilli(timestamp);
}
@Override
public Map<String, String> getEventSourcePosition(
DataCollectionId source, OffsetContext offset, Object key, Struct value) {
if (value == null) {
return null;
}
final Struct sourceInfo = value.getStruct(FieldName.SOURCE);
if (source == null) {
return null;
}
return Collect.hashMapOf(
SourceInfo.COMMIT_LSN_KEY, sourceInfo.getString(SourceInfo.COMMIT_LSN_KEY),
SourceInfo.CHANGE_LSN_KEY, sourceInfo.getString(SourceInfo.CHANGE_LSN_KEY));
}
@Override
public String getTransactionId(
DataCollectionId source, OffsetContext offset, Object key, Struct value) {
if (value == null) {
return null;
}
final Struct sourceInfo = value.getStruct(FieldName.SOURCE);
if (source == null) {
return null;
}
return sourceInfo.getString(SourceInfo.COMMIT_LSN_KEY);
}
}
}

@ -0,0 +1,162 @@
/*
* Copyright 2022 Ververica Inc.
*
* Licensed 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 com.ververica.cdc.connectors.sqlserver.source.reader.fetch;
import com.ververica.cdc.connectors.base.relational.JdbcSourceEventDispatcher;
import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase;
import com.ververica.cdc.connectors.base.source.meta.split.StreamSplit;
import com.ververica.cdc.connectors.base.source.meta.wartermark.WatermarkKind;
import com.ververica.cdc.connectors.base.source.reader.external.FetchTask;
import com.ververica.cdc.connectors.sqlserver.source.offset.LsnOffset;
import io.debezium.DebeziumException;
import io.debezium.connector.sqlserver.SqlServerConnection;
import io.debezium.connector.sqlserver.SqlServerConnectorConfig;
import io.debezium.connector.sqlserver.SqlServerDatabaseSchema;
import io.debezium.connector.sqlserver.SqlServerOffsetContext;
import io.debezium.connector.sqlserver.SqlServerStreamingChangeEventSource;
import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.source.spi.ChangeEventSource;
import io.debezium.util.Clock;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static com.ververica.cdc.connectors.sqlserver.source.offset.LsnOffset.NO_STOPPING_OFFSET;
import static com.ververica.cdc.connectors.sqlserver.source.utils.SqlServerUtils.getLsnPosition;
/** The task to work for fetching data of SqlServer table stream split . */
public class SqlServerStreamFetchTask implements FetchTask<SourceSplitBase> {
private final StreamSplit split;
private volatile boolean taskRunning = false;
private LsnSplitReadTask redoLogSplitReadTask;
public SqlServerStreamFetchTask(StreamSplit split) {
this.split = split;
}
@Override
public void execute(Context context) throws Exception {
SqlServerSourceFetchTaskContext sourceFetchContext =
(SqlServerSourceFetchTaskContext) context;
taskRunning = true;
redoLogSplitReadTask =
new LsnSplitReadTask(
sourceFetchContext.getDbzConnectorConfig(),
sourceFetchContext.getConnection(),
sourceFetchContext.getMetaDataConnection(),
sourceFetchContext.getDispatcher(),
sourceFetchContext.getErrorHandler(),
sourceFetchContext.getDatabaseSchema(),
split);
RedoLogSplitChangeEventSourceContext changeEventSourceContext =
new RedoLogSplitChangeEventSourceContext();
redoLogSplitReadTask.execute(
changeEventSourceContext, sourceFetchContext.getOffsetContext());
}
@Override
public boolean isRunning() {
return taskRunning;
}
@Override
public StreamSplit getSplit() {
return split;
}
/**
* A wrapped task to read all binlog for table and also supports read bounded (from lowWatermark
* to highWatermark) binlog.
*/
public static class LsnSplitReadTask extends SqlServerStreamingChangeEventSource {
private static final Logger LOG = LoggerFactory.getLogger(LsnSplitReadTask.class);
private final StreamSplit lsnSplit;
private final JdbcSourceEventDispatcher dispatcher;
private final ErrorHandler errorHandler;
private ChangeEventSourceContext context;
public LsnSplitReadTask(
SqlServerConnectorConfig connectorConfig,
SqlServerConnection connection,
SqlServerConnection metadataConnection,
JdbcSourceEventDispatcher dispatcher,
ErrorHandler errorHandler,
SqlServerDatabaseSchema schema,
StreamSplit lsnSplit) {
super(
connectorConfig,
connection,
metadataConnection,
dispatcher,
errorHandler,
Clock.system(),
schema);
this.lsnSplit = lsnSplit;
this.dispatcher = dispatcher;
this.errorHandler = errorHandler;
}
@Override
public void afterHandleLsn(SqlServerOffsetContext offsetContext) {
// check do we need to stop for fetch binlog for snapshot split.
if (isBoundedRead()) {
final LsnOffset currentRedoLogOffset = getLsnPosition(offsetContext.getOffset());
// reach the high watermark, the binlog fetcher should be finished
if (currentRedoLogOffset.isAtOrAfter(lsnSplit.getEndingOffset())) {
// send binlog end event
try {
dispatcher.dispatchWatermarkEvent(
offsetContext.getPartition(),
lsnSplit,
currentRedoLogOffset,
WatermarkKind.END);
} catch (InterruptedException e) {
LOG.error("Send signal event error.", e);
errorHandler.setProducerThrowable(
new DebeziumException("Error processing binlog signal event", e));
}
// tell fetcher the binlog task finished
((SqlServerScanFetchTask.SnapshotBinlogSplitChangeEventSourceContext) context)
.finished();
}
}
}
private boolean isBoundedRead() {
return !NO_STOPPING_OFFSET.equals(lsnSplit.getEndingOffset());
}
@Override
public void execute(ChangeEventSourceContext context, SqlServerOffsetContext offsetContext)
throws InterruptedException {
this.context = context;
super.execute(context, offsetContext);
}
}
/**
* The {@link ChangeEventSource.ChangeEventSourceContext} implementation for binlog split task.
*/
private class RedoLogSplitChangeEventSourceContext
implements ChangeEventSource.ChangeEventSourceContext {
@Override
public boolean isRunning() {
return taskRunning;
}
}
}

@ -0,0 +1,111 @@
/*
* Copyright 2022 Ververica Inc.
*
* Licensed 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 com.ververica.cdc.connectors.sqlserver.source.utils;
import io.debezium.config.Configuration;
import io.debezium.connector.sqlserver.SqlServerConnection;
import io.debezium.connector.sqlserver.SqlServerConnectorConfig;
import io.debezium.connector.sqlserver.SqlServerValueConverters;
import io.debezium.jdbc.JdbcConnection;
import io.debezium.relational.RelationalTableFilters;
import io.debezium.relational.TableId;
import io.debezium.util.Clock;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.sql.SQLException;
import java.util.ArrayList;
import java.util.List;
/** Utils for SqlServer connection. */
public class SqlServerConnectionUtils {
private static final Logger LOG = LoggerFactory.getLogger(SqlServerConnectionUtils.class);
public static SqlServerConnection createSqlServerConnection(Configuration dbzConfiguration) {
final SqlServerConnectorConfig connectorConfig =
new SqlServerConnectorConfig(dbzConfiguration);
final SqlServerValueConverters valueConverters =
new SqlServerValueConverters(
connectorConfig.getDecimalMode(),
connectorConfig.getTemporalPrecisionMode(),
connectorConfig.binaryHandlingMode());
return new SqlServerConnection(
dbzConfiguration,
Clock.system(),
connectorConfig.getSourceTimestampMode(),
valueConverters,
SqlServerConnectionUtils.class::getClassLoader,
connectorConfig.getSkippedOperations());
}
public static List<TableId> listTables(JdbcConnection jdbc, RelationalTableFilters tableFilters)
throws SQLException {
final List<TableId> capturedTableIds = new ArrayList<>();
// -------------------
// READ DATABASE NAMES
// -------------------
// Get the list of databases ...
LOG.info("Read list of available databases");
final List<String> databaseNames = new ArrayList<>();
jdbc.query(
"SELECT name, database_id, create_date \n" + "FROM sys.databases; ",
rs -> {
while (rs.next()) {
databaseNames.add(rs.getString(1));
}
});
LOG.info("\t list of available databases is: {}", databaseNames);
// ----------------
// READ TABLE NAMES
// ----------------
// Get the list of table IDs for each database. We can't use a prepared statement with
// SqlServer, so we have to build the SQL statement each time. Although in other cases this
// might lead to SQL injection, in our case we are reading the database names from the
// database and not taking them from the user ...
LOG.info("Read list of available tables in each database");
for (String dbName : databaseNames) {
try {
jdbc.query(
"SELECT * FROM "
+ dbName
+ ".INFORMATION_SCHEMA.TABLES WHERE TABLE_TYPE = 'BASE TABLE';",
rs -> {
while (rs.next()) {
TableId tableId =
new TableId(
rs.getString(1), rs.getString(2), rs.getString(3));
if (tableFilters.dataCollectionFilter().isIncluded(tableId)) {
capturedTableIds.add(tableId);
LOG.info("\t including '{}' for further processing", tableId);
} else {
LOG.info("\t '{}' is filtered out of capturing", tableId);
}
}
});
} catch (SQLException e) {
// We were unable to execute the query or process the results, so skip this ...
LOG.warn(
"\t skipping database '{}' due to error reading tables: {}",
dbName,
e.getMessage());
}
}
return capturedTableIds;
}
}

@ -0,0 +1,80 @@
/*
* Copyright 2022 Ververica Inc.
*
* Licensed 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 com.ververica.cdc.connectors.sqlserver.source.utils;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.types.DataType;
import io.debezium.relational.Column;
import java.sql.Types;
/** Utilities for converting from SqlServer types to Flink types. */
public class SqlServerTypeUtils {
/** Returns a corresponding Flink 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 data type from a debezium {@link Column} with nullable always
* be true.
*/
private static DataType convertFromColumn(Column column) {
switch (column.jdbcType()) {
case Types.CHAR:
case Types.VARCHAR:
case Types.NCHAR:
case Types.NVARCHAR:
case Types.STRUCT:
case Types.CLOB:
return DataTypes.STRING();
case Types.BLOB:
return DataTypes.BYTES();
case Types.INTEGER:
case Types.SMALLINT:
case Types.TINYINT:
return DataTypes.INT();
case Types.FLOAT:
case Types.REAL:
case Types.DOUBLE:
case Types.NUMERIC:
case Types.DECIMAL:
return DataTypes.DECIMAL(column.length(), column.scale().orElse(0));
case Types.DATE:
return DataTypes.DATE();
case Types.TIMESTAMP:
case Types.TIMESTAMP_WITH_TIMEZONE:
return column.length() >= 0
? DataTypes.TIMESTAMP(column.length())
: DataTypes.TIMESTAMP();
case Types.BOOLEAN:
return DataTypes.BOOLEAN();
default:
throw new UnsupportedOperationException(
String.format(
"Don't support SqlSever type '%s' yet, jdbcType:'%s'.",
column.typeName(), column.jdbcType()));
}
}
}

@ -0,0 +1,427 @@
/*
* Copyright 2022 Ververica Inc.
*
* Licensed 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 com.ververica.cdc.connectors.sqlserver.source.utils;
import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.util.FlinkRuntimeException;
import com.ververica.cdc.connectors.base.source.meta.offset.Offset;
import com.ververica.cdc.connectors.sqlserver.source.offset.LsnOffset;
import io.debezium.connector.sqlserver.Lsn;
import io.debezium.connector.sqlserver.SourceInfo;
import io.debezium.connector.sqlserver.SqlServerConnection;
import io.debezium.connector.sqlserver.SqlServerConnectorConfig;
import io.debezium.connector.sqlserver.SqlServerDatabaseSchema;
import io.debezium.connector.sqlserver.SqlServerTopicSelector;
import io.debezium.connector.sqlserver.SqlServerValueConverters;
import io.debezium.jdbc.JdbcConnection;
import io.debezium.relational.Column;
import io.debezium.relational.Table;
import io.debezium.relational.TableId;
import io.debezium.schema.TopicSelector;
import io.debezium.util.SchemaNameAdjuster;
import org.apache.kafka.connect.source.SourceRecord;
import java.sql.Connection;
import java.sql.PreparedStatement;
import java.sql.SQLException;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.rowToArray;
import static org.apache.flink.table.api.DataTypes.FIELD;
import static org.apache.flink.table.api.DataTypes.ROW;
/** The utils for SqlServer data source. */
public class SqlServerUtils {
public SqlServerUtils() {}
public static Object[] queryMinMax(JdbcConnection jdbc, TableId tableId, String columnName)
throws SQLException {
final String minMaxQuery =
String.format(
"SELECT MIN(%s), MAX(%s) FROM %s",
quote(columnName), quote(columnName), quote(tableId));
return jdbc.queryAndMap(
minMaxQuery,
rs -> {
if (!rs.next()) {
// this should never happen
throw new SQLException(
String.format(
"No result returned after running query [%s]",
minMaxQuery));
}
return rowToArray(rs, 2);
});
}
public static long queryApproximateRowCnt(JdbcConnection jdbc, TableId tableId)
throws SQLException {
// The statement used to get approximate row count which is less
// accurate than COUNT(*), but is more efficient for large table.
final String useDatabaseStatement = String.format("USE %s;", quote(tableId.catalog()));
final String rowCountQuery =
String.format(
"SELECT Total_Rows = SUM(st.row_count) FROM sys"
+ ".dm_db_partition_stats st WHERE object_name(object_id) = '%s' AND index_id < 2;",
tableId.table());
jdbc.executeWithoutCommitting(useDatabaseStatement);
return jdbc.queryAndMap(
rowCountQuery,
rs -> {
if (!rs.next()) {
throw new SQLException(
String.format(
"No result returned after running query [%s]",
rowCountQuery));
}
return rs.getLong(1);
});
}
public static Object queryMin(
JdbcConnection jdbc, TableId tableId, String columnName, Object excludedLowerBound)
throws SQLException {
final String minQuery =
String.format(
"SELECT MIN(%s) FROM %s WHERE %s > ?",
quote(columnName), quote(tableId), quote(columnName));
return jdbc.prepareQueryAndMap(
minQuery,
ps -> ps.setObject(1, excludedLowerBound),
rs -> {
if (!rs.next()) {
// this should never happen
throw new SQLException(
String.format(
"No result returned after running query [%s]", minQuery));
}
return rs.getObject(1);
});
}
/**
* Returns the next LSN to be read from the database. This is the LSN of the last record that
* was read from the database.
*/
public static Object queryNextChunkMax(
JdbcConnection jdbc,
TableId tableId,
String splitColumnName,
int chunkSize,
Object includedLowerBound)
throws SQLException {
String quotedColumn = quote(splitColumnName);
String query =
String.format(
"SELECT MAX(%s) FROM ("
+ "SELECT TOP (%s) %s FROM %s WHERE %s >= ? ORDER BY %s ASC "
+ ") AS T",
quotedColumn,
chunkSize,
quotedColumn,
quote(tableId),
quotedColumn,
quotedColumn);
return jdbc.prepareQueryAndMap(
query,
ps -> ps.setObject(1, includedLowerBound),
rs -> {
if (!rs.next()) {
// this should never happen
throw new SQLException(
String.format(
"No result returned after running query [%s]", query));
}
return rs.getObject(1);
});
}
public static Column getSplitColumn(Table table) {
List<Column> primaryKeys = table.primaryKeyColumns();
if (primaryKeys.isEmpty()) {
throw new ValidationException(
String.format(
"Incremental snapshot for tables requires primary key,"
+ " but table %s doesn't have primary key.",
table.id()));
}
// use first field in primary key as the split key
return primaryKeys.get(0);
}
public static RowType getSplitType(Table table) {
return getSplitType(getSplitColumn(table));
}
private static RowType getSplitType(Column splitColumn) {
return (RowType)
ROW(FIELD(splitColumn.name(), SqlServerTypeUtils.fromDbzColumn(splitColumn)))
.getLogicalType();
}
public static Offset getLsn(SourceRecord record) {
return getLsnPosition(record.sourceOffset());
}
public static LsnOffset getLsnPosition(Map<String, ?> offset) {
Map<String, String> offsetStrMap = new HashMap<>();
for (Map.Entry<String, ?> entry : offset.entrySet()) {
offsetStrMap.put(
entry.getKey(), entry.getValue() == null ? null : entry.getValue().toString());
}
return new LsnOffset(Lsn.valueOf(offsetStrMap.get(SourceInfo.CHANGE_LSN_KEY)));
}
/** Fetch current largest log sequence number (LSN) of the database. */
public static LsnOffset currentLsn(SqlServerConnection connection) {
try {
Lsn maxLsn = connection.getMaxLsn();
return new LsnOffset(maxLsn);
} catch (SQLException e) {
throw new FlinkRuntimeException(e.getMessage(), e);
}
}
/** Get split scan query for the given table. */
public static String buildSplitScanQuery(
TableId tableId, RowType pkRowType, boolean isFirstSplit, boolean isLastSplit) {
return buildSplitQuery(tableId, pkRowType, isFirstSplit, isLastSplit, -1, true);
}
/** Get table split data PreparedStatement. */
public static PreparedStatement readTableSplitDataStatement(
JdbcConnection jdbc,
String sql,
boolean isFirstSplit,
boolean isLastSplit,
Object[] splitStart,
Object[] splitEnd,
int primaryKeyNum,
int fetchSize) {
try {
final PreparedStatement statement = initStatement(jdbc, sql, fetchSize);
if (isFirstSplit && isLastSplit) {
return statement;
}
if (isFirstSplit) {
for (int i = 0; i < primaryKeyNum; i++) {
statement.setObject(i + 1, splitEnd[i]);
statement.setObject(i + 1 + primaryKeyNum, splitEnd[i]);
}
} else if (isLastSplit) {
for (int i = 0; i < primaryKeyNum; i++) {
statement.setObject(i + 1, splitStart[i]);
}
} else {
for (int i = 0; i < primaryKeyNum; i++) {
statement.setObject(i + 1, splitStart[i]);
statement.setObject(i + 1 + primaryKeyNum, splitEnd[i]);
statement.setObject(i + 1 + 2 * primaryKeyNum, splitEnd[i]);
}
}
return statement;
} catch (Exception e) {
throw new RuntimeException("Failed to build the split data read statement.", e);
}
}
public static SqlServerDatabaseSchema createSqlServerDatabaseSchema(
SqlServerConnectorConfig connectorConfig) {
TopicSelector<TableId> topicSelector =
SqlServerTopicSelector.defaultSelector(connectorConfig);
SchemaNameAdjuster schemaNameAdjuster = SchemaNameAdjuster.create();
SqlServerValueConverters valueConverters =
new SqlServerValueConverters(
connectorConfig.getDecimalMode(),
connectorConfig.getTemporalPrecisionMode(),
connectorConfig.binaryHandlingMode());
return new SqlServerDatabaseSchema(
connectorConfig, valueConverters, topicSelector, schemaNameAdjuster);
}
// --------------------------private method-------------------------------
private static String getPrimaryKeyColumnsProjection(RowType pkRowType) {
StringBuilder sql = new StringBuilder();
for (Iterator<String> fieldNamesIt = pkRowType.getFieldNames().iterator();
fieldNamesIt.hasNext(); ) {
sql.append(fieldNamesIt.next());
if (fieldNamesIt.hasNext()) {
sql.append(" , ");
}
}
return sql.toString();
}
private static String buildSplitQuery(
TableId tableId,
RowType pkRowType,
boolean isFirstSplit,
boolean isLastSplit,
int limitSize,
boolean isScanningData) {
final String condition;
if (isFirstSplit && isLastSplit) {
condition = null;
} else if (isFirstSplit) {
final StringBuilder sql = new StringBuilder();
addPrimaryKeyColumnsToCondition(pkRowType, sql, " <= ?");
if (isScanningData) {
sql.append(" AND NOT (");
addPrimaryKeyColumnsToCondition(pkRowType, sql, " = ?");
sql.append(")");
}
condition = sql.toString();
} else if (isLastSplit) {
final StringBuilder sql = new StringBuilder();
addPrimaryKeyColumnsToCondition(pkRowType, sql, " >= ?");
condition = sql.toString();
} else {
final StringBuilder sql = new StringBuilder();
addPrimaryKeyColumnsToCondition(pkRowType, sql, " >= ?");
if (isScanningData) {
sql.append(" AND NOT (");
addPrimaryKeyColumnsToCondition(pkRowType, sql, " = ?");
sql.append(")");
}
sql.append(" AND ");
addPrimaryKeyColumnsToCondition(pkRowType, sql, " <= ?");
condition = sql.toString();
}
if (isScanningData) {
return buildSelectWithRowLimits(
tableId, limitSize, "*", Optional.ofNullable(condition), Optional.empty());
} else {
final String orderBy =
pkRowType.getFieldNames().stream().collect(Collectors.joining(", "));
return buildSelectWithBoundaryRowLimits(
tableId,
limitSize,
getPrimaryKeyColumnsProjection(pkRowType),
getMaxPrimaryKeyColumnsProjection(pkRowType),
Optional.ofNullable(condition),
orderBy);
}
}
private static PreparedStatement initStatement(JdbcConnection jdbc, String sql, int fetchSize)
throws SQLException {
final Connection connection = jdbc.connection();
connection.setAutoCommit(false);
final PreparedStatement statement = connection.prepareStatement(sql);
statement.setFetchSize(fetchSize);
return statement;
}
private static String getMaxPrimaryKeyColumnsProjection(RowType pkRowType) {
StringBuilder sql = new StringBuilder();
for (Iterator<String> fieldNamesIt = pkRowType.getFieldNames().iterator();
fieldNamesIt.hasNext(); ) {
sql.append("MAX(" + fieldNamesIt.next() + ")");
if (fieldNamesIt.hasNext()) {
sql.append(" , ");
}
}
return sql.toString();
}
private static String buildSelectWithRowLimits(
TableId tableId,
int limit,
String projection,
Optional<String> condition,
Optional<String> orderBy) {
final StringBuilder sql = new StringBuilder("SELECT ");
if (limit > 0) {
sql.append(" TOP( ").append(limit).append(") ");
}
sql.append(projection).append(" FROM ");
sql.append(quoteSchemaAndTable(tableId));
if (condition.isPresent()) {
sql.append(" WHERE ").append(condition.get());
}
if (orderBy.isPresent()) {
sql.append(" ORDER BY ").append(orderBy.get());
}
return sql.toString();
}
private static String quoteSchemaAndTable(TableId tableId) {
StringBuilder quoted = new StringBuilder();
if (tableId.schema() != null && !tableId.schema().isEmpty()) {
quoted.append(quote(tableId.schema())).append(".");
}
quoted.append(quote(tableId.table()));
return quoted.toString();
}
public static String quote(String dbOrTableName) {
return "[" + dbOrTableName + "]";
}
public static String quote(TableId tableId) {
return "[" + tableId.schema() + "].[" + tableId.table() + "]";
}
private static void addPrimaryKeyColumnsToCondition(
RowType pkRowType, StringBuilder sql, String predicate) {
for (Iterator<String> fieldNamesIt = pkRowType.getFieldNames().iterator();
fieldNamesIt.hasNext(); ) {
sql.append(fieldNamesIt.next()).append(predicate);
if (fieldNamesIt.hasNext()) {
sql.append(" AND ");
}
}
}
private static String buildSelectWithBoundaryRowLimits(
TableId tableId,
int limit,
String projection,
String maxColumnProjection,
Optional<String> condition,
String orderBy) {
final StringBuilder sql = new StringBuilder("SELECT ");
sql.append(maxColumnProjection);
sql.append(" FROM (");
sql.append("SELECT ");
sql.append(" TOP( ").append(limit).append(") ");
sql.append(projection);
sql.append(" FROM ");
sql.append(quoteSchemaAndTable(tableId));
if (condition.isPresent()) {
sql.append(" WHERE ").append(condition.get());
}
sql.append(" ORDER BY ").append(orderBy);
sql.append(") T");
return sql.toString();
}
}

@ -25,13 +25,28 @@ import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.factories.DynamicTableSourceFactory;
import org.apache.flink.table.factories.FactoryUtil;
import com.ververica.cdc.connectors.base.options.StartupOptions;
import java.time.Duration;
import java.time.ZoneId;
import java.util.HashSet;
import java.util.Set;
import static com.ververica.cdc.connectors.base.options.JdbcSourceOptions.CONNECTION_POOL_SIZE;
import static com.ververica.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_MAX_RETRIES;
import static com.ververica.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_TIMEOUT;
import static com.ververica.cdc.connectors.base.options.JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_KEY_COLUMN;
import static com.ververica.cdc.connectors.base.options.SourceOptions.CHUNK_META_GROUP_SIZE;
import static com.ververica.cdc.connectors.base.options.SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE;
import static com.ververica.cdc.connectors.base.options.SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_ENABLED;
import static com.ververica.cdc.connectors.base.options.SourceOptions.SCAN_SNAPSHOT_FETCH_SIZE;
import static com.ververica.cdc.connectors.base.options.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND;
import static com.ververica.cdc.connectors.base.options.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND;
import static com.ververica.cdc.connectors.base.utils.ObjectUtils.doubleCompare;
import static com.ververica.cdc.debezium.table.DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX;
import static com.ververica.cdc.debezium.table.DebeziumOptions.getDebeziumProperties;
import static com.ververica.cdc.debezium.utils.ResolvedSchemaUtils.getPhysicalSchema;
import static org.apache.flink.util.Preconditions.checkState;
/** Factory for creating configured instance of {@link SqlServerTableSource}. */
public class SqlServerTableFactory implements DynamicTableSourceFactory {
@ -69,13 +84,6 @@ public class SqlServerTableFactory implements DynamicTableSourceFactory {
.stringType()
.noDefaultValue()
.withDescription("Database name of the SqlServer server to monitor.");
private static final ConfigOption<String> SCHEMA_NAME =
ConfigOptions.key("schema-name")
.stringType()
.noDefaultValue()
.withDescription("Schema name of the SqlServer database to monitor.");
private static final ConfigOption<String> TABLE_NAME =
ConfigOptions.key("table-name")
.stringType()
@ -106,7 +114,6 @@ public class SqlServerTableFactory implements DynamicTableSourceFactory {
String hostname = config.get(HOSTNAME);
String username = config.get(USERNAME);
String password = config.get(PASSWORD);
String schemaName = config.get(SCHEMA_NAME);
String databaseName = config.get(DATABASE_NAME);
String tableName = config.get(TABLE_NAME);
ZoneId serverTimeZone = ZoneId.of(config.get(SERVER_TIME_ZONE));
@ -115,18 +122,49 @@ public class SqlServerTableFactory implements DynamicTableSourceFactory {
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);
Duration connectTimeout = config.get(CONNECT_TIMEOUT);
int connectMaxRetries = config.get(CONNECT_MAX_RETRIES);
int connectionPoolSize = config.get(CONNECTION_POOL_SIZE);
double distributionFactorUpper = config.get(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND);
double distributionFactorLower = config.get(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND);
String chunkKeyColumn =
config.getOptional(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_KEY_COLUMN).orElse(null);
if (enableParallelRead) {
validateIntegerOption(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE, splitSize, 1);
validateIntegerOption(SCAN_SNAPSHOT_FETCH_SIZE, fetchSize, 1);
validateIntegerOption(CHUNK_META_GROUP_SIZE, splitMetaGroupSize, 1);
validateIntegerOption(CONNECTION_POOL_SIZE, connectionPoolSize, 1);
validateIntegerOption(CONNECT_MAX_RETRIES, connectMaxRetries, 0);
validateDistributionFactorUpper(distributionFactorUpper);
validateDistributionFactorLower(distributionFactorLower);
}
return new SqlServerTableSource(
physicalSchema,
port,
hostname,
databaseName,
schemaName,
tableName,
serverTimeZone,
username,
password,
getDebeziumProperties(context.getCatalogTable().getOptions()),
startupOptions);
startupOptions,
enableParallelRead,
splitSize,
splitMetaGroupSize,
fetchSize,
connectTimeout,
connectionPoolSize,
connectMaxRetries,
distributionFactorUpper,
distributionFactorLower,
chunkKeyColumn);
}
@Override
@ -141,7 +179,6 @@ public class SqlServerTableFactory implements DynamicTableSourceFactory {
options.add(USERNAME);
options.add(PASSWORD);
options.add(DATABASE_NAME);
options.add(SCHEMA_NAME);
options.add(TABLE_NAME);
return options;
}
@ -152,12 +189,20 @@ public class SqlServerTableFactory implements DynamicTableSourceFactory {
options.add(PORT);
options.add(SERVER_TIME_ZONE);
options.add(SCAN_STARTUP_MODE);
options.add(SCAN_INCREMENTAL_SNAPSHOT_ENABLED);
options.add(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE);
options.add(CHUNK_META_GROUP_SIZE);
options.add(SCAN_SNAPSHOT_FETCH_SIZE);
options.add(CONNECT_TIMEOUT);
options.add(CONNECT_MAX_RETRIES);
options.add(CONNECTION_POOL_SIZE);
options.add(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND);
options.add(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND);
options.add(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_KEY_COLUMN);
return options;
}
private static final String SCAN_STARTUP_MODE_VALUE_INITIAL = "initial";
private static final String SCAN_STARTUP_MODE_VALUE_INITIAL_ONLY = "initial-only";
private static final String SCAN_STARTUP_MODE_VALUE_LATEST = "latest-offset";
private static StartupOptions getStartupOptions(ReadableConfig config) {
@ -167,21 +212,51 @@ public class SqlServerTableFactory implements DynamicTableSourceFactory {
case SCAN_STARTUP_MODE_VALUE_INITIAL:
return StartupOptions.initial();
case SCAN_STARTUP_MODE_VALUE_INITIAL_ONLY:
return StartupOptions.initialOnly();
case SCAN_STARTUP_MODE_VALUE_LATEST:
return StartupOptions.latest();
default:
throw new ValidationException(
String.format(
"Invalid value for option '%s'. Supported values are [%s, %s, %s], but was: %s",
"Invalid value for option '%s'. Supported values are [%s, %s], but was: %s",
SCAN_STARTUP_MODE.key(),
SCAN_STARTUP_MODE_VALUE_INITIAL,
SCAN_STARTUP_MODE_VALUE_INITIAL_ONLY,
SCAN_STARTUP_MODE_VALUE_LATEST,
modeString));
}
}
/** Checks the value of given integer option is valid. */
private void validateIntegerOption(
ConfigOption<Integer> option, int optionValue, int exclusiveMin) {
checkState(
optionValue > exclusiveMin,
String.format(
"The value of option '%s' must larger than %d, but is %d",
option.key(), exclusiveMin, optionValue));
}
/** Checks the value of given evenly distribution factor upper bound is valid. */
private void validateDistributionFactorUpper(double distributionFactorUpper) {
checkState(
doubleCompare(distributionFactorUpper, 1.0d) >= 0,
String.format(
"The value of option '%s' must larger than or equals %s, but is %s",
SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND.key(),
1.0d,
distributionFactorUpper));
}
/** Checks the value of given evenly distribution factor lower bound is valid. */
private void validateDistributionFactorLower(double distributionFactorLower) {
checkState(
doubleCompare(distributionFactorLower, 0.0d) >= 0
&& doubleCompare(distributionFactorLower, 1.0d) <= 0,
String.format(
"The value of option '%s' must between %s and %s inclusively, but is %s",
SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND.key(),
0.0d,
1.0d,
distributionFactorLower));
}
}

@ -22,17 +22,24 @@ import org.apache.flink.table.connector.ChangelogMode;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.connector.source.ScanTableSource;
import org.apache.flink.table.connector.source.SourceFunctionProvider;
import org.apache.flink.table.connector.source.SourceProvider;
import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.RowType;
import com.ververica.cdc.connectors.base.options.StartupOptions;
import com.ververica.cdc.connectors.base.source.jdbc.JdbcIncrementalSource;
import com.ververica.cdc.connectors.sqlserver.SqlServerSource;
import com.ververica.cdc.connectors.sqlserver.source.SqlServerSourceBuilder;
import com.ververica.cdc.debezium.DebeziumDeserializationSchema;
import com.ververica.cdc.debezium.DebeziumSourceFunction;
import com.ververica.cdc.debezium.table.MetadataConverter;
import com.ververica.cdc.debezium.table.RowDataDebeziumDeserializeSchema;
import javax.annotation.Nullable;
import java.time.Duration;
import java.time.ZoneId;
import java.util.Collections;
import java.util.List;
@ -54,13 +61,22 @@ public class SqlServerTableSource implements ScanTableSource, SupportsReadingMet
private final int port;
private final String hostname;
private final String database;
private final String schemaName;
private final String tableName;
private final ZoneId serverTimeZone;
private final String username;
private final String password;
private final Properties dbzProperties;
private final StartupOptions startupOptions;
private final boolean enableParallelRead;
private final int splitSize;
private final int splitMetaGroupSize;
private final int fetchSize;
private final Duration connectTimeout;
private final int connectionPoolSize;
private final int connectMaxRetries;
private final double distributionFactorUpper;
private final double distributionFactorLower;
private final String chunkKeyColumn;
// --------------------------------------------------------------------------------------------
// Mutable attributes
@ -77,26 +93,44 @@ public class SqlServerTableSource implements ScanTableSource, SupportsReadingMet
int port,
String hostname,
String database,
String schemaName,
String tableName,
ZoneId serverTimeZone,
String username,
String password,
Properties dbzProperties,
StartupOptions startupOptions) {
StartupOptions startupOptions,
boolean enableParallelRead,
int splitSize,
int splitMetaGroupSize,
int fetchSize,
Duration connectTimeout,
int connectMaxRetries,
int connectionPoolSize,
double distributionFactorUpper,
double distributionFactorLower,
@Nullable String chunkKeyColumn) {
this.physicalSchema = physicalSchema;
this.port = port;
this.hostname = checkNotNull(hostname);
this.database = checkNotNull(database);
this.schemaName = checkNotNull(schemaName);
this.tableName = checkNotNull(tableName);
this.serverTimeZone = serverTimeZone;
this.username = checkNotNull(username);
this.password = checkNotNull(password);
this.dbzProperties = dbzProperties;
this.startupOptions = startupOptions;
this.producedDataType = physicalSchema.toPhysicalRowDataType();
this.metadataKeys = Collections.emptyList();
this.startupOptions = startupOptions;
this.enableParallelRead = enableParallelRead;
this.splitSize = splitSize;
this.splitMetaGroupSize = splitMetaGroupSize;
this.fetchSize = fetchSize;
this.connectTimeout = connectTimeout;
this.connectionPoolSize = connectionPoolSize;
this.connectMaxRetries = connectMaxRetries;
this.distributionFactorUpper = distributionFactorUpper;
this.distributionFactorLower = distributionFactorLower;
this.chunkKeyColumn = chunkKeyColumn;
}
@Override
@ -120,19 +154,44 @@ public class SqlServerTableSource implements ScanTableSource, SupportsReadingMet
.setUserDefinedConverterFactory(
SqlServerDeserializationConverterFactory.instance())
.build();
DebeziumSourceFunction<RowData> sourceFunction =
SqlServerSource.<RowData>builder()
.hostname(hostname)
.port(port)
.database(database)
.tableList(schemaName + "." + tableName)
.username(username)
.password(password)
.debeziumProperties(dbzProperties)
.startupOptions(startupOptions)
.deserializer(deserializer)
.build();
return SourceFunctionProvider.of(sourceFunction, false);
if (enableParallelRead) {
JdbcIncrementalSource<RowData> sqlServerChangeEventSource =
SqlServerSourceBuilder.SqlServerIncrementalSource.<RowData>builder()
.hostname(hostname)
.port(port)
.databaseList(database)
.tableList(tableName)
.username(username)
.password(password)
.startupOptions(startupOptions)
.deserializer(deserializer)
.debeziumProperties(dbzProperties)
.splitSize(splitSize)
.splitMetaGroupSize(splitMetaGroupSize)
.fetchSize(fetchSize)
.connectTimeout(connectTimeout)
.connectionPoolSize(connectionPoolSize)
.connectMaxRetries(connectMaxRetries)
.distributionFactorUpper(distributionFactorUpper)
.distributionFactorLower(distributionFactorLower)
.build();
return SourceProvider.of(sqlServerChangeEventSource);
} else {
DebeziumSourceFunction<RowData> sourceFunction =
SqlServerSource.<RowData>builder()
.hostname(hostname)
.port(port)
.database(database)
.tableList(tableName)
.username(username)
.password(password)
.debeziumProperties(dbzProperties)
.startupOptions(startupOptions)
.deserializer(deserializer)
.build();
return SourceFunctionProvider.of(sourceFunction, false);
}
}
private MetadataConverter[] getMetadataConverters() {
@ -159,13 +218,22 @@ public class SqlServerTableSource implements ScanTableSource, SupportsReadingMet
port,
hostname,
database,
schemaName,
tableName,
serverTimeZone,
username,
password,
dbzProperties,
startupOptions);
startupOptions,
enableParallelRead,
splitSize,
splitMetaGroupSize,
fetchSize,
connectTimeout,
connectionPoolSize,
connectMaxRetries,
distributionFactorUpper,
distributionFactorLower,
chunkKeyColumn);
source.metadataKeys = metadataKeys;
source.producedDataType = producedDataType;
return source;
@ -184,7 +252,6 @@ public class SqlServerTableSource implements ScanTableSource, SupportsReadingMet
&& Objects.equals(physicalSchema, that.physicalSchema)
&& Objects.equals(hostname, that.hostname)
&& Objects.equals(database, that.database)
&& Objects.equals(schemaName, that.schemaName)
&& Objects.equals(tableName, that.tableName)
&& Objects.equals(serverTimeZone, that.serverTimeZone)
&& Objects.equals(username, that.username)
@ -192,7 +259,17 @@ public class SqlServerTableSource implements ScanTableSource, SupportsReadingMet
&& Objects.equals(dbzProperties, that.dbzProperties)
&& Objects.equals(startupOptions, that.startupOptions)
&& Objects.equals(producedDataType, that.producedDataType)
&& Objects.equals(metadataKeys, that.metadataKeys);
&& Objects.equals(metadataKeys, that.metadataKeys)
&& Objects.equals(enableParallelRead, that.enableParallelRead)
&& Objects.equals(splitSize, that.splitSize)
&& Objects.equals(splitMetaGroupSize, that.splitMetaGroupSize)
&& Objects.equals(fetchSize, that.fetchSize)
&& Objects.equals(connectTimeout, that.connectTimeout)
&& Objects.equals(connectMaxRetries, that.connectMaxRetries)
&& Objects.equals(connectionPoolSize, that.connectionPoolSize)
&& Objects.equals(distributionFactorUpper, that.distributionFactorUpper)
&& Objects.equals(distributionFactorLower, that.distributionFactorLower)
&& Objects.equals(chunkKeyColumn, that.chunkKeyColumn);
}
@Override
@ -202,7 +279,6 @@ public class SqlServerTableSource implements ScanTableSource, SupportsReadingMet
port,
hostname,
database,
schemaName,
tableName,
serverTimeZone,
username,
@ -210,7 +286,17 @@ public class SqlServerTableSource implements ScanTableSource, SupportsReadingMet
dbzProperties,
startupOptions,
producedDataType,
metadataKeys);
metadataKeys,
enableParallelRead,
splitSize,
splitMetaGroupSize,
fetchSize,
connectTimeout,
connectMaxRetries,
connectionPoolSize,
distributionFactorUpper,
distributionFactorLower,
chunkKeyColumn);
}
@Override

@ -1,78 +0,0 @@
/*
* Copyright 2022 Ververica Inc.
*
* Licensed 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 com.ververica.cdc.connectors.sqlserver.table;
import java.util.Objects;
/** Debezium startup options. */
public final class StartupOptions {
public final StartupMode startupMode;
/**
* Takes a snapshot of structure and data of captured tables; useful if topics should be
* populated with a complete representation of the data from the captured tables.
*/
public static StartupOptions initial() {
return new StartupOptions(StartupMode.INITIAL);
}
/**
* Takes a snapshot of structure and data like initial but instead does not transition into
* streaming changes once the snapshot has completed.
*/
public static StartupOptions initialOnly() {
return new StartupOptions(StartupMode.INITIAL_ONLY);
}
/**
* Takes a snapshot of the structure of captured tables only; useful if only changes happening
* from now onwards should be propagated to topics.
*/
public static StartupOptions latest() {
return new StartupOptions(StartupMode.LATEST_OFFSET);
}
private StartupOptions(StartupMode startupMode) {
this.startupMode = startupMode;
switch (startupMode) {
case INITIAL:
case INITIAL_ONLY:
case LATEST_OFFSET:
break;
default:
throw new UnsupportedOperationException(startupMode + " mode is not supported.");
}
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
StartupOptions that = (StartupOptions) o;
return startupMode == that.startupMode;
}
@Override
public int hashCode() {
return Objects.hash(startupMode);
}
}

@ -0,0 +1,555 @@
/*
* Copyright 2022 Ververica Inc.
*
* Licensed 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 io.debezium.connector.sqlserver;
import io.debezium.connector.sqlserver.SqlServerConnectorConfig.SnapshotMode;
import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.source.spi.StreamingChangeEventSource;
import io.debezium.relational.Table;
import io.debezium.relational.TableId;
import io.debezium.schema.SchemaChangeEvent.SchemaChangeEventType;
import io.debezium.util.Clock;
import io.debezium.util.ElapsedTimeStrategy;
import io.debezium.util.Metronome;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.sql.SQLException;
import java.time.Duration;
import java.time.Instant;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.PriorityQueue;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
/**
* Copied from Debezium project(1.6.4.final) to add method {@link
* SqlServerStreamingChangeEventSource#afterHandleLsn(SqlServerOffsetContext)}. A {@link
* StreamingChangeEventSource} based on SQL Server change data capture functionality. A main loop
* polls database DDL change and change data tables and turns them into change events.
*
* <p>The connector uses CDC functionality of SQL Server that is implemented as as a process that
* monitors source table and write changes from the table into the change table.
*
* <p>The main loop keeps a pointer to the LSN of changes that were already processed. It queries
* all change tables and get result set of changes. It always finds the smallest LSN across all
* tables and the change is converted into the event message and sent downstream. The process
* repeats until all result sets are empty. The LSN is marked and the procedure repeats.
*
* <p>The schema changes detection follows the procedure recommended by SQL Server CDC
* documentation. The database operator should create one more capture process (and table) when a
* table schema is updated. The code detects presence of two change tables for a single source
* table. It decides which table is the new one depending on LSNs stored in them. The loop streams
* changes from the older table till there are events in new table with the LSN larger than in the
* old one. Then the change table is switched and streaming is executed from the new one.
*/
public class SqlServerStreamingChangeEventSource
implements StreamingChangeEventSource<SqlServerOffsetContext> {
private static final Pattern MISSING_CDC_FUNCTION_CHANGES_ERROR =
Pattern.compile("Invalid object name 'cdc.fn_cdc_get_all_changes_(.*)'\\.");
private static final Logger LOGGER =
LoggerFactory.getLogger(SqlServerStreamingChangeEventSource.class);
private static final Duration DEFAULT_INTERVAL_BETWEEN_COMMITS = Duration.ofMinutes(1);
private static final int INTERVAL_BETWEEN_COMMITS_BASED_ON_POLL_FACTOR = 3;
/** Connection used for reading CDC tables. */
private final SqlServerConnection dataConnection;
/**
* A separate connection for retrieving timestamps; without it, adaptive buffering will not
* work.
*
* @link
* https://docs.microsoft.com/en-us/sql/connect/jdbc/using-adaptive-buffering?view=sql-server-2017#guidelines-for-using-adaptive-buffering
*/
private final SqlServerConnection metadataConnection;
private final EventDispatcher<TableId> dispatcher;
private final ErrorHandler errorHandler;
private final Clock clock;
private final SqlServerDatabaseSchema schema;
private final Duration pollInterval;
private final SqlServerConnectorConfig connectorConfig;
private final ElapsedTimeStrategy pauseBetweenCommits;
public SqlServerStreamingChangeEventSource(
SqlServerConnectorConfig connectorConfig,
SqlServerConnection dataConnection,
SqlServerConnection metadataConnection,
EventDispatcher<TableId> dispatcher,
ErrorHandler errorHandler,
Clock clock,
SqlServerDatabaseSchema schema) {
this.connectorConfig = connectorConfig;
this.dataConnection = dataConnection;
this.metadataConnection = metadataConnection;
this.dispatcher = dispatcher;
this.errorHandler = errorHandler;
this.clock = clock;
this.schema = schema;
this.pollInterval = connectorConfig.getPollInterval();
final Duration intervalBetweenCommitsBasedOnPoll =
this.pollInterval.multipliedBy(INTERVAL_BETWEEN_COMMITS_BASED_ON_POLL_FACTOR);
this.pauseBetweenCommits =
ElapsedTimeStrategy.constant(
clock,
DEFAULT_INTERVAL_BETWEEN_COMMITS.compareTo(
intervalBetweenCommitsBasedOnPoll)
> 0
? DEFAULT_INTERVAL_BETWEEN_COMMITS.toMillis()
: intervalBetweenCommitsBasedOnPoll.toMillis());
this.pauseBetweenCommits.hasElapsed();
}
@Override
public void execute(ChangeEventSourceContext context, SqlServerOffsetContext offsetContext)
throws InterruptedException {
if (connectorConfig.getSnapshotMode().equals(SnapshotMode.INITIAL_ONLY)) {
LOGGER.info("Streaming is not enabled in current configuration");
return;
}
final Metronome metronome = Metronome.sleeper(pollInterval, clock);
final Queue<SqlServerChangeTable> schemaChangeCheckpoints =
new PriorityQueue<>((x, y) -> x.getStopLsn().compareTo(y.getStopLsn()));
try {
final AtomicReference<SqlServerChangeTable[]> tablesSlot =
new AtomicReference<SqlServerChangeTable[]>(getCdcTablesToQuery(offsetContext));
final TxLogPosition lastProcessedPositionOnStart = offsetContext.getChangePosition();
final long lastProcessedEventSerialNoOnStart = offsetContext.getEventSerialNo();
LOGGER.info(
"Last position recorded in offsets is {}[{}]",
lastProcessedPositionOnStart,
lastProcessedEventSerialNoOnStart);
final AtomicBoolean changesStoppedBeingMonotonic = new AtomicBoolean(false);
final int maxTransactionsPerIteration =
connectorConfig.getMaxTransactionsPerIteration();
TxLogPosition lastProcessedPosition = lastProcessedPositionOnStart;
// LSN should be increased for the first run only immediately after snapshot completion
// otherwise we might skip an incomplete transaction after restart
boolean shouldIncreaseFromLsn = offsetContext.isSnapshotCompleted();
while (context.isRunning()) {
commitTransaction();
afterHandleLsn(offsetContext);
final Lsn toLsn =
getToLsn(
dataConnection, lastProcessedPosition, maxTransactionsPerIteration);
// Shouldn't happen if the agent is running, but it is better to guard against such
// situation
if (!toLsn.isAvailable()) {
LOGGER.warn(
"No maximum LSN recorded in the database; please ensure that the SQL Server Agent is running");
metronome.pause();
continue;
}
// There is no change in the database
if (toLsn.compareTo(lastProcessedPosition.getCommitLsn()) <= 0
&& shouldIncreaseFromLsn) {
LOGGER.debug("No change in the database");
metronome.pause();
continue;
}
// Reading interval is inclusive so we need to move LSN forward but not for first
// run as TX might not be streamed completely
final Lsn fromLsn =
lastProcessedPosition.getCommitLsn().isAvailable() && shouldIncreaseFromLsn
? dataConnection.incrementLsn(lastProcessedPosition.getCommitLsn())
: lastProcessedPosition.getCommitLsn();
shouldIncreaseFromLsn = true;
while (!schemaChangeCheckpoints.isEmpty()) {
migrateTable(schemaChangeCheckpoints, offsetContext);
}
if (!dataConnection.listOfNewChangeTables(fromLsn, toLsn).isEmpty()) {
final SqlServerChangeTable[] tables = getCdcTablesToQuery(offsetContext);
tablesSlot.set(tables);
for (SqlServerChangeTable table : tables) {
if (table.getStartLsn().isBetween(fromLsn, toLsn)) {
LOGGER.info("Schema will be changed for {}", table);
schemaChangeCheckpoints.add(table);
}
}
}
try {
dataConnection.getChangesForTables(
tablesSlot.get(),
fromLsn,
toLsn,
resultSets -> {
long eventSerialNoInInitialTx = 1;
final int tableCount = resultSets.length;
final SqlServerChangeTablePointer[] changeTables =
new SqlServerChangeTablePointer[tableCount];
final SqlServerChangeTable[] tables = tablesSlot.get();
for (int i = 0; i < tableCount; i++) {
changeTables[i] =
new SqlServerChangeTablePointer(
tables[i],
resultSets[i],
connectorConfig.getSourceTimestampMode());
changeTables[i].next();
}
for (; ; ) {
SqlServerChangeTablePointer tableWithSmallestLsn = null;
for (SqlServerChangeTablePointer changeTable : changeTables) {
if (changeTable.isCompleted()) {
continue;
}
if (tableWithSmallestLsn == null
|| changeTable.compareTo(tableWithSmallestLsn)
< 0) {
tableWithSmallestLsn = changeTable;
}
}
if (tableWithSmallestLsn == null) {
// No more LSNs available
break;
}
if (!(tableWithSmallestLsn.getChangePosition().isAvailable()
&& tableWithSmallestLsn
.getChangePosition()
.getInTxLsn()
.isAvailable())) {
LOGGER.error(
"Skipping change {} as its LSN is NULL which is not expected",
tableWithSmallestLsn);
tableWithSmallestLsn.next();
continue;
}
if (tableWithSmallestLsn.isNewTransaction()
&& changesStoppedBeingMonotonic.get()) {
LOGGER.info(
"Resetting changesStoppedBeingMonotonic as transaction changes");
changesStoppedBeingMonotonic.set(false);
}
// After restart for changes that are not monotonic to avoid
// data loss
if (tableWithSmallestLsn
.isCurrentPositionSmallerThanPreviousPosition()) {
LOGGER.info(
"Disabling skipping changes due to not monotonic order of changes");
changesStoppedBeingMonotonic.set(true);
}
// After restart for changes that were executed before the last
// committed offset
if (!changesStoppedBeingMonotonic.get()
&& tableWithSmallestLsn
.getChangePosition()
.compareTo(lastProcessedPositionOnStart)
< 0) {
LOGGER.info(
"Skipping change {} as its position is smaller than the last recorded position {}",
tableWithSmallestLsn,
lastProcessedPositionOnStart);
tableWithSmallestLsn.next();
continue;
}
// After restart for change that was the last committed and
// operations in it before the last committed offset
if (!changesStoppedBeingMonotonic.get()
&& tableWithSmallestLsn
.getChangePosition()
.compareTo(lastProcessedPositionOnStart)
== 0
&& eventSerialNoInInitialTx
<= lastProcessedEventSerialNoOnStart) {
LOGGER.info(
"Skipping change {} as its order in the transaction {} is smaller than or equal to the last recorded operation {}[{}]",
tableWithSmallestLsn,
eventSerialNoInInitialTx,
lastProcessedPositionOnStart,
lastProcessedEventSerialNoOnStart);
eventSerialNoInInitialTx++;
tableWithSmallestLsn.next();
continue;
}
if (tableWithSmallestLsn
.getChangeTable()
.getStopLsn()
.isAvailable()
&& tableWithSmallestLsn
.getChangeTable()
.getStopLsn()
.compareTo(
tableWithSmallestLsn
.getChangePosition()
.getCommitLsn())
<= 0) {
LOGGER.debug(
"Skipping table change {} as its stop LSN is smaller than the last recorded LSN {}",
tableWithSmallestLsn,
tableWithSmallestLsn.getChangePosition());
tableWithSmallestLsn.next();
continue;
}
LOGGER.trace("Processing change {}", tableWithSmallestLsn);
LOGGER.trace(
"Schema change checkpoints {}",
schemaChangeCheckpoints);
if (!schemaChangeCheckpoints.isEmpty()) {
if (tableWithSmallestLsn
.getChangePosition()
.getCommitLsn()
.compareTo(
schemaChangeCheckpoints
.peek()
.getStartLsn())
>= 0) {
migrateTable(schemaChangeCheckpoints, offsetContext);
}
}
final TableId tableId =
tableWithSmallestLsn
.getChangeTable()
.getSourceTableId();
final int operation = tableWithSmallestLsn.getOperation();
final Object[] data = tableWithSmallestLsn.getData();
// UPDATE consists of two consecutive events, first event
// contains
// the row before it was updated and the second the row after
// it was updated
int eventCount = 1;
if (operation
== SqlServerChangeRecordEmitter.OP_UPDATE_BEFORE) {
if (!tableWithSmallestLsn.next()
|| tableWithSmallestLsn.getOperation()
!= SqlServerChangeRecordEmitter
.OP_UPDATE_AFTER) {
throw new IllegalStateException(
"The update before event at "
+ tableWithSmallestLsn
.getChangePosition()
+ " for table "
+ tableId
+ " was not followed by after event.\n Please report this as a bug together with a events around given LSN.");
}
eventCount = 2;
}
final Object[] dataNext =
(operation
== SqlServerChangeRecordEmitter
.OP_UPDATE_BEFORE)
? tableWithSmallestLsn.getData()
: null;
offsetContext.setChangePosition(
tableWithSmallestLsn.getChangePosition(), eventCount);
offsetContext.event(
tableWithSmallestLsn
.getChangeTable()
.getSourceTableId(),
connectorConfig
.getSourceTimestampMode()
.getTimestamp(
metadataConnection,
clock,
tableWithSmallestLsn.getResultSet()));
dispatcher.dispatchDataChangeEvent(
tableId,
new SqlServerChangeRecordEmitter(
offsetContext,
operation,
data,
dataNext,
clock));
tableWithSmallestLsn.next();
}
});
lastProcessedPosition = TxLogPosition.valueOf(toLsn);
// Terminate the transaction otherwise CDC could not be disabled for tables
dataConnection.rollback();
} catch (SQLException e) {
tablesSlot.set(processErrorFromChangeTableQuery(e, tablesSlot.get()));
}
}
} catch (Exception e) {
errorHandler.setProducerThrowable(e);
}
}
private void commitTransaction() throws SQLException {
// When reading from read-only Always On replica the default and only transaction isolation
// is snapshot. This means that CDC metadata are not visible for long-running transactions.
// It is thus necessary to restart the transaction before every read.
// For R/W database it is important to execute regular commits to maintain the size of
// TempDB
if (connectorConfig.isReadOnlyDatabaseConnection() || pauseBetweenCommits.hasElapsed()) {
dataConnection.commit();
}
}
private void migrateTable(
final Queue<SqlServerChangeTable> schemaChangeCheckpoints,
SqlServerOffsetContext offsetContext)
throws InterruptedException, SQLException {
final SqlServerChangeTable newTable = schemaChangeCheckpoints.poll();
LOGGER.info("Migrating schema to {}", newTable);
Table tableSchema = metadataConnection.getTableSchemaFromTable(newTable);
dispatcher.dispatchSchemaChangeEvent(
newTable.getSourceTableId(),
new SqlServerSchemaChangeEventEmitter(
offsetContext, newTable, tableSchema, SchemaChangeEventType.ALTER));
newTable.setSourceTable(tableSchema);
}
private SqlServerChangeTable[] processErrorFromChangeTableQuery(
SQLException exception, SqlServerChangeTable[] currentChangeTables) throws Exception {
final Matcher m = MISSING_CDC_FUNCTION_CHANGES_ERROR.matcher(exception.getMessage());
if (m.matches()) {
final String captureName = m.group(1);
LOGGER.info("Table is no longer captured with capture instance {}", captureName);
return Arrays.asList(currentChangeTables).stream()
.filter(x -> !x.getCaptureInstance().equals(captureName))
.collect(Collectors.toList())
.toArray(new SqlServerChangeTable[0]);
}
throw exception;
}
private SqlServerChangeTable[] getCdcTablesToQuery(SqlServerOffsetContext offsetContext)
throws SQLException, InterruptedException {
final Set<SqlServerChangeTable> cdcEnabledTables = dataConnection.listOfChangeTables();
if (cdcEnabledTables.isEmpty()) {
LOGGER.warn(
"No table has enabled CDC or security constraints prevents getting the list of change tables");
}
final Map<TableId, List<SqlServerChangeTable>> includeListCdcEnabledTables =
cdcEnabledTables.stream()
.filter(
changeTable -> {
if (connectorConfig
.getTableFilters()
.dataCollectionFilter()
.isIncluded(changeTable.getSourceTableId())) {
return true;
} else {
LOGGER.info(
"CDC is enabled for table {} but the table is not whitelisted by connector",
changeTable);
return false;
}
})
.collect(Collectors.groupingBy(x -> x.getSourceTableId()));
if (includeListCdcEnabledTables.isEmpty()) {
LOGGER.warn(
"No whitelisted table has enabled CDC, whitelisted table list does not contain any table with CDC enabled or no table match the white/blacklist filter(s)");
}
final List<SqlServerChangeTable> tables = new ArrayList<>();
for (List<SqlServerChangeTable> captures : includeListCdcEnabledTables.values()) {
SqlServerChangeTable currentTable = captures.get(0);
if (captures.size() > 1) {
SqlServerChangeTable futureTable;
if (captures.get(0).getStartLsn().compareTo(captures.get(1).getStartLsn()) < 0) {
futureTable = captures.get(1);
} else {
currentTable = captures.get(1);
futureTable = captures.get(0);
}
currentTable.setStopLsn(futureTable.getStartLsn());
futureTable.setSourceTable(dataConnection.getTableSchemaFromTable(futureTable));
tables.add(futureTable);
LOGGER.info(
"Multiple capture instances present for the same table: {} and {}",
currentTable,
futureTable);
}
if (schema.tableFor(currentTable.getSourceTableId()) == null) {
LOGGER.info(
"Table {} is new to be monitored by capture instance {}",
currentTable.getSourceTableId(),
currentTable.getCaptureInstance());
// We need to read the source table schema - nullability information cannot be
// obtained from change table
// There might be no start LSN in the new change table at this time so current
// timestamp is used
offsetContext.event(currentTable.getSourceTableId(), Instant.now());
dispatcher.dispatchSchemaChangeEvent(
currentTable.getSourceTableId(),
new SqlServerSchemaChangeEventEmitter(
offsetContext,
currentTable,
dataConnection.getTableSchemaFromTable(currentTable),
SchemaChangeEventType.CREATE));
}
// If a column was renamed, then the old capture instance had been dropped and a new one
// created. In consequence, a table with out-dated schema might be assigned here.
// A proper value will be set when migration happens.
currentTable.setSourceTable(schema.tableFor(currentTable.getSourceTableId()));
tables.add(currentTable);
}
return tables.toArray(new SqlServerChangeTable[tables.size()]);
}
/**
* @return the log sequence number up until which the connector should query changes from the
* database.
*/
private Lsn getToLsn(
SqlServerConnection connection,
TxLogPosition lastProcessedPosition,
int maxTransactionsPerIteration)
throws SQLException {
if (maxTransactionsPerIteration == 0) {
return connection.getMaxTransactionLsn();
}
final Lsn fromLsn = lastProcessedPosition.getCommitLsn();
if (!fromLsn.isAvailable()) {
return connection.getNthTransactionLsnFromBeginning(maxTransactionsPerIteration);
}
return connection.getNthTransactionLsnFromLast(fromLsn, maxTransactionsPerIteration);
}
/** expose control to the user to stop the connector. */
protected void afterHandleLsn(SqlServerOffsetContext offsetContext) {
// do nothing
}
}

@ -0,0 +1,64 @@
/*
* Copyright 2022 Ververica Inc.
*
* Licensed 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 com.ververica.cdc.connectors.sqlserver;
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import com.ververica.cdc.connectors.base.options.StartupOptions;
import com.ververica.cdc.connectors.sqlserver.source.SqlServerSourceBuilder;
import com.ververica.cdc.connectors.sqlserver.source.SqlServerSourceBuilder.SqlServerIncrementalSource;
import com.ververica.cdc.connectors.sqlserver.source.SqlServerSourceTestBase;
import com.ververica.cdc.debezium.JsonDebeziumDeserializationSchema;
import org.junit.Ignore;
import org.junit.Test;
/** Example Tests for {@link SqlServerIncrementalSource}. */
public class SqlServerParallelSourceExampleTest extends SqlServerSourceTestBase {
@Test
@Ignore("Test ignored because it won't stop and is used for manual test")
public void testSqlServerExampleSource() throws Exception {
initializeSqlServerTable("inventory");
SqlServerIncrementalSource<String> sqlServerSource =
new SqlServerSourceBuilder()
.hostname(MSSQL_SERVER_CONTAINER.getHost())
.port(MSSQL_SERVER_CONTAINER.getFirstMappedPort())
.databaseList("inventory")
.tableList("dbo.products")
.username(MSSQL_SERVER_CONTAINER.getUsername())
.password(MSSQL_SERVER_CONTAINER.getPassword())
.deserializer(new JsonDebeziumDeserializationSchema())
.startupOptions(StartupOptions.initial())
.build();
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
// enable checkpoint
env.enableCheckpointing(3000);
// set the source parallelism to 2
env.fromSource(
sqlServerSource,
WatermarkStrategy.noWatermarks(),
"SqlServerIncrementalSource")
.setParallelism(2)
.print()
.setParallelism(1);
env.execute("Print SqlServer Snapshot + Change Stream");
}
}

@ -16,10 +16,12 @@
package com.ververica.cdc.connectors.sqlserver;
import org.apache.flink.table.planner.factories.TestValuesTableFactory;
import org.apache.flink.test.util.AbstractTestBase;
import org.awaitility.Awaitility;
import org.awaitility.core.ConditionTimeoutException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -70,6 +72,15 @@ public class SqlServerTestBase extends AbstractTestBase {
LOG.info("Containers are started.");
}
@AfterClass
public static void stopContainers() {
LOG.info("Stopping containers...");
if (MSSQL_SERVER_CONTAINER != null) {
MSSQL_SERVER_CONTAINER.stop();
}
LOG.info("Containers are stopped.");
}
protected Connection getJdbcConnection() throws SQLException {
return DriverManager.getConnection(
MSSQL_SERVER_CONTAINER.getJdbcUrl(),
@ -186,4 +197,28 @@ public class SqlServerTestBase extends AbstractTestBase {
throw new RuntimeException(e);
}
}
protected static void waitForSnapshotStarted(String sinkName) throws InterruptedException {
while (sinkSize(sinkName) == 0) {
Thread.sleep(100);
}
}
protected static void waitForSinkSize(String sinkName, int expectedSize)
throws InterruptedException {
while (sinkSize(sinkName) < expectedSize) {
Thread.sleep(100);
}
}
protected static int sinkSize(String sinkName) {
synchronized (TestValuesTableFactory.class) {
try {
return TestValuesTableFactory.getRawResults(sinkName).size();
} catch (IllegalArgumentException e) {
// job is not started yet
return 0;
}
}
}
}

@ -0,0 +1,261 @@
/*
* Copyright 2022 Ververica Inc.
*
* Licensed 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 com.ververica.cdc.connectors.sqlserver.source;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.table.api.TableResult;
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
import org.apache.flink.types.Row;
import org.apache.flink.util.CloseableIterator;
import org.apache.commons.lang3.StringUtils;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.Timeout;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Iterator;
import java.util.List;
import static java.lang.String.format;
import static org.apache.flink.util.Preconditions.checkState;
import static org.testcontainers.containers.MSSQLServerContainer.MS_SQL_SERVER_PORT;
/** IT tests for {@link SqlServerSourceBuilder.SqlServerIncrementalSource}. */
public class SqlServerSourceITCase extends SqlServerSourceTestBase {
private static final Logger LOG = LoggerFactory.getLogger(SqlServerSourceITCase.class);
@Rule public final Timeout timeoutPerTest = Timeout.seconds(300);
@Test
public void testReadSingleTableWithSingleParallelism() throws Exception {
testSqlServerParallelSource(
1, FailoverType.NONE, FailoverPhase.NEVER, new String[] {"dbo.customers"});
}
@Test
public void testReadSingleTableWithMultipleParallelism() throws Exception {
testSqlServerParallelSource(
4, FailoverType.NONE, FailoverPhase.NEVER, new String[] {"dbo.customers"});
}
// Failover tests
@Test
public void testTaskManagerFailoverInSnapshotPhase() throws Exception {
testSqlServerParallelSource(
FailoverType.TM, FailoverPhase.SNAPSHOT, new String[] {"dbo.customers"});
}
@Test
public void testTaskManagerFailoverInBinlogPhase() throws Exception {
testSqlServerParallelSource(
FailoverType.TM, FailoverPhase.STREAM, new String[] {"dbo.customers"});
}
@Test
public void testJobManagerFailoverInSnapshotPhase() throws Exception {
testSqlServerParallelSource(
FailoverType.JM, FailoverPhase.SNAPSHOT, new String[] {"dbo.customers"});
}
@Test
public void testJobManagerFailoverInBinlogPhase() throws Exception {
testSqlServerParallelSource(
FailoverType.JM, FailoverPhase.STREAM, new String[] {"dbo.customers"});
}
@Test
public void testJobManagerFailoverSingleParallelism() throws Exception {
testSqlServerParallelSource(
1, FailoverType.JM, FailoverPhase.SNAPSHOT, new String[] {"dbo.customers"});
}
private void testSqlServerParallelSource(
FailoverType failoverType, FailoverPhase failoverPhase, String[] captureCustomerTables)
throws Exception {
testSqlServerParallelSource(
DEFAULT_PARALLELISM, failoverType, failoverPhase, captureCustomerTables);
}
private void testSqlServerParallelSource(
int parallelism,
FailoverType failoverType,
FailoverPhase failoverPhase,
String[] captureCustomerTables)
throws Exception {
String databaseName = "customer";
initializeSqlServerTable(databaseName);
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
env.setParallelism(parallelism);
env.enableCheckpointing(200L);
env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
String sourceDDL =
format(
"CREATE TABLE customers ("
+ " id INT NOT NULL,"
+ " name STRING,"
+ " address STRING,"
+ " phone_number STRING,"
+ " primary key (id) not enforced"
+ ") WITH ("
+ " 'connector' = 'sqlserver-cdc',"
+ " 'hostname' = '%s',"
+ " 'port' = '%s',"
+ " 'username' = '%s',"
+ " 'password' = '%s',"
+ " 'database-name' = '%s',"
+ " 'table-name' = '%s',"
+ " 'scan.incremental.snapshot.enabled' = 'true',"
+ " 'scan.incremental.snapshot.chunk.size' = '4'"
+ ")",
MSSQL_SERVER_CONTAINER.getHost(),
MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT),
MSSQL_SERVER_CONTAINER.getUsername(),
MSSQL_SERVER_CONTAINER.getPassword(),
databaseName,
getTableNameRegex(captureCustomerTables));
// first step: check the snapshot data
String[] snapshotForSingleTable =
new String[] {
"+I[101, user_1, Shanghai, 123567891234]",
"+I[102, user_2, Shanghai, 123567891234]",
"+I[103, user_3, Shanghai, 123567891234]",
"+I[109, user_4, Shanghai, 123567891234]",
"+I[110, user_5, Shanghai, 123567891234]",
"+I[111, user_6, Shanghai, 123567891234]",
"+I[118, user_7, Shanghai, 123567891234]",
"+I[121, user_8, Shanghai, 123567891234]",
"+I[123, user_9, Shanghai, 123567891234]",
"+I[1009, user_10, Shanghai, 123567891234]",
"+I[1010, user_11, Shanghai, 123567891234]",
"+I[1011, user_12, Shanghai, 123567891234]",
"+I[1012, user_13, Shanghai, 123567891234]",
"+I[1013, user_14, Shanghai, 123567891234]",
"+I[1014, user_15, Shanghai, 123567891234]",
"+I[1015, user_16, Shanghai, 123567891234]",
"+I[1016, user_17, Shanghai, 123567891234]",
"+I[1017, user_18, Shanghai, 123567891234]",
"+I[1018, user_19, Shanghai, 123567891234]",
"+I[1019, user_20, Shanghai, 123567891234]",
"+I[2000, user_21, Shanghai, 123567891234]"
};
tEnv.executeSql(sourceDDL);
TableResult tableResult = tEnv.executeSql("select * from customers");
CloseableIterator<Row> iterator = tableResult.collect();
JobID jobId = tableResult.getJobClient().get().getJobID();
List<String> expectedSnapshotData = new ArrayList<>();
for (int i = 0; i < captureCustomerTables.length; i++) {
expectedSnapshotData.addAll(Arrays.asList(snapshotForSingleTable));
}
// trigger failover after some snapshot splits read finished
if (failoverPhase == FailoverPhase.SNAPSHOT && iterator.hasNext()) {
triggerFailover(
failoverType, jobId, miniClusterResource.getMiniCluster(), () -> sleepMs(100));
}
LOG.info("snapshot data start");
assertEqualsInAnyOrder(
expectedSnapshotData, fetchRows(iterator, expectedSnapshotData.size()));
// second step: check the change stream data
for (String tableId : captureCustomerTables) {
makeFirstPartChangeStreamEvents(databaseName + "." + tableId);
}
if (failoverPhase == FailoverPhase.STREAM) {
triggerFailover(
failoverType, jobId, miniClusterResource.getMiniCluster(), () -> sleepMs(200));
}
for (String tableId : captureCustomerTables) {
makeSecondPartBinlogEvents(databaseName + "." + tableId);
}
String[] binlogForSingleTable =
new String[] {
"-U[103, user_3, Shanghai, 123567891234]",
"+U[103, user_3, Hangzhou, 123567891234]",
"-D[102, user_2, Shanghai, 123567891234]",
"+I[102, user_2, Shanghai, 123567891234]",
"-U[103, user_3, Hangzhou, 123567891234]",
"+U[103, user_3, Shanghai, 123567891234]",
"-U[1010, user_11, Shanghai, 123567891234]",
"+U[1010, user_11, Hangzhou, 123567891234]",
"+I[2001, user_22, Shanghai, 123567891234]",
"+I[2002, user_23, Shanghai, 123567891234]",
"+I[2003, user_24, Shanghai, 123567891234]"
};
List<String> expectedBinlogData = new ArrayList<>();
for (int i = 0; i < captureCustomerTables.length; i++) {
expectedBinlogData.addAll(Arrays.asList(binlogForSingleTable));
}
assertEqualsInAnyOrder(expectedBinlogData, fetchRows(iterator, expectedBinlogData.size()));
tableResult.getJobClient().get().cancel().get();
}
private void makeFirstPartChangeStreamEvents(String tableId) {
executeSql("UPDATE " + tableId + " SET address = 'Hangzhou' where id = 103");
executeSql("DELETE FROM " + tableId + " where id = 102");
executeSql("INSERT INTO " + tableId + " VALUES(102, 'user_2','Shanghai','123567891234')");
executeSql("UPDATE " + tableId + " SET address = 'Shanghai' where id = 103");
}
private void makeSecondPartBinlogEvents(String tableId) {
executeSql("UPDATE " + tableId + " SET address = 'Hangzhou' where id = 1010");
executeSql("INSERT INTO " + tableId + " VALUES(2001, 'user_22','Shanghai','123567891234')");
executeSql("INSERT INTO " + tableId + " VALUES(2002, 'user_23','Shanghai','123567891234')");
executeSql("INSERT INTO " + tableId + " VALUES(2003, 'user_24','Shanghai','123567891234')");
}
private void sleepMs(long millis) {
try {
Thread.sleep(millis);
} catch (InterruptedException ignored) {
}
}
private static List<String> fetchRows(Iterator<Row> iter, int size) {
List<String> rows = new ArrayList<>(size);
while (size > 0 && iter.hasNext()) {
Row row = iter.next();
rows.add(row.toString());
size--;
}
return rows;
}
private String getTableNameRegex(String[] captureCustomerTables) {
checkState(captureCustomerTables.length > 0);
if (captureCustomerTables.length == 1) {
return captureCustomerTables[0];
} else {
// pattern that matches multiple tables
return format("(%s)", StringUtils.join(captureCustomerTables, ","));
}
}
}

@ -0,0 +1,317 @@
/*
* Copyright 2022 Ververica Inc.
*
* Licensed 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 com.ververica.cdc.connectors.sqlserver.source;
import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.highavailability.nonha.embedded.HaLeadershipControl;
import org.apache.flink.runtime.minicluster.MiniCluster;
import org.apache.flink.runtime.minicluster.RpcServiceSharing;
import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
import org.apache.flink.table.planner.factories.TestValuesTableFactory;
import org.apache.flink.test.util.MiniClusterWithClientResource;
import org.apache.flink.util.TestLogger;
import com.ververica.cdc.connectors.sqlserver.SqlServerSource;
import com.ververica.cdc.connectors.sqlserver.SqlServerTestBase;
import org.awaitility.Awaitility;
import org.awaitility.core.ConditionTimeoutException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.Rule;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testcontainers.containers.MSSQLServerContainer;
import org.testcontainers.containers.Network;
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.sql.Connection;
import java.sql.DriverManager;
import java.sql.SQLException;
import java.sql.Statement;
import java.util.Arrays;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.TimeUnit;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
/** Basic class for testing {@link SqlServerSource}. */
public abstract class SqlServerSourceTestBase extends TestLogger {
@ClassRule public static final Network NETWORK = Network.newNetwork();
protected static final Logger LOG = LoggerFactory.getLogger(SqlServerSourceTestBase.class);
public static final MSSQLServerContainer MSSQL_SERVER_CONTAINER =
new MSSQLServerContainer<>("mcr.microsoft.com/mssql/server:2019-latest")
.withPassword("Password!")
.withEnv("MSSQL_AGENT_ENABLED", "true")
.withEnv("MSSQL_PID", "Standard")
.withLogConsumer(new Slf4jLogConsumer(LOG));
protected static final int DEFAULT_PARALLELISM = 4;
private static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)--.*$");
private static final String STATEMENTS_PLACEHOLDER = "#";
private static final String DISABLE_DB_CDC =
"IF EXISTS(select 1 from sys.databases where name='#' AND is_cdc_enabled=1)\n"
+ "EXEC sys.sp_cdc_disable_db";
@Rule
public final MiniClusterWithClientResource miniClusterResource =
new MiniClusterWithClientResource(
new MiniClusterResourceConfiguration.Builder()
.setNumberTaskManagers(1)
.setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM)
.setRpcServiceSharing(RpcServiceSharing.DEDICATED)
.withHaLeadershipControl()
.build());
@BeforeClass
public static void startContainers() {
LOG.info("Starting containers...");
Startables.deepStart(Stream.of(MSSQL_SERVER_CONTAINER)).join();
LOG.info("Containers are started.");
}
@AfterClass
public static void stopContainers() {
LOG.info("Stopping containers...");
if (MSSQL_SERVER_CONTAINER != null) {
MSSQL_SERVER_CONTAINER.stop();
}
LOG.info("Containers are stopped.");
}
private static void dropTestDatabase(Connection connection, String databaseName)
throws SQLException {
try {
Awaitility.await("Disabling CDC")
.atMost(60, TimeUnit.SECONDS)
.until(
() -> {
try {
connection
.createStatement()
.execute(String.format("USE [%s]", databaseName));
} catch (SQLException e) {
// if the database doesn't yet exist, there is no need to
// disable CDC
return true;
}
try {
disableDbCdc(connection, databaseName);
return true;
} catch (SQLException e) {
return false;
}
});
} catch (ConditionTimeoutException e) {
throw new IllegalArgumentException(
String.format("Failed to disable CDC on %s", databaseName), e);
}
connection.createStatement().execute("USE master");
try {
Awaitility.await(String.format("Dropping database %s", databaseName))
.atMost(60, TimeUnit.SECONDS)
.until(
() -> {
try {
String sql =
String.format(
"IF EXISTS(select 1 from sys.databases where name = '%s') DROP DATABASE [%s]",
databaseName, databaseName);
connection.createStatement().execute(sql);
return true;
} catch (SQLException e) {
LOG.warn(
String.format(
"DROP DATABASE %s failed (will be retried): {}",
databaseName),
e.getMessage());
try {
connection
.createStatement()
.execute(
String.format(
"ALTER DATABASE [%s] SET SINGLE_USER WITH ROLLBACK IMMEDIATE;",
databaseName));
} catch (SQLException e2) {
LOG.error("Failed to rollbackimmediately", e2);
}
return false;
}
});
} catch (ConditionTimeoutException e) {
throw new IllegalStateException("Failed to drop test database", e);
}
}
/**
* Disables CDC for a given database, if not already disabled.
*
* @param name the name of the DB, may not be {@code null}
* @throws SQLException if anything unexpected fails
*/
protected static void disableDbCdc(Connection connection, String name) throws SQLException {
Objects.requireNonNull(name);
connection.createStatement().execute(DISABLE_DB_CDC.replace(STATEMENTS_PLACEHOLDER, name));
}
protected static void assertEqualsInAnyOrder(List<String> expected, List<String> actual) {
assertTrue(expected != null && actual != null);
assertEqualsInOrder(
expected.stream().sorted().collect(Collectors.toList()),
actual.stream().sorted().collect(Collectors.toList()));
}
protected static void assertEqualsInOrder(List<String> expected, List<String> actual) {
assertTrue(expected != null && actual != null);
assertEquals(expected.size(), actual.size());
assertArrayEquals(expected.toArray(new String[0]), actual.toArray(new String[0]));
}
protected static void waitForSnapshotStarted(String sinkName) throws InterruptedException {
while (sinkSize(sinkName) == 0) {
Thread.sleep(100);
}
}
protected static void waitForSinkSize(String sinkName, int expectedSize)
throws InterruptedException {
while (sinkSize(sinkName) < expectedSize) {
Thread.sleep(100);
}
}
protected static int sinkSize(String sinkName) {
synchronized (TestValuesTableFactory.class) {
try {
return TestValuesTableFactory.getRawResults(sinkName).size();
} catch (IllegalArgumentException e) {
// job is not started yet
return 0;
}
}
}
protected Connection getJdbcConnection() throws SQLException {
return DriverManager.getConnection(
MSSQL_SERVER_CONTAINER.getJdbcUrl(),
MSSQL_SERVER_CONTAINER.getUsername(),
MSSQL_SERVER_CONTAINER.getPassword());
}
protected void executeSql(String sql) {
try (Connection connection = getJdbcConnection()) {
connection.createStatement().execute(sql);
} catch (SQLException e) {
throw new RuntimeException(e);
}
}
/**
* Executes a JDBC statement using the default jdbc config without autocommitting the
* connection.
*/
protected void initializeSqlServerTable(String sqlFile) {
final String ddlFile = String.format("ddl/%s.sql", sqlFile);
final URL ddlTestFile = SqlServerTestBase.class.getClassLoader().getResource(ddlFile);
assertNotNull("Cannot locate " + ddlFile, ddlTestFile);
try (Connection connection = getJdbcConnection();
Statement statement = connection.createStatement()) {
dropTestDatabase(connection, sqlFile);
final List<String> statements =
Arrays.stream(
Files.readAllLines(Paths.get(ddlTestFile.toURI())).stream()
.map(String::trim)
.filter(x -> !x.startsWith("--") && !x.isEmpty())
.map(
x -> {
final Matcher m =
COMMENT_PATTERN.matcher(x);
return m.matches() ? m.group(1) : x;
})
.collect(Collectors.joining("\n"))
.split(";"))
.collect(Collectors.toList());
for (String stmt : statements) {
statement.execute(stmt);
}
} catch (Exception e) {
throw new RuntimeException(e);
}
}
/** The type of failover. */
protected enum FailoverType {
TM,
JM,
NONE
}
/** The phase of failover. */
protected enum FailoverPhase {
SNAPSHOT,
STREAM,
NEVER
}
protected static void triggerFailover(
FailoverType type, JobID jobId, MiniCluster miniCluster, Runnable afterFailAction)
throws Exception {
switch (type) {
case TM:
restartTaskManager(miniCluster, afterFailAction);
break;
case JM:
triggerJobManagerFailover(jobId, miniCluster, afterFailAction);
break;
case NONE:
break;
default:
throw new IllegalStateException("Unexpected value: " + type);
}
}
protected static void triggerJobManagerFailover(
JobID jobId, MiniCluster miniCluster, Runnable afterFailAction) throws Exception {
final HaLeadershipControl haLeadershipControl = miniCluster.getHaLeadershipControl().get();
haLeadershipControl.revokeJobMasterLeadership(jobId).get();
afterFailAction.run();
haLeadershipControl.grantJobMasterLeadership(jobId).get();
}
protected static void restartTaskManager(MiniCluster miniCluster, Runnable afterFailAction)
throws Exception {
miniCluster.terminateTaskManager(0).get();
afterFailAction.run();
miniCluster.startTaskManager();
}
}

@ -76,7 +76,6 @@ public class SqlServerConnectorITCase extends SqlServerTestBase {
+ " 'username' = '%s',"
+ " 'password' = '%s',"
+ " 'database-name' = '%s',"
+ " 'schema-name' = '%s',"
+ " 'table-name' = '%s'"
+ ")",
MSSQL_SERVER_CONTAINER.getHost(),
@ -84,8 +83,7 @@ public class SqlServerConnectorITCase extends SqlServerTestBase {
MSSQL_SERVER_CONTAINER.getUsername(),
MSSQL_SERVER_CONTAINER.getPassword(),
"inventory",
"dbo",
"products");
"dbo.products");
String sinkDDL =
"CREATE TABLE sink ("
+ " name STRING,"
@ -202,7 +200,6 @@ public class SqlServerConnectorITCase extends SqlServerTestBase {
+ " 'username' = '%s',"
+ " 'password' = '%s',"
+ " 'database-name' = '%s',"
+ " 'schema-name' = '%s',"
+ " 'table-name' = '%s'"
+ ")",
MSSQL_SERVER_CONTAINER.getHost(),
@ -210,8 +207,7 @@ public class SqlServerConnectorITCase extends SqlServerTestBase {
MSSQL_SERVER_CONTAINER.getUsername(),
MSSQL_SERVER_CONTAINER.getPassword(),
"column_type_test",
"dbo",
"full_types");
"dbo.full_types");
String sinkDDL =
"CREATE TABLE sink (\n"
+ " id int NOT NULL,\n"
@ -293,7 +289,6 @@ public class SqlServerConnectorITCase extends SqlServerTestBase {
+ " 'username' = '%s',"
+ " 'password' = '%s',"
+ " 'database-name' = '%s',"
+ " 'schema-name' = '%s',"
+ " 'table-name' = '%s'"
+ ")",
MSSQL_SERVER_CONTAINER.getHost(),
@ -301,8 +296,7 @@ public class SqlServerConnectorITCase extends SqlServerTestBase {
MSSQL_SERVER_CONTAINER.getUsername(),
MSSQL_SERVER_CONTAINER.getPassword(),
"inventory",
"dbo",
"products");
"dbo.products");
String sinkDDL =
"CREATE TABLE sink ("
@ -370,28 +364,4 @@ public class SqlServerConnectorITCase extends SqlServerTestBase {
assertEquals(expected, actual);
result.getJobClient().get().cancel().get();
}
private static void waitForSnapshotStarted(String sinkName) throws InterruptedException {
while (sinkSize(sinkName) == 0) {
Thread.sleep(100);
}
}
private static void waitForSinkSize(String sinkName, int expectedSize)
throws InterruptedException {
while (sinkSize(sinkName) < expectedSize) {
Thread.sleep(100);
}
}
private static int sinkSize(String sinkName) {
synchronized (TestValuesTableFactory.class) {
try {
return TestValuesTableFactory.getRawResults(sinkName).size();
} catch (IllegalArgumentException e) {
// job is not started yet
return 0;
}
}
}
}

@ -26,13 +26,11 @@ import org.apache.flink.table.catalog.ResolvedCatalogTable;
import org.apache.flink.table.catalog.ResolvedSchema;
import org.apache.flink.table.catalog.UniqueConstraint;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.connector.source.ScanTableSource;
import org.apache.flink.table.connector.source.SourceFunctionProvider;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.factories.FactoryUtil;
import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
import com.ververica.cdc.debezium.DebeziumSourceFunction;
import com.ververica.cdc.connectors.base.options.JdbcSourceOptions;
import com.ververica.cdc.connectors.base.options.SourceOptions;
import com.ververica.cdc.connectors.base.options.StartupOptions;
import com.ververica.cdc.debezium.utils.ResolvedSchemaUtils;
import org.junit.Test;
@ -44,7 +42,6 @@ import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
import static com.ververica.cdc.connectors.utils.AssertUtils.assertProducedTypeOfSourceFunction;
import static org.junit.Assert.assertEquals;
/** Test for {@link SqlServerTableSource} created by {@link SqlServerTableFactory}. */
@ -79,8 +76,7 @@ public class SqlServerTableFactoryTest {
private static final String MY_USERNAME = "flinkuser";
private static final String MY_PASSWORD = "flinkpw";
private static final String MY_DATABASE = "myDB";
private static final String MY_SCHEMA = "dbo";
private static final String MY_TABLE = "myTable";
private static final String MY_TABLE = "dbo.myTable";
private static final Properties PROPERTIES = new Properties();
@Test
@ -95,13 +91,24 @@ public class SqlServerTableFactoryTest {
1433,
MY_LOCALHOST,
MY_DATABASE,
MY_SCHEMA,
MY_TABLE,
ZoneId.of("UTC"),
MY_USERNAME,
MY_PASSWORD,
PROPERTIES,
StartupOptions.initial());
StartupOptions.initial(),
SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_ENABLED.defaultValue(),
SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE.defaultValue(),
SourceOptions.CHUNK_META_GROUP_SIZE.defaultValue(),
SourceOptions.SCAN_SNAPSHOT_FETCH_SIZE.defaultValue(),
JdbcSourceOptions.CONNECT_TIMEOUT.defaultValue(),
JdbcSourceOptions.CONNECTION_POOL_SIZE.defaultValue(),
JdbcSourceOptions.CONNECT_MAX_RETRIES.defaultValue(),
JdbcSourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND
.defaultValue(),
JdbcSourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND
.defaultValue(),
null);
assertEquals(expectedSource, actualSource);
}
@ -120,13 +127,24 @@ public class SqlServerTableFactoryTest {
1433,
MY_LOCALHOST,
MY_DATABASE,
MY_SCHEMA,
MY_TABLE,
ZoneId.of("UTC"),
MY_USERNAME,
MY_PASSWORD,
dbzProperties,
StartupOptions.initial());
StartupOptions.initial(),
SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_ENABLED.defaultValue(),
SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE.defaultValue(),
SourceOptions.CHUNK_META_GROUP_SIZE.defaultValue(),
SourceOptions.SCAN_SNAPSHOT_FETCH_SIZE.defaultValue(),
JdbcSourceOptions.CONNECT_TIMEOUT.defaultValue(),
JdbcSourceOptions.CONNECTION_POOL_SIZE.defaultValue(),
JdbcSourceOptions.CONNECT_MAX_RETRIES.defaultValue(),
JdbcSourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND
.defaultValue(),
JdbcSourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND
.defaultValue(),
null);
assertEquals(expectedSource, actualSource);
}
@ -147,25 +165,29 @@ public class SqlServerTableFactoryTest {
1433,
MY_LOCALHOST,
MY_DATABASE,
MY_SCHEMA,
MY_TABLE,
ZoneId.of("UTC"),
MY_USERNAME,
MY_PASSWORD,
PROPERTIES,
StartupOptions.initial());
StartupOptions.initial(),
SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_ENABLED.defaultValue(),
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);
expectedSource.producedDataType = SCHEMA_WITH_METADATA.toSourceRowDataType();
expectedSource.metadataKeys =
Arrays.asList("op_ts", "database_name", "schema_name", "table_name");
assertEquals(expectedSource, actualSource);
ScanTableSource.ScanRuntimeProvider provider =
sqlServerTableSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
DebeziumSourceFunction<RowData> debeziumSourceFunction =
(DebeziumSourceFunction<RowData>)
((SourceFunctionProvider) provider).createSourceFunction();
assertProducedTypeOfSourceFunction(debeziumSourceFunction, expectedSource.producedDataType);
}
private Map<String, String> getAllOptions() {
@ -173,7 +195,6 @@ public class SqlServerTableFactoryTest {
options.put("connector", "sqlserver-cdc");
options.put("hostname", MY_LOCALHOST);
options.put("database-name", MY_DATABASE);
options.put("schema-name", MY_SCHEMA);
options.put("table-name", MY_TABLE);
options.put("username", MY_USERNAME);
options.put("password", MY_PASSWORD);

@ -29,6 +29,7 @@ import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import static org.junit.Assert.assertEquals;
@ -79,7 +80,6 @@ public class SqlServerTimezoneITCase extends SqlServerTestBase {
+ " 'username' = '%s',"
+ " 'password' = '%s',"
+ " 'database-name' = '%s',"
+ " 'schema-name' = '%s',"
+ " 'table-name' = '%s',"
+ " 'server-time-zone'='%s'"
+ ")",
@ -88,8 +88,7 @@ public class SqlServerTimezoneITCase extends SqlServerTestBase {
MSSQL_SERVER_CONTAINER.getUsername(),
MSSQL_SERVER_CONTAINER.getPassword(),
"column_type_test",
"dbo",
"full_types",
"dbo.full_types",
localTimeZone);
String sinkDDL =
"CREATE TABLE sink (\n"
@ -119,17 +118,17 @@ public class SqlServerTimezoneITCase extends SqlServerTestBase {
switch (localTimeZone) {
case "Asia/Shanghai":
expected =
Arrays.asList(
Collections.singletonList(
"+I[0, 2018-07-13, 10:23:45.680, 10:23:45.678, 2018-07-13T11:23:45.340, 2018-07-13T09:23:45.456Z, 2018-07-13T13:23:45.780, 2018-07-13T14:24]");
break;
case "Europe/Berlin":
expected =
Arrays.asList(
Collections.singletonList(
"+I[0, 2018-07-13, 10:23:45.680, 10:23:45.678, 2018-07-13T11:23:45.340, 2018-07-13T03:23:45.456Z, 2018-07-13T13:23:45.780, 2018-07-13T14:24]");
break;
default:
expected =
Arrays.asList(
Collections.singletonList(
"+I[0, 2018-07-13, 10:23:45.680, 10:23:45.678, 2018-07-13T11:23:45.340, 2018-07-13T01:23:45.456Z, 2018-07-13T13:23:45.780, 2018-07-13T14:24]");
break;
}
@ -138,21 +137,4 @@ public class SqlServerTimezoneITCase extends SqlServerTestBase {
result.getJobClient().get().cancel().get();
}
private static void waitForSnapshotStarted(String sinkName) throws InterruptedException {
while (sinkSize(sinkName) == 0) {
Thread.sleep(100);
}
}
private static int sinkSize(String sinkName) {
synchronized (TestValuesTableFactory.class) {
try {
return TestValuesTableFactory.getRawResults(sinkName).size();
} catch (IllegalArgumentException e) {
// job is not started yet
return 0;
}
}
}
}

@ -0,0 +1,85 @@
-- Copyright 2022 Ververica Inc.
--
-- Licensed 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: customer
-- ----------------------------------------------------------------------------------------------------------------
-- Create and populate our users using a single insert with many rows
CREATE DATABASE customer;
USE customer;
EXEC sys.sp_cdc_enable_db;
CREATE TABLE customers (
id INTEGER NOT NULL PRIMARY KEY,
name VARCHAR(255) NOT NULL DEFAULT 'flink',
address VARCHAR(1024),
phone_number VARCHAR(512)
);
INSERT INTO customers
VALUES (101,'user_1','Shanghai','123567891234'),
(102,'user_2','Shanghai','123567891234'),
(103,'user_3','Shanghai','123567891234'),
(109,'user_4','Shanghai','123567891234'),
(110,'user_5','Shanghai','123567891234'),
(111,'user_6','Shanghai','123567891234'),
(118,'user_7','Shanghai','123567891234'),
(121,'user_8','Shanghai','123567891234'),
(123,'user_9','Shanghai','123567891234'),
(1009,'user_10','Shanghai','123567891234'),
(1010,'user_11','Shanghai','123567891234'),
(1011,'user_12','Shanghai','123567891234'),
(1012,'user_13','Shanghai','123567891234'),
(1013,'user_14','Shanghai','123567891234'),
(1014,'user_15','Shanghai','123567891234'),
(1015,'user_16','Shanghai','123567891234'),
(1016,'user_17','Shanghai','123567891234'),
(1017,'user_18','Shanghai','123567891234'),
(1018,'user_19','Shanghai','123567891234'),
(1019,'user_20','Shanghai','123567891234'),
(2000,'user_21','Shanghai','123567891234');
EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'customers', @role_name = NULL, @supports_net_changes = 0;
-- table has same name prefix with 'customers.*'
CREATE TABLE customers_1 (
id INTEGER NOT NULL PRIMARY KEY,
name VARCHAR(255) NOT NULL DEFAULT 'flink',
address VARCHAR(1024),
phone_number VARCHAR(512)
);
INSERT INTO customers_1
VALUES (101,'user_1','Shanghai','123567891234'),
(102,'user_2','Shanghai','123567891234'),
(103,'user_3','Shanghai','123567891234'),
(109,'user_4','Shanghai','123567891234'),
(110,'user_5','Shanghai','123567891234'),
(111,'user_6','Shanghai','123567891234'),
(118,'user_7','Shanghai','123567891234'),
(121,'user_8','Shanghai','123567891234'),
(123,'user_9','Shanghai','123567891234'),
(1009,'user_10','Shanghai','123567891234'),
(1010,'user_11','Shanghai','123567891234'),
(1011,'user_12','Shanghai','123567891234'),
(1012,'user_13','Shanghai','123567891234'),
(1013,'user_14','Shanghai','123567891234'),
(1014,'user_15','Shanghai','123567891234'),
(1015,'user_16','Shanghai','123567891234'),
(1016,'user_17','Shanghai','123567891234'),
(1017,'user_18','Shanghai','123567891234'),
(1018,'user_19','Shanghai','123567891234'),
(1019,'user_20','Shanghai','123567891234'),
(2000,'user_21','Shanghai','123567891234');
EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'customers_1', @role_name = NULL, @supports_net_changes = 0;

@ -55,6 +55,7 @@ under the License.
<include>io.debezium:debezium-core</include>
<include>io.debezium:debezium-connector-sqlserver</include>
<include>com.ververica:flink-connector-debezium</include>
<include>com.ververica:flink-cdc-base</include>
<include>com.ververica:flink-connector-sqlserver-cdc</include>
<include>com.microsoft.sqlserver:*</include>
<include>org.apache.kafka:*</include>

Loading…
Cancel
Save