diff --git a/docs/content/connectors/sqlserver-cdc.md b/docs/content/connectors/sqlserver-cdc.md index a500f57ea9..3ed01905c8 100644 --- a/docs/content/connectors/sqlserver-cdc.md +++ b/docs/content/connectors/sqlserver-cdc.md @@ -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 String Database name of the SQLServer database to monitor. - - schema-name - required - (none) - String - Schema name of the SQLServer database to monitor. - table-name required (none) String - Table name of the SQLServer database to monitor. + Table name of the SQLServer database to monitor, e.g.: "db1.table1" port @@ -167,6 +159,38 @@ Connector Options String The session time zone in database server, e.g. "Asia/Shanghai". + + scan.incremental.snapshot.enabled + optional + true + Boolean + Whether enable parallelism snapshot. + + + chunk-meta.group.size + optional + 1000 + Integer + The group size of chunk meta, if the meta size exceeds the group size, the meta will be divided into multiple groups. + + + chunk-key.even-distribution.factor.lower-bound + optional + 0.05d + Double + 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. + + + chunk-key.even-distribution.factor.upper-bound + optional + 1000.0d + Double + 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. + debezium.* optional @@ -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 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 diff --git a/docs/content/quickstart/sqlserver-tutorial.md b/docs/content/quickstart/sqlserver-tutorial.md index e17fa353e6..dbe317c718 100644 --- a/docs/content/quickstart/sqlserver-tutorial.md +++ b/docs/content/quickstart/sqlserver-tutorial.md @@ -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 ( diff --git "a/docs/content/\345\277\253\351\200\237\344\270\212\346\211\213/sqlserver-tutorial-zh.md" "b/docs/content/\345\277\253\351\200\237\344\270\212\346\211\213/sqlserver-tutorial-zh.md" index 85226a716f..730179e072 100644 --- "a/docs/content/\345\277\253\351\200\237\344\270\212\346\211\213/sqlserver-tutorial-zh.md" +++ "b/docs/content/\345\277\253\351\200\237\344\270\212\346\211\213/sqlserver-tutorial-zh.md" @@ -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 ( diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/utils/SourceRecordUtils.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/utils/SourceRecordUtils.java index 99aca11c3f..9087e520ca 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/utils/SourceRecordUtils.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/utils/SourceRecordUtils.java @@ -99,7 +99,8 @@ public static boolean isSchemaChangeEvent(SourceRecord sourceRecord) { 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; } diff --git a/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/SqlServerE2eITCase.java b/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/SqlServerE2eITCase.java index f2a6a57c00..6e2e744efb 100644 --- a/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/SqlServerE2eITCase.java +++ b/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/SqlServerE2eITCase.java @@ -23,6 +23,7 @@ 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.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 parameters() { + final List flinkVersions = getFlinkVersion(); + List 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 void testSqlServerCDC() throws Exception { " '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 void testSqlServerCDC() throws Exception { 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 void testSqlServerCDC() throws Exception { expectResult, "products_sink", new String[] {"id", "name", "description", "weight"}, - 60000L); + 80000L); } private void initializeSqlServerTable(String sqlFile) { diff --git a/flink-connector-sqlserver-cdc/pom.xml b/flink-connector-sqlserver-cdc/pom.xml index 3364947c6e..c96fe106bb 100644 --- a/flink-connector-sqlserver-cdc/pom.xml +++ b/flink-connector-sqlserver-cdc/pom.xml @@ -30,6 +30,12 @@ under the License. + + com.ververica + flink-cdc-base + ${project.version} + + com.ververica @@ -138,6 +144,7 @@ under the License. test + diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/SqlServerSource.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/SqlServerSource.java index a3302deb62..27148baefe 100644 --- a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/SqlServerSource.java +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/SqlServerSource.java @@ -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 DebeziumSourceFunction build() { 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; diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/SqlServerSourceBuilder.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/SqlServerSourceBuilder.java new file mode 100644 index 0000000000..ea734d49dc --- /dev/null +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/SqlServerSourceBuilder.java @@ -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}. + * + *

Check the Java docs of each individual method to learn more about the settings to build a + * {@link SqlServerIncrementalSource}. + */ +public class SqlServerSourceBuilder { + + private final SqlServerSourceConfigFactory configFactory = new SqlServerSourceConfigFactory(); + + private LsnFactory offsetFactory; + + private SqlServerDialect dialect; + + private DebeziumDeserializationSchema deserializer; + + /** Hostname of the SQL Server database server. */ + public SqlServerSourceBuilder hostname(String hostname) { + this.configFactory.hostname(hostname); + return this; + } + + /** Integer port number of the SqlServer database server. */ + public SqlServerSourceBuilder 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 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 .}. + */ + public SqlServerSourceBuilder 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 username(String username) { + this.configFactory.username(username); + return this; + } + + /** Password to use when connecting to the SqlSever database server. */ + public SqlServerSourceBuilder 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 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 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 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 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 distributionFactorLower(double distributionFactorLower) { + this.configFactory.distributionFactorLower(distributionFactorLower); + return this; + } + + /** The maximum fetch size for per poll when read table snapshot. */ + public SqlServerSourceBuilder 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 connectTimeout(Duration connectTimeout) { + this.configFactory.connectTimeout(connectTimeout); + return this; + } + + /** The max retry times to get connection. */ + public SqlServerSourceBuilder connectMaxRetries(int connectMaxRetries) { + this.configFactory.connectMaxRetries(connectMaxRetries); + return this; + } + + /** The connection pool size. */ + public SqlServerSourceBuilder connectionPoolSize(int connectionPoolSize) { + this.configFactory.connectionPoolSize(connectionPoolSize); + return this; + } + + /** Whether the {@link SqlServerIncrementalSource} should output the schema changes or not. */ + public SqlServerSourceBuilder includeSchemaChanges(boolean includeSchemaChanges) { + this.configFactory.includeSchemaChanges(includeSchemaChanges); + return this; + } + + /** Specifies the startup options. */ + public SqlServerSourceBuilder 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 chunkKeyColumn(String chunkKeyColumn) { + this.configFactory.chunkKeyColumn(chunkKeyColumn); + return this; + } + + /** The Debezium SqlSever connector properties. For example, "snapshot.mode". */ + public SqlServerSourceBuilder 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 deserializer(DebeziumDeserializationSchema deserializer) { + this.deserializer = deserializer; + return this; + } + + /** + * Build the {@link SqlServerIncrementalSource}. + * + * @return a SqlSeverParallelSource with the settings made for this builder. + */ + public SqlServerIncrementalSource build() { + this.offsetFactory = new LsnFactory(); + this.dialect = new SqlServerDialect(configFactory); + return new SqlServerIncrementalSource( + configFactory, checkNotNull(deserializer), offsetFactory, dialect); + } + + /** The {@link JdbcIncrementalSource} implementation for SqlServer. */ + public static class SqlServerIncrementalSource extends JdbcIncrementalSource { + + public SqlServerIncrementalSource( + SqlServerSourceConfigFactory configFactory, + DebeziumDeserializationSchema deserializationSchema, + LsnFactory offsetFactory, + SqlServerDialect dataSourceDialect) { + super(configFactory, deserializationSchema, offsetFactory, dataSourceDialect); + } + + public static SqlServerSourceBuilder builder() { + return new SqlServerSourceBuilder<>(); + } + } +} diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/config/SqlServerSourceConfig.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/config/SqlServerSourceConfig.java new file mode 100644 index 0000000000..fd842a4a6a --- /dev/null +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/config/SqlServerSourceConfig.java @@ -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 databaseList, + List 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(); + } +} diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/config/SqlServerSourceConfigFactory.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/config/SqlServerSourceConfigFactory.java new file mode 100644 index 0000000000..41ebb93bbf --- /dev/null +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/config/SqlServerSourceConfigFactory.java @@ -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 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); + } +} diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/dialect/SqlServerChunkSplitter.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/dialect/SqlServerChunkSplitter.java new file mode 100644 index 0000000000..09bc3d2cf3 --- /dev/null +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/dialect/SqlServerChunkSplitter.java @@ -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 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 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 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 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 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 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 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 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 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; + } +} diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/dialect/SqlServerDialect.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/dialect/SqlServerDialect.java new file mode 100644 index 0000000000..dc12b39f50 --- /dev/null +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/dialect/SqlServerDialect.java @@ -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 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 discoverDataCollectionSchemas(JdbcSourceConfig sourceConfig) { + final List capturedTableIds = discoverDataCollections(sourceConfig); + + try (SqlServerConnection jdbc = + createSqlServerConnection(sourceConfig.getDbzConnectorConfig().getJdbcConfig())) { + // fetch table schemas + Map 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 createFetchTask(SourceSplitBase sourceSplitBase) { + if (sourceSplitBase.isSnapshotSplit()) { + return new SqlServerScanFetchTask(sourceSplitBase.asSnapshotSplit()); + } else { + return new SqlServerStreamFetchTask(sourceSplitBase.asStreamSplit()); + } + } +} diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/dialect/SqlServerPooledDataSourceFactory.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/dialect/SqlServerPooledDataSourceFactory.java new file mode 100644 index 0000000000..848b835961 --- /dev/null +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/dialect/SqlServerPooledDataSourceFactory.java @@ -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); + } +} diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/dialect/SqlServerSchema.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/dialect/SqlServerSchema.java new file mode 100644 index 0000000000..37d5064db6 --- /dev/null +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/dialect/SqlServerSchema.java @@ -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 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 tableIdSet = new HashSet<>(); + tableIdSet.add(tableId); + + final Map 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); + } +} diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/offset/LsnFactory.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/offset/LsnFactory.java new file mode 100644 index 0000000000..3d6993d68f --- /dev/null +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/offset/LsnFactory.java @@ -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 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; + } +} diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/offset/LsnOffset.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/offset/LsnOffset.java new file mode 100644 index 0000000000..d8f6e05fe9 --- /dev/null +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/offset/LsnOffset.java @@ -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 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); + } +} diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/reader/fetch/SqlServerScanFetchTask.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/reader/fetch/SqlServerScanFetchTask.java new file mode 100644 index 0000000000..5efee9276a --- /dev/null +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/reader/fetch/SqlServerScanFetchTask.java @@ -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 { + + 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 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; + } + } +} diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/reader/fetch/SqlServerSourceFetchTaskContext.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/reader/fetch/SqlServerSourceFetchTaskContext.java new file mode 100644 index 0000000000..5f7c197577 --- /dev/null +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/reader/fetch/SqlServerSourceFetchTaskContext.java @@ -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 queue; + private SqlServerTaskContext taskContext; + private TopicSelector 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() + .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 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 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); + } + } +} diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/reader/fetch/SqlServerStreamFetchTask.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/reader/fetch/SqlServerStreamFetchTask.java new file mode 100644 index 0000000000..d53c7f94ac --- /dev/null +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/reader/fetch/SqlServerStreamFetchTask.java @@ -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 { + + 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; + } + } +} diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/utils/SqlServerConnectionUtils.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/utils/SqlServerConnectionUtils.java new file mode 100644 index 0000000000..f84a4d842e --- /dev/null +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/utils/SqlServerConnectionUtils.java @@ -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 listTables(JdbcConnection jdbc, RelationalTableFilters tableFilters) + throws SQLException { + final List capturedTableIds = new ArrayList<>(); + // ------------------- + // READ DATABASE NAMES + // ------------------- + // Get the list of databases ... + LOG.info("Read list of available databases"); + final List 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; + } +} diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/utils/SqlServerTypeUtils.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/utils/SqlServerTypeUtils.java new file mode 100644 index 0000000000..74a6ebfde3 --- /dev/null +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/utils/SqlServerTypeUtils.java @@ -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())); + } + } +} diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/utils/SqlServerUtils.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/utils/SqlServerUtils.java new file mode 100644 index 0000000000..97b59ff239 --- /dev/null +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/utils/SqlServerUtils.java @@ -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 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 offset) { + Map offsetStrMap = new HashMap<>(); + for (Map.Entry 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 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 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 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 condition, + Optional 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 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 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(); + } +} diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerTableFactory.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerTableFactory.java index 0d3e2b9174..ec8c9a443c 100644 --- a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerTableFactory.java +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerTableFactory.java @@ -25,13 +25,28 @@ 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 SCHEMA_NAME = - ConfigOptions.key("schema-name") - .stringType() - .noDefaultValue() - .withDescription("Schema name of the SqlServer database to monitor."); - private static final ConfigOption TABLE_NAME = ConfigOptions.key("table-name") .stringType() @@ -106,7 +114,6 @@ public DynamicTableSource createDynamicTableSource(Context context) { 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 DynamicTableSource createDynamicTableSource(Context context) { 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 Set> requiredOptions() { 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 Set> optionalOptions() { 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 @@ private static StartupOptions getStartupOptions(ReadableConfig config) { 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 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)); + } } diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerTableSource.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerTableSource.java index ccc876e1fc..4dc9225d43 100644 --- a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerTableSource.java +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerTableSource.java @@ -22,17 +22,24 @@ 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 SqlServerTableSource( 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 ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { .setUserDefinedConverterFactory( SqlServerDeserializationConverterFactory.instance()) .build(); - DebeziumSourceFunction sourceFunction = - SqlServerSource.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 sqlServerChangeEventSource = + SqlServerSourceBuilder.SqlServerIncrementalSource.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 sourceFunction = + SqlServerSource.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 DynamicTableSource copy() { 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 boolean equals(Object o) { && 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 boolean equals(Object o) { && 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 int hashCode() { port, hostname, database, - schemaName, tableName, serverTimeZone, username, @@ -210,7 +286,17 @@ public int hashCode() { dbzProperties, startupOptions, producedDataType, - metadataKeys); + metadataKeys, + enableParallelRead, + splitSize, + splitMetaGroupSize, + fetchSize, + connectTimeout, + connectMaxRetries, + connectionPoolSize, + distributionFactorUpper, + distributionFactorLower, + chunkKeyColumn); } @Override diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/table/StartupOptions.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/table/StartupOptions.java deleted file mode 100644 index 560497d265..0000000000 --- a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/table/StartupOptions.java +++ /dev/null @@ -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); - } -} diff --git a/flink-connector-sqlserver-cdc/src/main/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSource.java b/flink-connector-sqlserver-cdc/src/main/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSource.java new file mode 100644 index 0000000000..d934979e72 --- /dev/null +++ b/flink-connector-sqlserver-cdc/src/main/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSource.java @@ -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. + * + *

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. + * + *

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. + * + *

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 { + + 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 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 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 schemaChangeCheckpoints = + new PriorityQueue<>((x, y) -> x.getStopLsn().compareTo(y.getStopLsn())); + try { + final AtomicReference tablesSlot = + new AtomicReference(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 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 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> 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 tables = new ArrayList<>(); + for (List 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 + } +} diff --git a/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/SqlServerParallelSourceExampleTest.java b/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/SqlServerParallelSourceExampleTest.java new file mode 100644 index 0000000000..64ab60e81b --- /dev/null +++ b/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/SqlServerParallelSourceExampleTest.java @@ -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 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"); + } +} diff --git a/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/SqlServerTestBase.java b/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/SqlServerTestBase.java index 0bde58c586..f71cadaa5b 100644 --- a/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/SqlServerTestBase.java +++ b/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/SqlServerTestBase.java @@ -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 static void startContainers() { 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 @@ protected void initializeSqlServerTable(String sqlFile) { 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; + } + } + } } diff --git a/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/source/SqlServerSourceITCase.java b/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/source/SqlServerSourceITCase.java new file mode 100644 index 0000000000..8d1ce35760 --- /dev/null +++ b/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/source/SqlServerSourceITCase.java @@ -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 iterator = tableResult.collect(); + JobID jobId = tableResult.getJobClient().get().getJobID(); + List 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 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 fetchRows(Iterator iter, int size) { + List 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, ",")); + } + } +} diff --git a/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/source/SqlServerSourceTestBase.java b/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/source/SqlServerSourceTestBase.java new file mode 100644 index 0000000000..202dec1d72 --- /dev/null +++ b/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/source/SqlServerSourceTestBase.java @@ -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 expected, List actual) { + assertTrue(expected != null && actual != null); + assertEqualsInOrder( + expected.stream().sorted().collect(Collectors.toList()), + actual.stream().sorted().collect(Collectors.toList())); + } + + protected static void assertEqualsInOrder(List expected, List 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 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(); + } +} diff --git a/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerConnectorITCase.java b/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerConnectorITCase.java index 2608706f6c..6943941f55 100644 --- a/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerConnectorITCase.java +++ b/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerConnectorITCase.java @@ -76,7 +76,6 @@ public void testConsumingAllEvents() + " 'username' = '%s'," + " 'password' = '%s'," + " 'database-name' = '%s'," - + " 'schema-name' = '%s'," + " 'table-name' = '%s'" + ")", MSSQL_SERVER_CONTAINER.getHost(), @@ -84,8 +83,7 @@ public void testConsumingAllEvents() 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 void testAllTypes() throws Throwable { + " 'username' = '%s'," + " 'password' = '%s'," + " 'database-name' = '%s'," - + " 'schema-name' = '%s'," + " 'table-name' = '%s'" + ")", MSSQL_SERVER_CONTAINER.getHost(), @@ -210,8 +207,7 @@ public void testAllTypes() throws Throwable { 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 void testMetadataColumns() throws Throwable { + " 'username' = '%s'," + " 'password' = '%s'," + " 'database-name' = '%s'," - + " 'schema-name' = '%s'," + " 'table-name' = '%s'" + ")", MSSQL_SERVER_CONTAINER.getHost(), @@ -301,8 +296,7 @@ public void testMetadataColumns() throws Throwable { MSSQL_SERVER_CONTAINER.getUsername(), MSSQL_SERVER_CONTAINER.getPassword(), "inventory", - "dbo", - "products"); + "dbo.products"); String sinkDDL = "CREATE TABLE sink (" @@ -370,28 +364,4 @@ public void testMetadataColumns() throws Throwable { 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; - } - } - } } diff --git a/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerTableFactoryTest.java b/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerTableFactoryTest.java index 414275a285..82df653332 100644 --- a/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerTableFactoryTest.java +++ b/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerTableFactoryTest.java @@ -26,13 +26,11 @@ 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.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 void testCommonProperties() { 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 void testOptionalProperties() { 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 void testMetadataColumns() { 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 debeziumSourceFunction = - (DebeziumSourceFunction) - ((SourceFunctionProvider) provider).createSourceFunction(); - assertProducedTypeOfSourceFunction(debeziumSourceFunction, expectedSource.producedDataType); } private Map getAllOptions() { @@ -173,7 +195,6 @@ private Map getAllOptions() { 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); diff --git a/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerTimezoneITCase.java b/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerTimezoneITCase.java index 87e086cdef..1d6e05aa41 100644 --- a/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerTimezoneITCase.java +++ b/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/table/SqlServerTimezoneITCase.java @@ -29,6 +29,7 @@ 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 void testTemporalTypesWithTimeZone() throws Exception { + " 'username' = '%s'," + " 'password' = '%s'," + " 'database-name' = '%s'," - + " 'schema-name' = '%s'," + " 'table-name' = '%s'," + " 'server-time-zone'='%s'" + ")", @@ -88,8 +88,7 @@ public void testTemporalTypesWithTimeZone() throws Exception { 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 void testTemporalTypesWithTimeZone() throws Exception { 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 void testTemporalTypesWithTimeZone() throws Exception { 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; - } - } - } } diff --git a/flink-connector-sqlserver-cdc/src/test/resources/ddl/customer.sql b/flink-connector-sqlserver-cdc/src/test/resources/ddl/customer.sql new file mode 100644 index 0000000000..668f37a398 --- /dev/null +++ b/flink-connector-sqlserver-cdc/src/test/resources/ddl/customer.sql @@ -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; diff --git a/flink-sql-connector-sqlserver-cdc/pom.xml b/flink-sql-connector-sqlserver-cdc/pom.xml index 5a1525395d..1f5c629807 100644 --- a/flink-sql-connector-sqlserver-cdc/pom.xml +++ b/flink-sql-connector-sqlserver-cdc/pom.xml @@ -55,6 +55,7 @@ under the License. io.debezium:debezium-core io.debezium:debezium-connector-sqlserver com.ververica:flink-connector-debezium + com.ververica:flink-cdc-base com.ververica:flink-connector-sqlserver-cdc com.microsoft.sqlserver:* org.apache.kafka:*