From 8fa0754ac156ead918362d64eebc6493c7ae4bed Mon Sep 17 00:00:00 2001 From: sunxiaojian Date: Fri, 17 Nov 2023 17:06:16 +0800 Subject: [PATCH] Supports SQLServer cdc --- docs/content/flink/cdc-ingestion/overview.md | 2 + .../flink/cdc-ingestion/sqlserver-cdc.md | 211 +++++ .../generated/sqlserver_sync_database.html | 93 +++ .../generated/sqlserver_sync_table.html | 81 ++ paimon-flink/paimon-flink-cdc/pom.xml | 15 + .../action/cdc/CdcActionCommonUtils.java | 1 + .../action/cdc/CdcMetadataProcessor.java | 7 + .../flink/action/cdc/SyncJobHandler.java | 34 +- .../cdc/mysql/MySqlSyncDatabaseAction.java | 2 +- .../action/cdc/schema/JdbcSchemasInfo.java | 8 +- .../cdc/schema/UnmergedJdbcTableInfo.java | 13 +- .../cdc/sqlserver/SqlServerActionUtils.java | 301 ++++++++ .../cdc/sqlserver/SqlServerRecordParser.java | 286 +++++++ .../cdc/sqlserver/SqlServerSourceOptions.java | 84 ++ .../SqlServerSyncDatabaseAction.java | 250 ++++++ .../SqlServerSyncDatabaseActionFactory.java | 191 +++++ .../sqlserver/SqlServerSyncTableAction.java | 104 +++ .../SqlServerSyncTableActionFactory.java | 132 ++++ .../cdc/sqlserver/SqlServerTypeUtils.java | 174 +++++ .../org.apache.paimon.factories.Factory | 3 + .../flink/action/cdc/CdcActionITCaseBase.java | 9 + .../cdc/mysql/MySqlActionITCaseBase.java | 7 +- .../cdc/sqlserver/LicenseAcceptanceUtils.java | 60 ++ .../sqlserver/MSSQLServerContainerExtend.java | 38 + .../sqlserver/SqlServerActionITCaseBase.java | 146 ++++ .../SqlServerSyncDatabaseActionITCase.java | 497 ++++++++++++ .../SqlServerSyncDatabaseTableListITCase.java | 93 +++ .../SqlServerSyncTableActionITCase.java | 731 ++++++++++++++++++ .../licenses/container-license-acceptance.txt | 16 + .../sqlserver/sync_database_setup.sql | 326 ++++++++ .../resources/sqlserver/sync_table_setup.sql | 258 +++++++ .../sqlserver/tablelist_test_setup.sql | 75 ++ 32 files changed, 4237 insertions(+), 11 deletions(-) create mode 100644 docs/content/flink/cdc-ingestion/sqlserver-cdc.md create mode 100644 docs/layouts/shortcodes/generated/sqlserver_sync_database.html create mode 100644 docs/layouts/shortcodes/generated/sqlserver_sync_table.html create mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerActionUtils.java create mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerRecordParser.java create mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSourceOptions.java create mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncDatabaseAction.java create mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncDatabaseActionFactory.java create mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncTableAction.java create mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncTableActionFactory.java create mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerTypeUtils.java create mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/LicenseAcceptanceUtils.java create mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/MSSQLServerContainerExtend.java create mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerActionITCaseBase.java create mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncDatabaseActionITCase.java create mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncDatabaseTableListITCase.java create mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncTableActionITCase.java create mode 100644 paimon-flink/paimon-flink-cdc/src/test/resources/META-INF/licenses/container-license-acceptance.txt create mode 100644 paimon-flink/paimon-flink-cdc/src/test/resources/sqlserver/sync_database_setup.sql create mode 100644 paimon-flink/paimon-flink-cdc/src/test/resources/sqlserver/sync_table_setup.sql create mode 100644 paimon-flink/paimon-flink-cdc/src/test/resources/sqlserver/tablelist_test_setup.sql diff --git a/docs/content/flink/cdc-ingestion/overview.md b/docs/content/flink/cdc-ingestion/overview.md index 85b55f6775d7..506552abff43 100644 --- a/docs/content/flink/cdc-ingestion/overview.md +++ b/docs/content/flink/cdc-ingestion/overview.md @@ -40,6 +40,8 @@ We currently support the following sync ways: 7. MongoDB Synchronizing Database: synchronize the whole MongoDB database into one Paimon database. 8. Pulsar Synchronizing Table: synchronize one Pulsar topic's table into one Paimon table. 9. Pulsar Synchronizing Database: synchronize one Pulsar topic containing multiple tables or multiple topics containing one table each into one Paimon database. +10. SQLServer Synchronizing Table: synchronize one or multiple tables from SQLServer into one Paimon table. +11. SQLServer Synchronizing Database: synchronize the whole SQLServer database into one Paimon database. ## What is Schema Evolution diff --git a/docs/content/flink/cdc-ingestion/sqlserver-cdc.md b/docs/content/flink/cdc-ingestion/sqlserver-cdc.md new file mode 100644 index 000000000000..e33227da9074 --- /dev/null +++ b/docs/content/flink/cdc-ingestion/sqlserver-cdc.md @@ -0,0 +1,211 @@ +--- +title: "SQLServer CDC" +weight: 2 +type: docs +aliases: +- /cdc-ingestion/sqlserver-cdc.html +--- + + +# SQLServer CDC + +Paimon supports synchronizing changes from different databases using change data capture (CDC). This feature requires Flink and its [CDC connectors](https://ververica.github.io/flink-cdc-connectors/). + +## Prepare CDC Bundled Jar + +``` +flink-sql-connector-sqlserver-cdc-*.jar +``` + +## Synchronizing Tables + +By using [SqlServerSyncTableAction](/docs/{{< param Branch >}}/api/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncTableAction) in a Flink DataStream job or directly through `flink run`, users can synchronize one or multiple tables from SQLServer into one Paimon table. + +To use this feature through `flink run`, run the following shell command. + +```bash +/bin/flink run \ + /path/to/paimon-flink-action-{{< version >}}.jar \ + sqlserver-sync-table + --warehouse \ + --database \ + --table \ + [--partition-keys ] \ + [--primary-keys ] \ + [--type-mapping ] \ + [--computed-column <'column-name=expr-name(args[, ...])'> [--computed-column ...]] \ + [--metadata-column ] \ + [--sqlserver-conf [--sqlserver-conf ...]] \ + [--catalog-conf [--catalog-conf ...]] \ + [--table-conf [--table-conf ...]] +``` + +{{< generated/sqlserver_sync_table >}} + +Currently, only one database is supported for synchronization. Regular matching of 'database name' is not supported. + +If the Paimon table you specify does not exist, this action will automatically create the table. Its schema will be derived from all specified SQLServer tables. If the Paimon table already exists, its schema will be compared against the schema of all specified SQLServer tables. + +Example 1: synchronize tables into one Paimon table + +```bash +/bin/flink run \ + /path/to/paimon-flink-action-{{< version >}}.jar \ + sqlserver-sync-table \ + --warehouse hdfs:///path/to/warehouse \ + --database test_db \ + --table test_table \ + --partition-keys pt \ + --primary-keys pt,uid \ + --computed-column '_year=year(age)' \ + --sqlserver-conf hostname=127.0.0.1 \ + --sqlserver-conf username=root \ + --sqlserver-conf password=123456 \ + --sqlserver-conf database-name='source_db' \ + --sqlserver-conf schema-name='dbo' \ + --sqlserver-conf table-name='dbo.source_table1|dbo.source_table2' \ + --catalog-conf metastore=hive \ + --catalog-conf uri=thrift://hive-metastore:9083 \ + --table-conf bucket=4 \ + --table-conf changelog-producer=input \ + --table-conf sink.parallelism=4 +``` + +As example shows, the sqlserver-conf's table-name supports regular expressions to monitor multiple tables that satisfy +the regular expressions. The schemas of all the tables will be merged into one Paimon table schema. + +Example 2: synchronize shards into one Paimon table + +You can also use regular expressions to set the "schema_name" to capture multiple schemas. A typical scenario is to split the table "source_table" into databases "source_dbo1" and "source_dbo2"..., Then all the data of "source_table" can be synchronized to a Paimon table. + +```bash +/bin/flink run \ + /path/to/paimon-flink-action-{{< version >}}.jar \ + mysql-sync-table \ + --warehouse hdfs:///path/to/warehouse \ + --database test_db \ + --table test_table \ + --partition-keys pt \ + --primary-keys pt,uid \ + --computed-column '_year=year(age)' \ + --sqlserver-conf hostname=127.0.0.1 \ + --sqlserver-conf username=root \ + --sqlserver-conf password=123456 \ + --sqlserver-conf database-name='source_db' \ + --sqlserver-conf schema-name='source_dbo.+' \ + --sqlserver-conf table-name='source_table' \ + --catalog-conf metastore=hive \ + --catalog-conf uri=thrift://hive-metastore:9083 \ + --table-conf bucket=4 \ + --table-conf changelog-producer=input \ + --table-conf sink.parallelism=4 +``` + +## Synchronizing Databases + +By using [SqlServerSyncDatabaseAction](/docs/{{< param Branch >}}/api/java/org/apache/paimon/flink/action/cdc/mysql/SqlServerSyncDatabaseAction) in a Flink DataStream job or directly through `flink run`, users can synchronize the whole SQLServer database into one Paimon database. + +To use this feature through `flink run`, run the following shell command. + +```bash +/bin/flink run \ + /path/to/paimon-flink-action-{{< version >}}.jar \ + sqlserver-sync-database + --warehouse \ + --database \ + [--ignore-incompatible ] \ + [--merge-shards ] \ + [--table-prefix ] \ + [--table-suffix ] \ + [--including-tables ] \ + [--excluding-tables ] \ + [--mode ] \ + [--metadata-column ] \ + [--type-mapping ] \ + [--sqlserver-conf [--sqlserver-conf ...]] \ + [--catalog-conf [--catalog-conf ...]] \ + [--table-conf [--table-conf ...]] +``` + +{{< generated/sqlserver_sync_database >}} + +Currently, only one database is supported for synchronization. Regular matching of 'database_name' is not supported + +Only tables with primary keys will be synchronized. + +For each SQLServer table to be synchronized, if the corresponding Paimon table does not exist, this action will automatically create the table. Its schema will be derived from all specified SQLServer tables. If the Paimon table already exists, its schema will be compared against the schema of all specified SQLServer tables. + +Example 1: synchronize entire database + +```bash +/bin/flink run \ + /path/to/paimon-flink-action-{{< version >}}.jar \ + sqlserver-sync-database \ + --warehouse hdfs:///path/to/warehouse \ + --database test_db \ + --sqlserver-conf hostname=127.0.0.1 \ + --sqlserver-conf username=root \ + --sqlserver-conf password=123456 \ + --sqlserver-conf database-name=source_db \ + --sqlserver-conf schema-name=dbo \ + --catalog-conf metastore=hive \ + --catalog-conf uri=thrift://hive-metastore:9083 \ + --table-conf bucket=4 \ + --table-conf changelog-producer=input \ + --table-conf sink.parallelism=4 +``` + +Example 2: synchronize and merge multiple shards + +Let's say you have multiple schema shards `schema1`, `schema2`, ... and each schema has tables `tbl1`, `tbl2`, .... You can +synchronize all the `schema.+.tbl.+` into tables `test_db.tbl1`, `test_db.tbl2` ... by following command: + +```bash +/bin/flink run \ + /path/to/paimon-flink-action-{{< version >}}.jar \ + sqlserver-sync-database \ + --warehouse hdfs:///path/to/warehouse \ + --database test_db \ + --sqlserver-conf hostname=127.0.0.1 \ + --sqlserver-conf username=root \ + --sqlserver-conf password=123456 \ + --sqlserver-conf database-name='source_db' \ + --sqlserver-conf schema-name='db.+' \ + --catalog-conf metastore=hive \ + --catalog-conf uri=thrift://hive-metastore:9083 \ + --table-conf bucket=4 \ + --table-conf changelog-producer=input \ + --table-conf sink.parallelism=4 \ + --including-tables 'tbl.+' +``` + +By setting schema-name to a regular expression, the synchronization job will capture all tables under matched schemas +and merge tables of the same name into one table. + +{{< hint info >}} +You can set `--merge-shards false` to prevent merging shards. The synchronized tables will be named to 'databaseName_tableName' +to avoid potential name conflict. +{{< /hint >}} + +## FAQ + +1. Chinese characters in records ingested from MySQL are garbled. +* Try to set `env.java.opts: -Dfile.encoding=UTF-8` in `flink-conf.yaml` +(the option is changed to `env.java.opts.all` since Flink-1.17). \ No newline at end of file diff --git a/docs/layouts/shortcodes/generated/sqlserver_sync_database.html b/docs/layouts/shortcodes/generated/sqlserver_sync_database.html new file mode 100644 index 000000000000..10a123f1aa05 --- /dev/null +++ b/docs/layouts/shortcodes/generated/sqlserver_sync_database.html @@ -0,0 +1,93 @@ +{{/* +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. +*/}} +{{ $ref := ref . "maintenance/configurations.md" }} + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
ConfigurationDescription
--warehouse
The path to Paimon warehouse.
--database
The database name in Paimon catalog.
--ignore-incompatible
It is default false, in this case, if SQLServer table name exists in Paimon and their schema is incompatible,an exception will be thrown. You can specify it to true explicitly to ignore the incompatible tables and exception.
--merge-shards
It is default true, in this case, if some tables in different schemas have the same name, their table schemas will be merged and their records will be synchronized into one Paimon table. Otherwise, each table's records will be synchronized to a corresponding Paimon table, and the Paimon table will be named to 'databaseName_schemaName_tableName' to avoid potential name conflict.
--table-prefix
The prefix of all Paimon tables to be synchronized. For example, if you want all synchronized tables to have "ods_" as prefix, you can specify "--table-prefix ods_".
--table-suffix
The suffix of all Paimon tables to be synchronized. The usage is same as "--table-prefix".
--including-tables
It is used to specify which source tables are to be synchronized. You must use '|' to separate multiple tables.Because '|' is a special character, a comma is required, for example: 'a|b|c'.Regular expression is supported, for example, specifying "--including-tables test|paimon.*" means to synchronize table 'test' and all tables start with 'paimon'.
--excluding-tables
It is used to specify which source tables are not to be synchronized. The usage is same as "--including-tables". "--excluding-tables" has higher priority than "--including-tables" if you specified both.
--mode
It is used to specify synchronization mode.
Possible values:
  • "divided" (the default mode if you haven't specified one): start a sink for each table, the synchronization of the new table requires restarting the job.
  • "combined": start a single combined sink for all tables, the new table will be automatically synchronized.
--metadata-column
--metadata-column is used to specify which metadata columns to include in the output schema of the connector. Metadata columns provide additional information related to the source data, such as the `table_name`, `schema_name`, `database_name`, and `op_ts`. Each configuration should be specified in the format "key=value". See its document for a complete list of available metadata.
--type-mapping
It is used to specify how to map SQLServer data type to Paimon type.
+ Supported options: +
    +
  • "tinyint1-not-bool": maps SQLServer TINYINT(1) to TINYINT instead of BOOLEAN.
  • +
  • "to-nullable": ignores all NOT NULL constraints (except for primary keys).
  • +
  • "to-string": maps all SQLServer types to STRING.
  • +
  • "char-to-string": maps SQLServer CHAR(length)/VARCHAR(length)/NCHAR(length)/NVARCHAR(length) types to STRING.
  • +
+
--sqlserver-conf
The configuration for Flink CDC SQLServer sources. Each configuration should be specified in the format "key=value". hostname, username, password, database-name, schema-name and table-name are required configurations, others are optional. See its document for a complete list of configurations.
--catalog-conf
The configuration for Paimon catalog. Each configuration should be specified in the format "key=value". See here for a complete list of catalog configurations.
--table-conf
The configuration for Paimon table sink. Each configuration should be specified in the format "key=value". See here for a complete list of table configurations.
\ No newline at end of file diff --git a/docs/layouts/shortcodes/generated/sqlserver_sync_table.html b/docs/layouts/shortcodes/generated/sqlserver_sync_table.html new file mode 100644 index 000000000000..9a608ef7dae8 --- /dev/null +++ b/docs/layouts/shortcodes/generated/sqlserver_sync_table.html @@ -0,0 +1,81 @@ +{{/* +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. +*/}} +{{ $ref := ref . "maintenance/configurations.md" }} + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
ConfigurationDescription
--warehouse
The path to Paimon warehouse.
--database
The database name in Paimon catalog.
--table
The Paimon table name.
--partition-keys
The partition keys for Paimon table. If there are multiple partition keys, connect them with comma, for example "dt,hh,mm".
--primary-keys
The primary keys for Paimon table. If there are multiple primary keys, connect them with comma, for example "buyer_id,seller_id".
--type-mapping
It is used to specify how to map SQLServer data type to Paimon type.
+ Supported options: +
    +
  • "tinyint1-not-bool": maps SQLServer TINYINT(1) to TINYINT instead of BOOLEAN.
  • +
  • "to-nullable": ignores all NOT NULL constraints (except for primary keys).
  • +
  • "to-string": maps all SQLServer types to STRING.
  • +
  • "char-to-string": maps SQLServer CHAR(length)/VARCHAR(length)/NCHAR(length)/NVARCHAR(length) types to STRING.
  • +
+
--computed-column
The definitions of computed columns. The argument field is from SQLServer table field name. See here for a complete list of configurations.
--metadata-column
--metadata-column is used to specify which metadata columns to include in the output schema of the connector. Metadata columns provide additional information related to the source data, such as the `table_name`,`schema_name`, `database_name`, and `op_ts`. Each configuration should be specified in the format "key=value". See its document for a complete list of available metadata.
--sqlserver-conf
The configuration for Flink CDC SQLServer sources. Each configuration should be specified in the format "key=value". hostname, username, password, database-name, schema-name and table-name are required configurations, others are optional. See its document for a complete list of configurations.
--catalog-conf
The configuration for Paimon catalog. Each configuration should be specified in the format "key=value". See here for a complete list of catalog configurations.
--table-conf
The configuration for Paimon table sink. Each configuration should be specified in the format "key=value". See here for a complete list of table configurations.
\ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/pom.xml b/paimon-flink/paimon-flink-cdc/pom.xml index 1b18833a466d..7d9ad260668e 100644 --- a/paimon-flink/paimon-flink-cdc/pom.xml +++ b/paimon-flink/paimon-flink-cdc/pom.xml @@ -41,6 +41,7 @@ under the License. 2.2.0 2.9.0 1.19.1 + 1.19.1 4.0.0-1.17 @@ -89,6 +90,13 @@ under the License. provided + + com.ververica + flink-connector-sqlserver-cdc + ${flink.cdc.version} + provided + + org.apache.flink flink-connector-kafka @@ -263,6 +271,13 @@ under the License. + + + org.testcontainers + mssqlserver + ${sqlserver.testcontainers.version} + test + diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcActionCommonUtils.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcActionCommonUtils.java index bc41683fd3ec..b9354ebb2d73 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcActionCommonUtils.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcActionCommonUtils.java @@ -55,6 +55,7 @@ public class CdcActionCommonUtils { public static final String MONGODB_CONF = "mongodb_conf"; public static final String MYSQL_CONF = "mysql_conf"; public static final String POSTGRES_CONF = "postgres_conf"; + public static final String SQLSERVER_CONF = "sqlserver_conf"; public static final String PULSAR_CONF = "pulsar_conf"; public static final String TABLE_PREFIX = "table_prefix"; public static final String TABLE_SUFFIX = "table_suffix"; diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcMetadataProcessor.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcMetadataProcessor.java index 9fdd7a4377e7..828509cd191e 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcMetadataProcessor.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcMetadataProcessor.java @@ -49,6 +49,13 @@ public enum CdcMetadataProcessor { new CdcMetadataConverter.DatabaseNameConverter(), new CdcMetadataConverter.TableNameConverter(), new CdcMetadataConverter.SchemaNameConverter(), + new CdcMetadataConverter.OpTsConverter()), + + SQLSERVER_METADATA_PROCESSOR( + SyncJobHandler.SourceType.SQLSERVER, + new CdcMetadataConverter.DatabaseNameConverter(), + new CdcMetadataConverter.TableNameConverter(), + new CdcMetadataConverter.SchemaNameConverter(), new CdcMetadataConverter.OpTsConverter()); private final SyncJobHandler.SourceType sourceType; diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncJobHandler.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncJobHandler.java index 255e1ca7e0e8..d7333237a2fd 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncJobHandler.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncJobHandler.java @@ -26,6 +26,8 @@ import org.apache.paimon.flink.action.cdc.postgres.PostgresActionUtils; import org.apache.paimon.flink.action.cdc.postgres.PostgresRecordParser; import org.apache.paimon.flink.action.cdc.pulsar.PulsarActionUtils; +import org.apache.paimon.flink.action.cdc.sqlserver.SqlServerRecordParser; +import org.apache.paimon.flink.action.cdc.sqlserver.SqlServerSourceOptions; import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord; import com.ververica.cdc.connectors.mongodb.source.config.MongoDBSourceOptions; @@ -46,6 +48,7 @@ import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.MYSQL_CONF; import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.POSTGRES_CONF; import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.PULSAR_CONF; +import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.SQLSERVER_CONF; import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.checkOneRequiredOption; import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.checkRequiredOptions; import static org.apache.paimon.utils.Preconditions.checkArgument; @@ -138,6 +141,26 @@ public void checkRequiredOption() { + "use postgres_sync_table instead."); } break; + case SQLSERVER: + checkRequiredOptions( + cdcSourceConfig, + SQLSERVER_CONF, + SqlServerSourceOptions.HOSTNAME, + SqlServerSourceOptions.USERNAME, + SqlServerSourceOptions.PASSWORD, + SqlServerSourceOptions.DATABASE_NAME); + if (isTableSync) { + checkRequiredOptions( + cdcSourceConfig, SQLSERVER_CONF, SqlServerSourceOptions.TABLE_NAME); + } else { + checkArgument( + !cdcSourceConfig.contains(SqlServerSourceOptions.TABLE_NAME), + SqlServerSourceOptions.TABLE_NAME.key() + + " cannot be set for sqlserver_sync_database. " + + "If you want to sync several SQLServer tables into one Paimon table, " + + "use sqlserver_sync_table instead."); + } + break; case KAFKA: checkRequiredOptions( cdcSourceConfig, @@ -212,6 +235,13 @@ public FlatMapFunction provideRecordParser( computedColumns, typeMapping, metadataConverters); + case SQLSERVER: + return new SqlServerRecordParser( + cdcSourceConfig, + caseSensitive, + computedColumns, + typeMapping, + metadataConverters); case KAFKA: case PULSAR: DataFormat dataFormat = provideDataFormat(); @@ -257,8 +287,8 @@ public enum SourceType { KAFKA("Kafka Source", "Kafka-Paimon %s Sync: %s"), MONGODB("MongoDB Source", "MongoDB-Paimon %s Sync: %s"), PULSAR("Pulsar Source", "Pulsar-Paimon %s Sync: %s"), - POSTGRES("Postgres Source", "Postgres-Paimon %s Sync: %s"); - + POSTGRES("Postgres Source", "Postgres-Paimon %s Sync: %s"), + SQLSERVER("SQlServer Source", "SQlServer-Paimon %s Sync: %s"); private final String sourceName; private final String defaultJobNameFormat; diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java index da3b80c5f09a..aecc4484a923 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java @@ -127,7 +127,7 @@ protected void beforeBuildingSourceSink() throws Exception { typeMapping); logNonPkTables(mySqlSchemasInfo.nonPkTables()); - List jdbcTableInfos = mySqlSchemasInfo.toMySqlTableInfos(mergeShards); + List jdbcTableInfos = mySqlSchemasInfo.toTableInfos(mergeShards); checkArgument( !jdbcTableInfos.isEmpty(), diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/schema/JdbcSchemasInfo.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/schema/JdbcSchemasInfo.java index eb9cc96ae393..e1028de19aa2 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/schema/JdbcSchemasInfo.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/schema/JdbcSchemasInfo.java @@ -40,6 +40,10 @@ public void addSchema(Identifier identifier, Schema schema) { addSchema(identifier, null, schema); } + public List schemaInfos() { + return schemasInfo; + } + public void addSchema(Identifier identifier, String schemaName, Schema schema) { JdbcSchemaInfo schemaInfo = new JdbcSchemaInfo(identifier, schemaName, !schema.primaryKeys().isEmpty(), schema); @@ -78,13 +82,13 @@ public JdbcTableInfo mergeAll() { } // only handle pk tables now - public List toMySqlTableInfos(boolean mergeShards) { + public List toTableInfos(boolean mergeShards) { if (mergeShards) { return mergeShards(); } else { return schemasInfo.stream() .filter(JdbcSchemaInfo::isPkTable) - .map(e -> new UnmergedJdbcTableInfo(e.identifier(), e.schema())) + .map(e -> new UnmergedJdbcTableInfo(e.identifier(), e.schemaName(), e.schema())) .collect(Collectors.toList()); } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/schema/UnmergedJdbcTableInfo.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/schema/UnmergedJdbcTableInfo.java index 52c5fff47bb9..49f20852533f 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/schema/UnmergedJdbcTableInfo.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/schema/UnmergedJdbcTableInfo.java @@ -20,6 +20,7 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.schema.Schema; +import org.apache.paimon.utils.StringUtils; import java.util.Collections; import java.util.List; @@ -29,10 +30,12 @@ public class UnmergedJdbcTableInfo implements JdbcTableInfo { private final Identifier identifier; private final Schema schema; + private final String schemaName; - public UnmergedJdbcTableInfo(Identifier identifier, Schema schema) { + public UnmergedJdbcTableInfo(Identifier identifier, String schemaName, Schema schema) { this.identifier = identifier; this.schema = schema; + this.schemaName = schemaName; } @Override @@ -54,7 +57,13 @@ public String tableName() { public String toPaimonTableName() { // the Paimon table name should be compound of origin database name and table name // together to avoid name conflict - return identifier.getDatabaseName() + "_" + identifier.getObjectName(); + if (StringUtils.isBlank(schemaName)) { + return String.format("%s_%s", identifier.getDatabaseName(), identifier.getObjectName()); + } else { + return String.format( + "%s_%s_%s", + identifier.getDatabaseName(), schemaName, identifier.getObjectName()); + } } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerActionUtils.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerActionUtils.java new file mode 100644 index 000000000000..f4b3a302a1c0 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerActionUtils.java @@ -0,0 +1,301 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action.cdc.sqlserver; + +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.flink.action.MultiTablesSinkMode; +import org.apache.paimon.flink.action.cdc.TypeMapping; +import org.apache.paimon.flink.action.cdc.schema.JdbcSchemaUtils; +import org.apache.paimon.flink.action.cdc.schema.JdbcSchemasInfo; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.utils.Pair; + +import com.ververica.cdc.connectors.base.options.JdbcSourceOptions; +import com.ververica.cdc.connectors.base.options.StartupOptions; +import com.ververica.cdc.connectors.sqlserver.source.SqlServerSourceBuilder; +import com.ververica.cdc.debezium.JsonDebeziumDeserializationSchema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.ValidationException; +import org.apache.kafka.connect.json.JsonConverterConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Predicate; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import static com.ververica.cdc.debezium.table.DebeziumOptions.getDebeziumProperties; +import static org.apache.paimon.flink.action.MultiTablesSinkMode.COMBINED; +import static org.apache.paimon.flink.action.MultiTablesSinkMode.DIVIDED; +import static org.apache.paimon.flink.action.cdc.sqlserver.SqlServerSourceOptions.SCAN_STARTUP_MODE; +import static org.apache.paimon.flink.action.cdc.sqlserver.SqlServerTypeUtils.toPaimonTypeVisitor; +import static org.apache.paimon.utils.Preconditions.checkArgument; + +/** Utility class for SqlServer action. */ +public class SqlServerActionUtils { + private static final Logger LOG = LoggerFactory.getLogger(SqlServerActionUtils.class); + private static final String SCAN_STARTUP_MODE_VALUE_INITIAL = "initial"; + private static final String SCAN_STARTUP_MODE_VALUE_LATEST = "latest-offset"; + + public static Connection getConnection(Configuration sqlServerConfig) throws Exception { + String url = + String.format( + "jdbc:sqlserver://%s:%s", + sqlServerConfig.get(SqlServerSourceOptions.HOSTNAME), + sqlServerConfig.get(SqlServerSourceOptions.PORT)); + + return DriverManager.getConnection( + url, + sqlServerConfig.get(SqlServerSourceOptions.USERNAME), + sqlServerConfig.get(SqlServerSourceOptions.PASSWORD)); + } + + public static JdbcSchemasInfo getSqlServerTableInfos( + Configuration sqlServerConfig, + Predicate monitorTablePredication, + List> excludedTables, + TypeMapping typeMapping) + throws Exception { + String databaseName = sqlServerConfig.get(SqlServerSourceOptions.DATABASE_NAME); + Pattern schemaPattern = + Pattern.compile(sqlServerConfig.get(SqlServerSourceOptions.SCHEMA_NAME)); + + JdbcSchemasInfo jdbcSchemasInfo = new JdbcSchemasInfo(); + try (Connection conn = getConnection(sqlServerConfig)) { + DatabaseMetaData metaData = conn.getMetaData(); + try (ResultSet schemas = metaData.getSchemas(databaseName, null)) { + while (schemas.next()) { + String schemaName = schemas.getString("TABLE_SCHEM"); + Matcher schemaMatcher = schemaPattern.matcher(schemaName); + if (!schemaMatcher.matches()) { + continue; + } + try (ResultSet tables = + metaData.getTables(databaseName, schemaName, "%", null)) { + while (tables.next()) { + String tableName = tables.getString("TABLE_NAME"); + if (tableName.startsWith("sys")) { + LOG.warn( + String.format( + "Filter out system or not cdc table [%s]", + tableName)); + continue; + } + + Identifier identifier = Identifier.create(databaseName, tableName); + if (monitorTablePredication.test(tableName)) { + String tableComment = tables.getString("REMARKS"); + Schema tableSchema = + JdbcSchemaUtils.buildSchema( + metaData, + databaseName, + schemaName, + tableName, + tableComment, + typeMapping, + toPaimonTypeVisitor()); + jdbcSchemasInfo.addSchema(identifier, schemaName, tableSchema); + } else { + excludedTables.add(Pair.of(identifier, schemaName)); + } + } + } + } + } + } + return jdbcSchemasInfo; + } + + public static List databaseList(List identifiers) { + return new ArrayList<>( + identifiers.stream() + .map(jdbcSchemaInfo -> jdbcSchemaInfo.identifier().getDatabaseName()) + .collect(Collectors.toSet())); + } + + public static String tableList( + MultiTablesSinkMode mode, + String schemaPattern, + String includingTablePattern, + List> monitoredTables, + List> excludedTables) { + if (mode == DIVIDED) { + return dividedModeTableList(monitoredTables); + } else if (mode == COMBINED) { + return combinedModeTableList(schemaPattern, includingTablePattern, excludedTables); + } + throw new UnsupportedOperationException("Unknown MultiTablesSinkMode: " + mode); + } + + private static String dividedModeTableList(List> monitoredTables) { + // In DIVIDED mode, we only concern about existed tables + return monitoredTables.stream() + .map(t -> t.getRight() + "\\." + t.getLeft().getObjectName()) + .collect(Collectors.joining("|")); + } + + public static String combinedModeTableList( + String schemaPattern, + String includingTablePattern, + List> excludedTables) { + String includingPattern = + String.format("(%s)\\.(%s)", schemaPattern, includingTablePattern); + if (excludedTables.isEmpty()) { + return includingPattern; + } + + String excludingPattern = + excludedTables.stream() + .map( + t -> + String.format( + "(^%s$)", + t.getRight() + "\\." + t.getLeft().getObjectName())) + .collect(Collectors.joining("|")); + excludingPattern = "?!" + excludingPattern; + return String.format("(%s)(%s)", excludingPattern, includingPattern); + } + + public static SqlServerSourceBuilder.SqlServerIncrementalSource buildSqlServerSource( + Configuration sqlServerSourceConfig, String tableList) { + validateSqlServerConfig(sqlServerSourceConfig); + + Map converterConfigs = new HashMap<>(); + converterConfigs.put(JsonConverterConfig.DECIMAL_FORMAT_CONFIG, "numeric"); + JsonDebeziumDeserializationSchema debeziumDeserializationSchema = + new JsonDebeziumDeserializationSchema(true, converterConfigs); + + sqlServerSourceConfig.setBoolean("debezium.include.schema.changes", false); + SqlServerSourceBuilder sqlServerSourceBuilder = + new SqlServerSourceBuilder() + .hostname(sqlServerSourceConfig.get(SqlServerSourceOptions.HOSTNAME)) + .port(sqlServerSourceConfig.get(SqlServerSourceOptions.PORT)) + .username(sqlServerSourceConfig.get(SqlServerSourceOptions.USERNAME)) + .password(sqlServerSourceConfig.get(SqlServerSourceOptions.PASSWORD)) + .databaseList( + sqlServerSourceConfig.getString( + SqlServerSourceOptions.DATABASE_NAME)) + .tableList(tableList) + .debeziumProperties(getDebeziumProperties(sqlServerSourceConfig.toMap())) + .startupOptions(getStartupOptions(sqlServerSourceConfig)) + .includeSchemaChanges(false) + .deserializer(debeziumDeserializationSchema); + + sqlServerSourceConfig + .getOptional(SqlServerSourceOptions.SERVER_TIME_ZONE) + .ifPresent(sqlServerSourceBuilder::serverTimeZone); + + sqlServerSourceConfig + .getOptional(JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE) + .ifPresent(sqlServerSourceBuilder::splitSize); + sqlServerSourceConfig + .getOptional(JdbcSourceOptions.CHUNK_META_GROUP_SIZE) + .ifPresent(sqlServerSourceBuilder::splitMetaGroupSize); + sqlServerSourceConfig + .getOptional(JdbcSourceOptions.CONNECT_TIMEOUT) + .ifPresent(sqlServerSourceBuilder::connectTimeout); + sqlServerSourceConfig + .getOptional(JdbcSourceOptions.CONNECT_MAX_RETRIES) + .ifPresent(sqlServerSourceBuilder::connectMaxRetries); + sqlServerSourceConfig + .getOptional(JdbcSourceOptions.CONNECTION_POOL_SIZE) + .ifPresent(sqlServerSourceBuilder::connectionPoolSize); + sqlServerSourceConfig + .getOptional(JdbcSourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND) + .ifPresent(sqlServerSourceBuilder::distributionFactorUpper); + sqlServerSourceConfig + .getOptional(JdbcSourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND) + .ifPresent(sqlServerSourceBuilder::distributionFactorLower); + sqlServerSourceConfig + .getOptional(JdbcSourceOptions.SCAN_INCREMENTAL_CLOSE_IDLE_READER_ENABLED) + .ifPresent(sqlServerSourceBuilder::closeIdleReaders); + return sqlServerSourceBuilder.includeSchemaChanges(true).build(); + } + + private static StartupOptions getStartupOptions(Configuration config) { + String modeString = config.get(SCAN_STARTUP_MODE); + + switch (modeString.toLowerCase()) { + case SCAN_STARTUP_MODE_VALUE_INITIAL: + return com.ververica.cdc.connectors.base.options.StartupOptions.initial(); + + case SCAN_STARTUP_MODE_VALUE_LATEST: + return com.ververica.cdc.connectors.base.options.StartupOptions.latest(); + + default: + throw new ValidationException( + String.format( + "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_LATEST, + modeString)); + } + } + + public static void registerJdbcDriver() { + try { + Class.forName("com.microsoft.sqlserver.jdbc.SQLServerDriver"); + } catch (ClassNotFoundException ex) { + throw new RuntimeException( + "No suitable driver found. Cannot find class com.microsoft.sqlserver.jdbc.SQLServerDriver."); + } + } + + private static void validateSqlServerConfig(Configuration sqlServerSourceConfig) { + checkArgument( + sqlServerSourceConfig.get(SqlServerSourceOptions.HOSTNAME) != null, + String.format( + "sqlserver-conf [%s] must be specified.", + SqlServerSourceOptions.HOSTNAME.key())); + + checkArgument( + sqlServerSourceConfig.get(SqlServerSourceOptions.PORT) != null, + String.format( + "sqlserver-conf [%s] must be specified.", + SqlServerSourceOptions.PORT.key())); + + checkArgument( + sqlServerSourceConfig.get(SqlServerSourceOptions.PASSWORD) != null, + String.format( + "sqlserver-conf [%s] must be specified.", + SqlServerSourceOptions.PASSWORD.key())); + + checkArgument( + sqlServerSourceConfig.get(SqlServerSourceOptions.DATABASE_NAME) != null, + String.format( + "sqlserver-conf [%s] must be specified.", + SqlServerSourceOptions.DATABASE_NAME.key())); + + checkArgument( + sqlServerSourceConfig.get(SqlServerSourceOptions.SCHEMA_NAME) != null, + String.format( + "sqlserver-conf [%s] must be specified.", + SqlServerSourceOptions.SCHEMA_NAME.key())); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerRecordParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerRecordParser.java new file mode 100644 index 000000000000..ab96a1606593 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerRecordParser.java @@ -0,0 +1,286 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action.cdc.sqlserver; + +import org.apache.paimon.flink.action.cdc.CdcMetadataConverter; +import org.apache.paimon.flink.action.cdc.ComputedColumn; +import org.apache.paimon.flink.action.cdc.TypeMapping; +import org.apache.paimon.flink.action.cdc.mysql.format.DebeziumEvent; +import org.apache.paimon.flink.sink.cdc.CdcRecord; +import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord; +import org.apache.paimon.types.RowKind; +import org.apache.paimon.utils.DateTimeUtils; +import org.apache.paimon.utils.JsonSerdeUtil; +import org.apache.paimon.utils.Preconditions; +import org.apache.paimon.utils.StringUtils; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.DeserializationFeature; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import io.debezium.connector.AbstractSourceInfo; +import io.debezium.time.Conversions; +import io.debezium.time.Date; +import io.debezium.time.MicroTime; +import io.debezium.time.MicroTimestamp; +import io.debezium.time.NanoTime; +import io.debezium.time.NanoTimestamp; +import io.debezium.time.Time; +import io.debezium.time.Timestamp; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; +import org.apache.flink.util.Collector; +import org.apache.kafka.connect.data.Decimal; +import org.apache.kafka.connect.json.JsonConverterConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.math.BigDecimal; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneId; +import java.time.temporal.ChronoUnit; +import java.util.Base64; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.mapKeyCaseConvert; +import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.recordKeyDuplicateErrMsg; +import static org.apache.paimon.utils.JsonSerdeUtil.isNull; + +/** + * A parser for SqlServer Debezium JSON strings, converting them into a list of {@link + * RichCdcMultiplexRecord}s. + */ +public class SqlServerRecordParser implements FlatMapFunction { + + private static final Logger LOG = LoggerFactory.getLogger(SqlServerRecordParser.class); + private final ObjectMapper objectMapper = new ObjectMapper(); + private final ZoneId serverTimeZone; + private final boolean caseSensitive; + private final List computedColumns; + private final TypeMapping typeMapping; + private DebeziumEvent root; + private String currentTable; + private String schemaName; + private String databaseName; + private final Set nonPkTables = new HashSet<>(); + private final CdcMetadataConverter[] metadataConverters; + + public SqlServerRecordParser( + Configuration sqlServerConfig, + boolean caseSensitive, + TypeMapping typeMapping, + CdcMetadataConverter[] metadataConverters) { + this( + sqlServerConfig, + caseSensitive, + Collections.emptyList(), + typeMapping, + metadataConverters); + } + + public SqlServerRecordParser( + Configuration sqlServerConfig, + boolean caseSensitive, + List computedColumns, + TypeMapping typeMapping, + CdcMetadataConverter[] metadataConverters) { + this.caseSensitive = caseSensitive; + this.computedColumns = computedColumns; + this.typeMapping = typeMapping; + this.metadataConverters = metadataConverters; + objectMapper + .configure(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS, true) + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + String stringifyServerTimeZone = + sqlServerConfig.get(SqlServerSourceOptions.SERVER_TIME_ZONE); + this.serverTimeZone = + stringifyServerTimeZone == null + ? ZoneId.systemDefault() + : ZoneId.of(stringifyServerTimeZone); + } + + @Override + public void flatMap(String rawEvent, Collector out) throws Exception { + this.root = objectMapper.readValue(rawEvent, DebeziumEvent.class); + + currentTable = root.payload().source().get(AbstractSourceInfo.TABLE_NAME_KEY).asText(); + schemaName = root.payload().source().get(AbstractSourceInfo.SCHEMA_NAME_KEY).asText(); + databaseName = root.payload().source().get(AbstractSourceInfo.DATABASE_NAME_KEY).asText(); + + if (nonPkTables.contains(currentTable)) { + return; + } + if (root.payload().isSchemaChange()) { + throw new UnsupportedOperationException( + "Temporarily not supporting schema change events"); + } + convertRecords().forEach(out::collect); + } + + private List convertRecords() { + List records = Lists.newArrayList(); + Map before = extractRow(root.payload().before()); + if (!before.isEmpty()) { + before = mapKeyCaseConvert(before, caseSensitive, recordKeyDuplicateErrMsg(before)); + records.add(createRecord(RowKind.DELETE, before)); + } + Map after = extractRow(root.payload().after()); + if (!after.isEmpty()) { + after = mapKeyCaseConvert(after, caseSensitive, recordKeyDuplicateErrMsg(after)); + records.add(createRecord(RowKind.INSERT, after)); + } + return records; + } + + private Map extractRow(JsonNode record) { + if (JsonSerdeUtil.isNull(record)) { + return new HashMap<>(); + } + DebeziumEvent.Field schema = + Preconditions.checkNotNull( + root.schema(), + "SqlServerRecordParser only supports debezium JSON with schema. " + + "Please make sure that `includeSchema` is true " + + "in the JsonDebeziumDeserializationSchema you created"); + + Map fields = schema.beforeAndAfterFields(); + LinkedHashMap resultMap = new LinkedHashMap<>(); + for (Map.Entry field : fields.entrySet()) { + String fieldName = field.getKey(); + JsonNode value = record.get(fieldName); + if (isNull(value)) { + continue; + } + String newValue = this.maybeLogicalType(field.getValue(), value); + resultMap.put(fieldName, newValue); + } + // generate values of computed columns + for (ComputedColumn computedColumn : computedColumns) { + resultMap.put( + computedColumn.columnName(), + computedColumn.eval(resultMap.get(computedColumn.fieldReference()))); + } + for (CdcMetadataConverter metadataConverter : metadataConverters) { + resultMap.put( + metadataConverter.columnName(), + metadataConverter.read(root.payload().source())); + } + return resultMap; + } + + private String maybeLogicalType(DebeziumEvent.Field field, JsonNode value) { + // https://debezium.io/documentation/reference/stable/connectors/sqlserver.html#sqlserver-data-types + String logicalClassName = field.name(); + String schemaType = field.type(); + // Extract new value + String oldValue = value.asText(); + if (isSchemaBytes(schemaType) && logicalClassName == null) { + return new String(Base64.getDecoder().decode(oldValue)); + } + if (StringUtils.isBlank(logicalClassName)) { + return oldValue; + } + switch (logicalClassName) { + case Decimal.LOGICAL_NAME: + if (isSchemaBytes(schemaType)) { + try { + new BigDecimal(oldValue); + } catch (NumberFormatException e) { + throw new IllegalArgumentException( + "Invalid big decimal value " + + oldValue + + ". Make sure that in the `customConverterConfigs` " + + "of the JsonDebeziumDeserializationSchema you created, set '" + + JsonConverterConfig.DECIMAL_FORMAT_CONFIG + + "' to 'numeric'", + e); + } + } + return oldValue; + case Date.SCHEMA_NAME: + return DateTimeUtils.toLocalDate(Integer.parseInt(oldValue)).toString(); + case Timestamp.SCHEMA_NAME: + LocalDateTime localDateTime = + DateTimeUtils.toLocalDateTime(Long.parseLong(oldValue), serverTimeZone); + return DateTimeUtils.formatLocalDateTime(localDateTime, 3); + case Time.SCHEMA_NAME: + long durationMillis = Long.parseLong(oldValue); + LocalTime localTime = + LocalTime.ofNanoOfDay( + Duration.of(durationMillis, ChronoUnit.MILLIS).toNanos()); + return localTime.toString(); + case NanoTime.SCHEMA_NAME: + long durationNanos = Long.parseLong(oldValue); + LocalTime durationNanosLocalTime = + LocalTime.ofNanoOfDay( + Duration.of(durationNanos, ChronoUnit.NANOS).toNanos()); + return durationNanosLocalTime.toString(); + case MicroTime.SCHEMA_NAME: + long durationMicros = Long.parseLong(oldValue); + LocalTime durationMicrosLocalTime = + LocalTime.ofNanoOfDay( + Duration.of(durationMicros, ChronoUnit.MICROS).toNanos()); + return durationMicrosLocalTime.toString(); + case MicroTimestamp.SCHEMA_NAME: + long epochMicros = Long.parseLong(oldValue); + LocalDateTime epochMicrosLocalDateTime = + LocalDateTime.ofInstant( + Conversions.toInstantFromMicros(epochMicros), serverTimeZone); + return DateTimeUtils.formatLocalDateTime(epochMicrosLocalDateTime, 6); + case NanoTimestamp.SCHEMA_NAME: + long epochNanos = Long.parseLong(oldValue); + LocalDateTime epochNanosLocalDateTime = + LocalDateTime.ofInstant(toInstantFromNanos(epochNanos), serverTimeZone); + return DateTimeUtils.formatLocalDateTime(epochNanosLocalDateTime, 7); + default: + return oldValue; + } + } + + public Instant toInstantFromNanos(long epochNanos) { + final long epochSeconds = TimeUnit.NANOSECONDS.toSeconds(epochNanos); + final long adjustment = + TimeUnit.NANOSECONDS.toNanos(epochNanos % TimeUnit.SECONDS.toNanos(1)); + return Instant.ofEpochSecond(epochSeconds, adjustment); + } + + private boolean isSchemaBytes(String schemaType) { + return "bytes".equals(schemaType); + } + + protected RichCdcMultiplexRecord createRecord(RowKind rowKind, Map data) { + return new RichCdcMultiplexRecord( + String.format("%s_%s", databaseName, schemaName), + currentTable, + new LinkedHashMap<>(0), + Collections.emptyList(), + new CdcRecord(rowKind, data)); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSourceOptions.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSourceOptions.java new file mode 100644 index 000000000000..0eba698aa0c9 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSourceOptions.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action.cdc.sqlserver; + +import com.ververica.cdc.connectors.base.options.JdbcSourceOptions; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +/** SqlServer source configuration options. */ +public class SqlServerSourceOptions extends JdbcSourceOptions { + public static final ConfigOption HOSTNAME = + ConfigOptions.key("hostname") + .stringType() + .noDefaultValue() + .withDescription("IP address or hostname of the SqlServer database server."); + + public static final ConfigOption PORT = + ConfigOptions.key("port") + .intType() + .defaultValue(1433) + .withDescription("Integer port number of the SqlServer database server."); + + public static final ConfigOption USERNAME = + ConfigOptions.key("username") + .stringType() + .noDefaultValue() + .withDescription( + "Name of the SqlServer database to use when connecting to the SqlServer database server."); + + public static final ConfigOption PASSWORD = + ConfigOptions.key("password") + .stringType() + .noDefaultValue() + .withDescription( + "Password to use when connecting to the SqlServer database server."); + + public static final ConfigOption DATABASE_NAME = + ConfigOptions.key("database-name") + .stringType() + .noDefaultValue() + .withDescription("Database name of the SqlServer server to monitor."); + + public static final ConfigOption SCHEMA_NAME = + ConfigOptions.key("schema-name") + .stringType() + .noDefaultValue() + .withDescription("Schema name of the SqlServer server to monitor."); + + public static final ConfigOption TABLE_NAME = + ConfigOptions.key("table-name") + .stringType() + .noDefaultValue() + .withDescription("Table name of the SqlServer database to monitor."); + + public static final ConfigOption SERVER_TIME_ZONE = + ConfigOptions.key("server-time-zone") + .stringType() + .defaultValue("UTC") + .withDescription("The session time zone in database server."); + + public static final ConfigOption SCAN_STARTUP_MODE = + ConfigOptions.key("scan.startup.mode") + .stringType() + .defaultValue("initial") + .withDescription( + "Optional startup mode for SqlServer CDC consumer, valid enumerations are " + + "\"initial\", \"latest-offset\""); +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncDatabaseAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncDatabaseAction.java new file mode 100644 index 000000000000..b90f0c5172a6 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncDatabaseAction.java @@ -0,0 +1,250 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action.cdc.sqlserver; + +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.flink.action.Action; +import org.apache.paimon.flink.action.cdc.CdcActionCommonUtils; +import org.apache.paimon.flink.action.cdc.SyncDatabaseActionBase; +import org.apache.paimon.flink.action.cdc.SyncJobHandler; +import org.apache.paimon.flink.action.cdc.TableNameConverter; +import org.apache.paimon.flink.action.cdc.schema.JdbcSchemasInfo; +import org.apache.paimon.flink.action.cdc.schema.JdbcTableInfo; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.utils.Pair; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Supplier; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import static org.apache.paimon.flink.action.MultiTablesSinkMode.DIVIDED; +import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.schemaCompatible; + +/** An {@link Action} which synchronize the whole SqlServer database into one Paimon database. */ +public class SqlServerSyncDatabaseAction extends SyncDatabaseActionBase { + private static final Logger LOG = LoggerFactory.getLogger(SqlServerSyncDatabaseAction.class); + private boolean ignoreIncompatible = false; + private final List> monitoredTables = new ArrayList<>(); + private final List> excludedTables = new ArrayList<>(); + + public SqlServerSyncDatabaseAction( + String warehouse, + String database, + Map catalogConfig, + Map sqlServerConfig) { + super( + warehouse, + database, + catalogConfig, + sqlServerConfig, + SyncJobHandler.SourceType.SQLSERVER); + this.mode = DIVIDED; + } + + public SqlServerSyncDatabaseAction ignoreIncompatible(boolean ignoreIncompatible) { + this.ignoreIncompatible = ignoreIncompatible; + return this; + } + + @Override + protected void beforeBuildingSourceSink() throws Exception { + Pattern includingPattern = Pattern.compile(includingTables); + Pattern excludingPattern = + excludingTables == null ? null : Pattern.compile(excludingTables); + JdbcSchemasInfo jdbcSchemasInfo = + SqlServerActionUtils.getSqlServerTableInfos( + cdcSourceConfig, + tableName -> + shouldMonitorTable(tableName, includingPattern, excludingPattern), + excludedTables, + typeMapping); + + logNonPkTables(jdbcSchemasInfo); + + Map> schemaMappings = getSchemaMapping(jdbcSchemasInfo); + + List sqlServerTableInfos = jdbcSchemasInfo.toTableInfos(mergeShards); + + TableNameConverter tableNameConverter = + new TableNameConverter(caseSensitive, mergeShards, tablePrefix, tableSuffix); + for (JdbcTableInfo tableInfo : sqlServerTableInfos) { + Identifier identifier = + Identifier.create( + database, tableNameConverter.convert(tableInfo.toPaimonTableName())); + FileStoreTable table; + Schema fromSqlServer = + CdcActionCommonUtils.buildPaimonSchema( + identifier.getFullName(), + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + tableConfig, + tableInfo.schema(), + metadataConverters, + caseSensitive, + true); + try { + table = (FileStoreTable) catalog.getTable(identifier); + table = alterTableOptions(identifier, table); + Supplier errMsg = + incompatibleMessage(table.schema(), tableInfo, identifier); + + if (shouldMonitorTable(table.schema(), fromSqlServer, errMsg)) { + tables.add(table); + setTables(schemaMappings, tableInfo.identifiers(), monitoredTables); + } else { + setTables(schemaMappings, tableInfo.identifiers(), excludedTables); + } + } catch (Catalog.TableNotExistException e) { + catalog.createTable(identifier, fromSqlServer, false); + table = (FileStoreTable) catalog.getTable(identifier); + tables.add(table); + setTables(schemaMappings, tableInfo.identifiers(), monitoredTables); + } + } + } + + private void setTables( + Map> schemaMappings, + List identifiers, + List> tables) { + identifiers.stream() + .forEach( + item -> { + Set schemas = schemaMappings.get(item.getFullName()); + schemas.stream() + .forEach( + schemaName -> { + tables.add(Pair.of(item, schemaName)); + }); + }); + } + + private static Map> getSchemaMapping(JdbcSchemasInfo jdbcSchemasInfo) { + Map> schemaMapping = new HashMap<>(); + List jdbcSchemaInfos = jdbcSchemasInfo.schemaInfos(); + for (JdbcSchemasInfo.JdbcSchemaInfo jdbcSchemaInfo : jdbcSchemaInfos) { + if (!jdbcSchemaInfo.isPkTable()) { + continue; + } + String fullName = jdbcSchemaInfo.identifier().getFullName(); + if (!schemaMapping.containsKey(fullName)) { + Set schemaNames = new HashSet<>(); + schemaNames.add(jdbcSchemaInfo.schemaName()); + schemaMapping.put(fullName, schemaNames); + } else { + Set existsSchemas = schemaMapping.get(fullName); + existsSchemas.add(jdbcSchemaInfo.schemaName()); + } + } + return schemaMapping; + } + + @Override + protected Object buildSource() { + try { + return SqlServerActionUtils.buildSqlServerSource( + cdcSourceConfig, + SqlServerActionUtils.tableList( + mode, + cdcSourceConfig.get(SqlServerSourceOptions.SCHEMA_NAME), + includingTables, + monitoredTables, + excludedTables)); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private Supplier incompatibleMessage( + TableSchema paimonSchema, JdbcTableInfo tableInfo, Identifier identifier) { + return () -> + String.format( + "Incompatible schema found.\n" + + "Paimon table is: %s, fields are: %s.\n" + + "SqlServer table is: %s, fields are: %s.\n", + identifier.getFullName(), + paimonSchema.fields(), + tableInfo.location(), + tableInfo.schema().fields()); + } + + private void logNonPkTables(JdbcSchemasInfo jdbcSchemasInfo) { + List nonPkTables = jdbcSchemasInfo.nonPkTables(); + if (!nonPkTables.isEmpty()) { + LOG.debug( + "Didn't find primary keys for tables '{}'. " + + "These tables won't be synchronized.", + nonPkTables.stream() + .map(Identifier::getFullName) + .collect(Collectors.joining(","))); + jdbcSchemasInfo.schemaInfos().stream() + .forEach( + jdbcSchemaInfo -> { + if (!jdbcSchemaInfo.isPkTable()) { + excludedTables.add( + Pair.of( + jdbcSchemaInfo.identifier(), + jdbcSchemaInfo.schemaName())); + } + }); + } + } + + private boolean shouldMonitorTable( + String tableName, Pattern includingPattern, @Nullable Pattern excludingPattern) { + boolean shouldMonitor = includingPattern.matcher(tableName).matches(); + if (excludingPattern != null) { + shouldMonitor = shouldMonitor && !excludingPattern.matcher(tableName).matches(); + } + if (!shouldMonitor) { + LOG.debug("Source table '{}' is excluded.", tableName); + } + return shouldMonitor; + } + + private boolean shouldMonitorTable( + TableSchema tableSchema, Schema sqlServerSchema, Supplier errMsg) { + if (schemaCompatible(tableSchema, sqlServerSchema.fields())) { + return true; + } else if (ignoreIncompatible) { + LOG.warn(errMsg.get() + "This table will be ignored."); + return false; + } else { + throw new IllegalArgumentException( + errMsg.get() + + "If you want to ignore the incompatible tables, please specify --ignore-incompatible to true."); + } + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncDatabaseActionFactory.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncDatabaseActionFactory.java new file mode 100644 index 000000000000..a970b2003e82 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncDatabaseActionFactory.java @@ -0,0 +1,191 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action.cdc.sqlserver; + +import org.apache.paimon.flink.action.Action; +import org.apache.paimon.flink.action.ActionFactory; +import org.apache.paimon.flink.action.MultiTablesSinkMode; +import org.apache.paimon.flink.action.MultipleParameterToolAdapter; +import org.apache.paimon.flink.action.cdc.TypeMapping; + +import java.util.Arrays; +import java.util.Optional; + +import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.EXCLUDING_TABLES; +import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.INCLUDING_TABLES; +import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.METADATA_COLUMN; +import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.SQLSERVER_CONF; +import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.TABLE_PREFIX; +import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.TABLE_SUFFIX; +import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.TYPE_MAPPING; + +/** Factory to create {@link SqlServerSyncDatabaseAction}. */ +public class SqlServerSyncDatabaseActionFactory implements ActionFactory { + + public static final String IDENTIFIER = "sqlserver_sync_database"; + + @Override + public String identifier() { + return IDENTIFIER; + } + + @Override + public Optional create(MultipleParameterToolAdapter params) { + checkRequiredArgument(params, SQLSERVER_CONF); + + SqlServerSyncDatabaseAction action = + new SqlServerSyncDatabaseAction( + getRequiredValue(params, WAREHOUSE), + getRequiredValue(params, DATABASE), + optionalConfigMap(params, CATALOG_CONF), + optionalConfigMap(params, SQLSERVER_CONF)); + + action.withTableConfig(optionalConfigMap(params, TABLE_CONF)); + + action.ignoreIncompatible(Boolean.parseBoolean(params.get("ignore_incompatible"))) + .mergeShards( + !params.has("merge_shards") + || Boolean.parseBoolean(params.get("merge_shards"))) + .withTablePrefix(params.get(TABLE_PREFIX)) + .withTableSuffix(params.get(TABLE_SUFFIX)) + .includingTables(params.get(INCLUDING_TABLES)) + .excludingTables(params.get(EXCLUDING_TABLES)) + .withMode(MultiTablesSinkMode.fromString(params.get("mode"))); + if (params.has(METADATA_COLUMN)) { + action.withMetadataColumns(Arrays.asList(params.get(METADATA_COLUMN).split(","))); + } + + if (params.has(TYPE_MAPPING)) { + String[] options = params.get(TYPE_MAPPING).split(","); + action.withTypeMapping(TypeMapping.parse(options)); + } + + return Optional.of(action); + } + + @Override + public void printHelp() { + System.out.println( + "Action \"sqlserver-sync-database\" creates a streaming job " + + "with a Flink SqlServer CDC source and multiple Paimon table sinks " + + "to synchronize a whole SqlServer database into one Paimon database.\n" + + "Only SqlServer tables with primary keys will be considered. " + + "Newly created SqlServer tables after the job starts will not be included."); + System.out.println(); + + System.out.println("Syntax:"); + System.out.println( + " sqlserver-sync-database --warehouse --database " + + "[--ignore-incompatible ] " + + "[--merge-shards ] " + + "[--table-prefix ] " + + "[--table-suffix ] " + + "[--including-tables ] " + + "[--excluding-tables ] " + + "[--mode ] " + + "[--metadata-column ] " + + "[--type-mapping ] " + + "[--sqlserver-conf [--sqlserver-conf ...]] " + + "[--catalog-conf [--catalog-conf ...]] " + + "[--table-conf [--table-conf ...]]"); + System.out.println(); + + System.out.println( + "--ignore-incompatible is default false, in this case, if SqlServer table name exists in Paimon " + + "and their schema is incompatible, an exception will be thrown. " + + "You can specify it to true explicitly to ignore the incompatible tables and exception."); + System.out.println(); + + System.out.println( + "--merge-shards is default true, in this case, if some tables in different databases have the same name, " + + "their schemas will be merged and their records will be synchronized into one Paimon table. " + + "Otherwise, each table's records will be synchronized to a corresponding Paimon table, " + + "and the Paimon table will be named to 'databaseName_tableName' to avoid potential name conflict."); + System.out.println(); + + System.out.println( + "--table-prefix is the prefix of all Paimon tables to be synchronized. For example, if you want all " + + "synchronized tables to have \"ods_\" as prefix, you can specify `--table-prefix ods_`."); + System.out.println("The usage of --table-suffix is same as `--table-prefix`"); + System.out.println(); + + System.out.println( + "--including-tables is used to specify which source tables are to be synchronized. " + + "You must use '|' to separate multiple tables. Regular expression is supported."); + System.out.println( + "--excluding-tables is used to specify which source tables are not to be synchronized. " + + "The usage is same as --including-tables."); + System.out.println( + "--excluding-tables has higher priority than --including-tables if you specified both."); + System.out.println(); + + System.out.println( + "--mode is used to specify synchronization mode. You can specify two modes:"); + System.out.println( + " 1. 'divided' (the default mode if you haven't specified one): " + + "start a sink for each table, the synchronization of the new table requires restarting the job;"); + System.out.println( + " 2. 'combined': start a single combined sink for all tables, the new table will be automatically synchronized."); + System.out.println(); + + System.out.println( + "--metadata-column is used to specify which metadata columns to include in the output schema of the connector. Please see the doc for usage."); + System.out.println(); + + System.out.println( + "--type-mapping is used to specify how to map SqlServer type to Paimon type. Please see the doc for usage."); + System.out.println(); + + System.out.println("SqlServer CDC source conf syntax:"); + System.out.println(" key=value"); + System.out.println( + "'hostname', 'username', 'password' and 'database-name' " + + "are required configurations, others are optional. " + + "Note that 'database-name' should be the exact name " + + "of the SqlServer database you want to synchronize. " + + "It can't be a regular expression."); + System.out.println( + "For a complete list of supported configurations, " + + "see https://ververica.github.io/flink-cdc-connectors/master/content/connectors/sqlserver-cdc.html#connector-options"); + System.out.println(); + + System.out.println("Paimon catalog and table sink conf syntax:"); + System.out.println(" key=value"); + System.out.println("All Paimon sink table will be applied the same set of configurations."); + System.out.println( + "For a complete list of supported configurations, " + + "see https://paimon.apache.org/docs/master/maintenance/configurations/"); + System.out.println(); + + System.out.println("Examples:"); + System.out.println( + " sqlserver-sync-database \\\n" + + " --warehouse hdfs:///path/to/warehouse \\\n" + + " --database test_db \\\n" + + " --sqlserver-conf hostname=127.0.0.1 \\\n" + + " --sqlserver-conf username=root \\\n" + + " --sqlserver-conf password=123456 \\\n" + + " --sqlserver-conf database-name=source_db \\\n" + + " --catalog-conf metastore=hive \\\n" + + " --catalog-conf uri=thrift://hive-metastore:9083 \\\n" + + " --table-conf bucket=4 \\\n" + + " --table-conf changelog-producer=input \\\n" + + " --table-conf sink.parallelism=4"); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncTableAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncTableAction.java new file mode 100644 index 000000000000..ab32c46bc7b2 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncTableAction.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action.cdc.sqlserver; + +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.flink.action.Action; +import org.apache.paimon.flink.action.cdc.SyncJobHandler; +import org.apache.paimon.flink.action.cdc.SyncTableActionBase; +import org.apache.paimon.flink.action.cdc.schema.JdbcSchemasInfo; +import org.apache.paimon.flink.action.cdc.schema.JdbcTableInfo; +import org.apache.paimon.schema.Schema; + +import com.ververica.cdc.connectors.sqlserver.source.SqlServerSourceBuilder; +import org.apache.flink.configuration.Configuration; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.function.Predicate; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import static org.apache.paimon.flink.action.cdc.sqlserver.SqlServerActionUtils.buildSqlServerSource; +import static org.apache.paimon.utils.Preconditions.checkArgument; + +/** An {@link Action} which synchronize one or multiple SqlServer tables into one Paimon table. */ +public class SqlServerSyncTableAction extends SyncTableActionBase { + private JdbcSchemasInfo sqlServerSchemasInfo; + + public SqlServerSyncTableAction( + String warehouse, + String database, + String table, + Map catalogConfig, + Map cdcSourceConfig) { + super( + warehouse, + database, + table, + catalogConfig, + cdcSourceConfig, + SyncJobHandler.SourceType.SQLSERVER); + SqlServerActionUtils.registerJdbcDriver(); + } + + @Override + protected Schema retrieveSchema() throws Exception { + this.sqlServerSchemasInfo = + SqlServerActionUtils.getSqlServerTableInfos( + cdcSourceConfig, + matchTablePredication(cdcSourceConfig), + new ArrayList<>(), + typeMapping); + validateSqlServerTableInfos(sqlServerSchemasInfo); + JdbcTableInfo tableInfo = sqlServerSchemasInfo.mergeAll(); + return tableInfo.schema(); + } + + private static Predicate matchTablePredication(Configuration cdcSourceConfig) { + return tableName -> { + Pattern tableNamePattern = + Pattern.compile(cdcSourceConfig.get(SqlServerSourceOptions.TABLE_NAME)); + return tableNamePattern.matcher(tableName).matches(); + }; + } + + private void validateSqlServerTableInfos(JdbcSchemasInfo sqlServerSchemasInfo) { + List nonPkTables = sqlServerSchemasInfo.nonPkTables(); + checkArgument( + nonPkTables.isEmpty(), + "SqlServer source will not synchronize tables without a primary key.\n" + + "They are: %s", + nonPkTables.stream().map(Identifier::getFullName).collect(Collectors.joining(","))); + + checkArgument( + !sqlServerSchemasInfo.pkTables().isEmpty(), + "No tables matching the configuration were found."); + } + + @Override + protected SqlServerSourceBuilder.SqlServerIncrementalSource buildSource() { + String tableList = + sqlServerSchemasInfo.pkTables().stream() + .map(i -> i.schemaName() + "." + i.identifier().getObjectName()) + .collect(Collectors.joining("|")); + return buildSqlServerSource(cdcSourceConfig, tableList); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncTableActionFactory.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncTableActionFactory.java new file mode 100644 index 000000000000..d7583a0699a5 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncTableActionFactory.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action.cdc.sqlserver; + +import org.apache.paimon.flink.action.cdc.SyncTableActionBase; +import org.apache.paimon.flink.action.cdc.SyncTableActionFactoryBase; + +import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.SQLSERVER_CONF; + +/** Factory to create {@link SqlServerSyncTableAction}. */ +public class SqlServerSyncTableActionFactory extends SyncTableActionFactoryBase { + + public static final String IDENTIFIER = "sqlserver_sync_table"; + + @Override + public String identifier() { + return IDENTIFIER; + } + + @Override + public String cdcConfigIdentifier() { + return SQLSERVER_CONF; + } + + @Override + public SyncTableActionBase createAction() { + + return new SqlServerSyncTableAction( + this.tablePath.f0, + this.tablePath.f1, + this.tablePath.f2, + this.catalogConfig, + this.cdcSourceConfig); + } + + @Override + public void printHelp() { + System.out.println( + "Action \"sqlserver-sync-table\" creates a streaming job " + + "with a Flink SqlServer CDC source and a Paimon table sink to consume CDC events."); + System.out.println(); + + System.out.println("Syntax:"); + System.out.println( + " sqlserver-sync-table --warehouse --database " + + "--table " + + "[--partition-keys ] " + + "[--primary-keys ] " + + "[--type-mapping ] " + + "[--computed-column <'column-name=expr-name(args[, ...])'> [--computed-column ...]] " + + "[--metadata-column ] " + + "[--sqlserver-conf [--sqlserver-conf ...]] " + + "[--catalog-conf [--catalog-conf ...]] " + + "[--table-conf [--table-conf ...]]"); + System.out.println(); + + System.out.println("Partition keys syntax:"); + System.out.println(" key1,key2,..."); + System.out.println( + "If partition key is not defined and the specified Paimon table does not exist, " + + "this action will automatically create an unpartitioned Paimon table."); + System.out.println(); + + System.out.println("Primary keys syntax:"); + System.out.println(" key1,key2,..."); + System.out.println("Primary keys will be derived from SqlServer tables if not specified."); + System.out.println(); + + System.out.println( + "--type-mapping is used to specify how to map SqlServer type to Paimon type. Please see the doc for usage."); + System.out.println(); + + System.out.println("Please see doc for usage of --computed-column."); + System.out.println(); + + System.out.println( + "--metadata-column is used to specify which metadata columns to include in the output schema of the connector. Please see the doc for usage."); + System.out.println(); + + System.out.println("SqlServer CDC source conf syntax:"); + System.out.println(" key=value"); + System.out.println( + "'hostname', 'username', 'password', 'database-name' and 'table-name' " + + "are required configurations, others are optional."); + System.out.println( + "For a complete list of supported configurations, " + + "see https://ververica.github.io/flink-cdc-connectors/master/content/connectors/sqlserver-cdc.html#connector-options"); + System.out.println(); + + System.out.println("Paimon catalog and table sink conf syntax:"); + System.out.println(" key=value"); + System.out.println( + "For a complete list of supported configurations, " + + "see https://paimon.apache.org/docs/master/maintenance/configurations/"); + System.out.println(); + + System.out.println("Examples:"); + System.out.println( + " sqlserver-sync-table \\\n" + + " --warehouse hdfs:///path/to/warehouse \\\n" + + " --database test_db \\\n" + + " --table test_table \\\n" + + " --partition-keys pt \\\n" + + " --primary-keys pt,uid \\\n" + + " --sqlserver-conf hostname=127.0.0.1 \\\n" + + " --sqlserver-conf username=root \\\n" + + " --sqlserver-conf password=123456 \\\n" + + " --sqlserver-conf database-name=source_db \\\n" + + " --sqlserver-conf table-name='source_table' \\\n" + + " --catalog-conf metastore=hive \\\n" + + " --catalog-conf uri=thrift://hive-metastore:9083 \\\n" + + " --table-conf bucket=4 \\\n" + + " --table-conf changelog-producer=input \\\n" + + " --table-conf sink.parallelism=4"); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerTypeUtils.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerTypeUtils.java new file mode 100644 index 000000000000..f500b0e93d07 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerTypeUtils.java @@ -0,0 +1,174 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action.cdc.sqlserver; + +import org.apache.paimon.flink.action.cdc.JdbcToPaimonTypeVisitor; +import org.apache.paimon.flink.action.cdc.TypeMapping; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.TimestampType; + +import io.debezium.relational.Column; + +import javax.annotation.Nullable; + +import static org.apache.paimon.flink.action.cdc.TypeMapping.TypeMappingMode.CHAR_TO_STRING; +import static org.apache.paimon.flink.action.cdc.TypeMapping.TypeMappingMode.TINYINT1_NOT_BOOL; +import static org.apache.paimon.flink.action.cdc.TypeMapping.TypeMappingMode.TO_STRING; + +/** Converts from SqlServer type to {@link DataType}. */ +public class SqlServerTypeUtils { + + private static final String BIT = "BIT"; + private static final String TINYINT = "TINYINT"; + private static final String SMALLINT = "SMALLINT"; + private static final String INT = "INT"; + private static final String BIGINT = "BIGINT"; + private static final String REAL = "REAL"; + private static final String FLOAT = "FLOAT"; + private static final String CHAR = "CHAR"; + private static final String VARCHAR = "VARCHAR"; + private static final String TEXT = "TEXT"; + private static final String NCHAR = "NCHAR"; + private static final String NVARCHAR = "NVARCHAR"; + private static final String NTEXT = "NTEXT"; + private static final String XML = "XML"; + private static final String DATETIMEOFFSET = "DATETIMEOFFSET"; + private static final String DATE = "DATE"; + private static final String TIME = "TIME"; + private static final String DATETIME = "DATETIME"; + private static final String SMALLDATETIME = "SMALLDATETIME"; + private static final String DATETIME2 = "DATETIME2"; + private static final String NUMERIC = "NUMERIC"; + private static final String DECIMAL = "DECIMAL"; + private static final String SMALLMONEY = "SMALLMONEY"; + private static final String MONEY = "MONEY"; + + // binary + private static final String BINARY = "BINARY"; + private static final String VARBINARY = "VARBINARY"; + private static final String IMAGE = "IMAGE"; + + // Geospatial Types + private static final String GEOGRAPHY = "GEOGRAPHY"; + private static final String GEOMETRY = "GEOMETRY"; + + /** + * Returns a corresponding Paimon data type from a debezium {@link Column} with nullable always + * be true. + */ + public static DataType toPaimonDataType( + String type, + @Nullable Integer length, + @Nullable Integer scale, + TypeMapping typeMapping) { + if (typeMapping.containsMode(TO_STRING)) { + return DataTypes.STRING(); + } + switch (type.toUpperCase()) { + case BIT: + return DataTypes.BOOLEAN(); + case TINYINT: + return length != null && length == 1 && !typeMapping.containsMode(TINYINT1_NOT_BOOL) + ? DataTypes.BOOLEAN() + : DataTypes.TINYINT(); + case SMALLINT: + return DataTypes.SMALLINT(); + case INT: + return DataTypes.INT(); + case BIGINT: + return DataTypes.BIGINT(); + case REAL: + return DataTypes.DOUBLE(); + case FLOAT: + return DataTypes.FLOAT(); + case CHAR: + return length == null || length == 0 || typeMapping.containsMode(CHAR_TO_STRING) + ? DataTypes.STRING() + : DataTypes.CHAR(length); + case NCHAR: + case NVARCHAR: + case VARCHAR: + return length == null || length == 0 || typeMapping.containsMode(CHAR_TO_STRING) + ? DataTypes.STRING() + : DataTypes.VARCHAR(length); + case TEXT: + case NTEXT: + case XML: + return DataTypes.STRING(); + case DATETIMEOFFSET: + return DataTypes.STRING(); + case DATE: + return DataTypes.DATE(); + case TIME: + if (scale == null || scale <= 0) { + return DataTypes.TIME(); + } else if (scale <= TimestampType.MAX_PRECISION) { + return DataTypes.TIME(scale); + } else { + throw new UnsupportedOperationException( + "Unsupported length " + length + " for SqlServer TIME types"); + } + case DATETIME: + case SMALLDATETIME: + case DATETIME2: + if (scale == null || scale <= 0) { + return DataTypes.TIMESTAMP(0); + } else if (scale <= TimestampType.MAX_PRECISION) { + return DataTypes.TIMESTAMP(scale); + } else { + throw new UnsupportedOperationException( + "Unsupported length " + + length + + " for SqlServer DATETIME、SMALLDATETIME and DATETIME2 types"); + } + case NUMERIC: + case DECIMAL: + return length == null || length <= 38 + ? DataTypes.DECIMAL(length, scale != null && scale >= 0 ? scale : 0) + : DataTypes.STRING(); + case SMALLMONEY: + return DataTypes.DECIMAL(10, 4); + case MONEY: + return DataTypes.DECIMAL(19, 4); + default: + throw new UnsupportedOperationException( + String.format("Don't support SQLServer type '%s' yet.", type)); + } + } + + public static JdbcToPaimonTypeVisitor toPaimonTypeVisitor() { + return SqlServerTypeUtils.SqlServerToPaimonTypeVisitor.INSTANCE; + } + + private static class SqlServerToPaimonTypeVisitor implements JdbcToPaimonTypeVisitor { + + private static final SqlServerTypeUtils.SqlServerToPaimonTypeVisitor INSTANCE = + new SqlServerTypeUtils.SqlServerToPaimonTypeVisitor(); + + @Override + public DataType visit( + String type, + @Nullable Integer length, + @Nullable Integer scale, + TypeMapping typeMapping) { + return toPaimonDataType(type, length, scale, typeMapping); + } + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory b/paimon-flink/paimon-flink-cdc/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory index 917a80a832d4..ba55af1b9dc8 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory +++ b/paimon-flink/paimon-flink-cdc/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory @@ -25,3 +25,6 @@ org.apache.paimon.flink.action.cdc.pulsar.PulsarSyncDatabaseActionFactory org.apache.paimon.flink.action.cdc.mongodb.MongoDBSyncTableActionFactory org.apache.paimon.flink.action.cdc.mongodb.MongoDBSyncDatabaseActionFactory org.apache.paimon.flink.action.cdc.postgres.PostgresSyncTableActionFactory +org.apache.paimon.flink.action.cdc.sqlserver.SqlServerSyncDatabaseActionFactory +org.apache.paimon.flink.action.cdc.sqlserver.SqlServerSyncTableActionFactory + diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java index 63b42e627611..5937cc290ab0 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java @@ -29,6 +29,8 @@ import org.apache.paimon.flink.action.cdc.postgres.PostgresSyncTableActionFactory; import org.apache.paimon.flink.action.cdc.pulsar.PulsarSyncDatabaseActionFactory; import org.apache.paimon.flink.action.cdc.pulsar.PulsarSyncTableActionFactory; +import org.apache.paimon.flink.action.cdc.sqlserver.SqlServerSyncDatabaseActionFactory; +import org.apache.paimon.flink.action.cdc.sqlserver.SqlServerSyncTableActionFactory; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.source.ReadBuilder; import org.apache.paimon.table.source.TableScan; @@ -232,6 +234,10 @@ private String getActionName(Class clazz) { return PulsarSyncDatabaseActionFactory.IDENTIFIER; case "PostgresSyncTableAction": return PostgresSyncTableActionFactory.IDENTIFIER; + case "SqlServerSyncTableAction": + return SqlServerSyncTableActionFactory.IDENTIFIER; + case "SqlServerSyncDatabaseAction": + return SqlServerSyncDatabaseActionFactory.IDENTIFIER; default: throw new UnsupportedOperationException( "Unknown sync action: " + clazz.getSimpleName()); @@ -254,6 +260,9 @@ private String getConfKey(Class clazz) { return "--" + CdcActionCommonUtils.PULSAR_CONF; case "PostgresSyncTableAction": return "--" + CdcActionCommonUtils.POSTGRES_CONF; + case "SqlServerSyncTableAction": + case "SqlServerSyncDatabaseAction": + return "--" + CdcActionCommonUtils.SQLSERVER_CONF; default: throw new UnsupportedOperationException( "Unknown sync action: " + clazz.getSimpleName()); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlActionITCaseBase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlActionITCaseBase.java index 010041268c8a..393e93b1d94f 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlActionITCaseBase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlActionITCaseBase.java @@ -84,10 +84,9 @@ protected Map getBasicMySqlConfig() { config.put("password", PASSWORD); // see mysql/my.cnf in test resources config.put("server-time-zone", ZoneId.of("America/New_York").toString()); - - // TODO When setting the parameter scan.newly-added-table.enabled=true in version 2.4.2, the - // Insert data inserted after the newly created table cannot be captured. When set to false, - // the mysql cdc works normally. + // Configure after upgrading to version 2.4.1; + // Refer: + // https://github.com/ververica/flink-cdc-connectors/blob/release-2.4/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/reader/BinlogSplitReader.java#L272 config.put("scan.newly-added-table.enabled", "false"); return config; } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/LicenseAcceptanceUtils.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/LicenseAcceptanceUtils.java new file mode 100644 index 000000000000..d319dbd8a50c --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/LicenseAcceptanceUtils.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action.cdc.sqlserver; + +import org.testcontainers.shaded.com.google.common.base.Charsets; +import org.testcontainers.shaded.com.google.common.io.Resources; + +import java.net.URL; +import java.util.List; +import java.util.stream.Stream; + +/** Copy from org.testcontainers.utility.LicenseAcceptance, modify license location. */ +public final class LicenseAcceptanceUtils { + private static final String ACCEPTANCE_FILE_NAME = + "META-INF/licenses/container-license-acceptance.txt"; + + public static void assertLicenseAccepted(String imageName) { + try { + URL url = Resources.getResource(ACCEPTANCE_FILE_NAME); + List acceptedLicences = Resources.readLines(url, Charsets.UTF_8); + Stream licenses = acceptedLicences.stream().map(String::trim); + imageName.getClass(); + if (licenses.anyMatch(imageName::equals)) { + return; + } + } catch (Exception exception) { + } + + throw new IllegalStateException( + "The image " + + imageName + + " requires you to accept a license agreement. Please place a file at the root of the classpath named " + + "META-INF/licenses/container-license-acceptance.txt" + + ", e.g. at src/test/resources/" + + "META-INF/licenses/container-license-acceptance.txt" + + ". This file should contain the line:\n " + + imageName); + } + + private LicenseAcceptanceUtils() { + throw new UnsupportedOperationException( + "This is a utility class and cannot be instantiated"); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/MSSQLServerContainerExtend.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/MSSQLServerContainerExtend.java new file mode 100644 index 000000000000..1a219b03fecd --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/MSSQLServerContainerExtend.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action.cdc.sqlserver; + +import org.testcontainers.containers.MSSQLServerContainer; + +/** MSSQLServerContainer extend. */ +public class MSSQLServerContainerExtend extends MSSQLServerContainer { + + public MSSQLServerContainerExtend(String dockerImageName) { + super(dockerImageName); + } + + @Override + protected void configure() { + if (!this.getEnvMap().containsKey("ACCEPT_EULA")) { + LicenseAcceptanceUtils.assertLicenseAccepted(this.getDockerImageName()); + this.acceptLicense(); + } + this.addEnv("SA_PASSWORD", this.getPassword()); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerActionITCaseBase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerActionITCaseBase.java new file mode 100644 index 000000000000..f6f0142711ae --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerActionITCaseBase.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action.cdc.sqlserver; + +import org.apache.paimon.flink.action.cdc.CdcActionITCaseBase; + +import org.junit.jupiter.api.AfterAll; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.MSSQLServerContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Stream; + +import static org.testcontainers.containers.MSSQLServerContainer.MS_SQL_SERVER_PORT; + +/** Base test class for {@link org.apache.paimon.flink.action.Action}s related to SqlServer. */ +public class SqlServerActionITCaseBase extends CdcActionITCaseBase { + private static final Logger LOG = LoggerFactory.getLogger(SqlServerActionITCaseBase.class); + public static final MSSQLServerContainer MSSQL_SERVER_CONTAINER = createSqlServerContainer(); + private static final String PASSWORD = "Password!"; + + public static final String ENABLE_TABLE_CDC_WITH_CUSTOM_CAPTURE = + "EXEC sys.sp_cdc_enable_table @source_schema = N'dbo', @source_name = N'%s', @capture_instance = N'%s', @role_name = NULL, @supports_net_changes = 0, @captured_column_list = %s"; + public static final String DISABLE_TABLE_CDC = + "EXEC sys.sp_cdc_disable_table @source_schema = N'dbo', @source_name = N'#', @capture_instance = 'all'"; + private static final String STATEMENTS_PLACEHOLDER = "#"; + + @AfterAll + public static void stopContainers() { + LOG.info("Stopping containers..."); + MSSQL_SERVER_CONTAINER.stop(); + LOG.info("Containers are stopped."); + } + + private static MSSQLServerContainer createSqlServerContainer() { + return (MSSQLServerContainer) + new MSSQLServerContainerExtend("mcr.microsoft.com/mssql/server:2019-latest") + .withPassword(PASSWORD) + .withEnv("MSSQL_AGENT_ENABLED", "true") + .withEnv("MSSQL_PID", "Standard") + .withLogConsumer(new Slf4jLogConsumer(LOG)); + } + + protected static void start() { + LOG.info("Starting containers..."); + Startables.deepStart(Stream.of(MSSQL_SERVER_CONTAINER)).join(); + LOG.info("Containers are started."); + } + + protected static Statement getStatement() throws SQLException { + Connection conn = + DriverManager.getConnection( + MSSQL_SERVER_CONTAINER.getJdbcUrl(), + MSSQL_SERVER_CONTAINER.getUsername(), + MSSQL_SERVER_CONTAINER.getPassword()); + return conn.createStatement(); + } + + protected Map getBasicSqlServerConfig() { + Map config = new HashMap<>(); + config.put("hostname", MSSQL_SERVER_CONTAINER.getHost()); + config.put("username", MSSQL_SERVER_CONTAINER.getUsername()); + config.put("password", MSSQL_SERVER_CONTAINER.getPassword()); + config.put(SqlServerSourceOptions.SCHEMA_NAME.key(), "dbo"); + config.put("port", this.getPort().toString()); + return config; + } + + protected SqlServerSyncTableActionBuilder syncTableActionBuilder( + Map sqlServerConfig) { + return new SqlServerSyncTableActionBuilder(sqlServerConfig); + } + + protected SqlServerSyncDatabaseActionBuilder syncDatabaseActionBuilder( + Map sqlServerConfig) { + return new SqlServerSyncDatabaseActionBuilder(sqlServerConfig); + } + + /** Builder to build {@link SqlServerSyncTableAction} from action arguments. */ + protected class SqlServerSyncTableActionBuilder + extends SyncTableActionBuilder { + + public SqlServerSyncTableActionBuilder(Map sqlServerConfig) { + super(SqlServerSyncTableAction.class, sqlServerConfig); + } + } + + public Integer getPort() { + return MSSQL_SERVER_CONTAINER.getMappedPort(MS_SQL_SERVER_PORT); + } + + /** Builder to build {@link SqlServerSyncDatabaseAction} from action arguments. */ + protected class SqlServerSyncDatabaseActionBuilder + extends SyncDatabaseActionBuilder { + + public SqlServerSyncDatabaseActionBuilder(Map sqlServerConfig) { + super(SqlServerSyncDatabaseAction.class, sqlServerConfig); + } + } + + public static void enableTableCdc(Statement connection, String tableName, String captureName) + throws SQLException { + Objects.requireNonNull(tableName); + Objects.requireNonNull(captureName); + String enableCdcForTableStmt = + String.format(ENABLE_TABLE_CDC_WITH_CUSTOM_CAPTURE, tableName, captureName, "NULL"); + connection.execute(enableCdcForTableStmt); + } + + /** + * Disables CDC for a table for which it was enabled before. + * + * @param name the name of the table, may not be {@code null} + * @throws SQLException if anything unexpected fails + */ + public static void disableTableCdc(Statement connection, String name) throws SQLException { + Objects.requireNonNull(name); + String disableCdcForTableStmt = DISABLE_TABLE_CDC.replace(STATEMENTS_PLACEHOLDER, name); + connection.execute(disableCdcForTableStmt); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncDatabaseActionITCase.java new file mode 100644 index 000000000000..cd8c6f554345 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncDatabaseActionITCase.java @@ -0,0 +1,497 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action.cdc.sqlserver; + +import org.apache.paimon.catalog.FileSystemCatalogOptions; +import org.apache.paimon.flink.action.MultiTablesSinkMode; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.JsonSerdeUtil; + +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import javax.annotation.Nullable; + +import java.sql.Statement; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; + +import static org.apache.paimon.flink.action.MultiTablesSinkMode.COMBINED; +import static org.apache.paimon.flink.action.MultiTablesSinkMode.DIVIDED; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** IT cases for {@link SqlServerSyncDatabaseActionITCase}. */ +public class SqlServerSyncDatabaseActionITCase extends SqlServerActionITCaseBase { + + @BeforeAll + public static void startContainers() { + MSSQL_SERVER_CONTAINER.withInitScript("sqlserver/sync_database_setup.sql"); + start(); + } + + @Test + @Timeout(120) + public void testSyncSqlServerDatabase() throws Exception { + Map sqlServerConfig = getBasicSqlServerConfig(); + sqlServerConfig.put("database-name", "paimon_sync_database"); + + SqlServerSyncDatabaseAction action = + syncDatabaseActionBuilder(sqlServerConfig) + .withTableConfig(getBasicTableConfig()) + .build(); + runActionWithDefaultEnv(action); + + try (Statement statement = getStatement()) { + testSyncSqlServerDatabaseImpl(statement); + } + } + + private void testSyncSqlServerDatabaseImpl(Statement statement) throws Exception { + FileStoreTable table1 = getFileStoreTable("t1"); + FileStoreTable table2 = getFileStoreTable("t2"); + statement.executeUpdate("USE paimon_sync_database"); + statement.executeUpdate("INSERT INTO dbo.t1 VALUES (1, 'one')"); + statement.executeUpdate("INSERT INTO dbo.t2 VALUES (2, 'two', 20, 200)"); + statement.executeUpdate("INSERT INTO dbo.t1 VALUES (3, 'three')"); + statement.executeUpdate("INSERT INTO dbo.t2 VALUES (4, 'four', 40, 400)"); + statement.executeUpdate("INSERT INTO dbo.t3 VALUES (-1)"); + + RowType rowType1 = + RowType.of( + new DataType[] {DataTypes.INT().notNull(), DataTypes.VARCHAR(10)}, + new String[] {"k", "v1"}); + List primaryKeys1 = Collections.singletonList("k"); + List expected = Arrays.asList("+I[1, one]", "+I[3, three]"); + waitForResult(expected, table1, rowType1, primaryKeys1); + + RowType rowType2 = + RowType.of( + new DataType[] { + DataTypes.INT().notNull(), + DataTypes.VARCHAR(10).notNull(), + DataTypes.INT(), + DataTypes.BIGINT() + }, + new String[] {"k1", "k2", "v1", "v2"}); + List primaryKeys2 = Arrays.asList("k1", "k2"); + expected = Arrays.asList("+I[2, two, 20, 200]", "+I[4, four, 40, 400]"); + waitForResult(expected, table2, rowType2, primaryKeys2); + } + + @Test + public void testSpecifiedSqlServerTable() { + Map sqlServerConfig = getBasicSqlServerConfig(); + sqlServerConfig.put("database-name", "paimon_sync_database"); + sqlServerConfig.put("table-name", "my_table"); + SqlServerSyncDatabaseAction action = syncDatabaseActionBuilder(sqlServerConfig).build(); + assertThatThrownBy(action::run).isInstanceOf(IllegalArgumentException.class); + } + + @Test + @Timeout(60) + public void testIgnoreIncompatibleTables() throws Exception { + // create an incompatible table + createFileStoreTable( + "incompatible", + RowType.of( + new DataType[] {DataTypes.STRING(), DataTypes.STRING()}, + new String[] {"k", "v1"}), + Collections.emptyList(), + Collections.singletonList("k"), + Collections.emptyMap()); + + // try synchronization + Map sqlServerConfig = getBasicSqlServerConfig(); + sqlServerConfig.put("database-name", "paimon_sync_database_ignore_incompatible"); + SqlServerSyncDatabaseAction action = + syncDatabaseActionBuilder(sqlServerConfig) + .withTableConfig(getBasicTableConfig()) + .ignoreIncompatible(true) + .build(); + runActionWithDefaultEnv(action); + + // validate `compatible` can be synchronized + try (Statement statement = getStatement()) { + FileStoreTable table = getFileStoreTable("compatible"); + + statement.executeUpdate("USE paimon_sync_database_ignore_incompatible"); + statement.executeUpdate("INSERT INTO compatible VALUES (2, 'two', 20, 200)"); + statement.executeUpdate("INSERT INTO compatible VALUES (4, 'four', 40, 400)"); + + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.INT().notNull(), + DataTypes.VARCHAR(10).notNull(), + DataTypes.INT(), + DataTypes.BIGINT() + }, + new String[] {"k1", "k2", "v1", "v2"}); + List primaryKeys2 = Arrays.asList("k1", "k2"); + List expected = Arrays.asList("+I[2, two, 20, 200]", "+I[4, four, 40, 400]"); + waitForResult(expected, table, rowType, primaryKeys2); + } + } + + @Test + @Timeout(120) + public void testTableAffix() throws Exception { + // create table t1 + createFileStoreTable( + "test_prefix_t1_test_suffix", + RowType.of( + new DataType[] {DataTypes.INT(), DataTypes.VARCHAR(10)}, + new String[] {"k1", "v0"}), + Collections.emptyList(), + Collections.singletonList("k1"), + Collections.emptyMap()); + + // try synchronization + Map sqlServerConfig = getBasicSqlServerConfig(); + sqlServerConfig.put("database-name", "paimon_sync_database_affix"); + + SqlServerSyncDatabaseAction action = + syncDatabaseActionBuilder(sqlServerConfig) + .withTableConfig(getBasicTableConfig()) + .withTablePrefix("test_prefix_") + .withTableSuffix("_test_suffix") + // test including check with affix + .includingTables(ThreadLocalRandom.current().nextBoolean() ? "t1|t2" : ".*") + .build(); + runActionWithDefaultEnv(action); + + try (Statement statement = getStatement()) { + testTableAffixImpl(statement); + } + } + + private void testTableAffixImpl(Statement statement) throws Exception { + FileStoreTable table1 = getFileStoreTable("test_prefix_t1_test_suffix"); + FileStoreTable table2 = getFileStoreTable("test_prefix_t2_test_suffix"); + + statement.executeUpdate("USE paimon_sync_database_affix"); + + statement.executeUpdate("INSERT INTO t1 VALUES (1, 'one')"); + statement.executeUpdate("INSERT INTO t2 VALUES (2, 'two')"); + statement.executeUpdate("INSERT INTO t1 VALUES (3, 'three')"); + statement.executeUpdate("INSERT INTO t2 VALUES (4, 'four')"); + + RowType rowType1 = + RowType.of( + new DataType[] {DataTypes.INT().notNull(), DataTypes.VARCHAR(10)}, + new String[] {"k1", "v0"}); + List primaryKeys1 = Collections.singletonList("k1"); + List expected = Arrays.asList("+I[1, one]", "+I[3, three]"); + waitForResult(expected, table1, rowType1, primaryKeys1); + + RowType rowType2 = + RowType.of( + new DataType[] {DataTypes.INT().notNull(), DataTypes.VARCHAR(10)}, + new String[] {"k2", "v0"}); + List primaryKeys2 = Collections.singletonList("k2"); + expected = Arrays.asList("+I[2, two]", "+I[4, four]"); + waitForResult(expected, table2, rowType2, primaryKeys2); + } + + @Test + @Timeout(60) + public void testIncludingTables() throws Exception { + includingAndExcludingTablesImpl( + "paimon_sync_database_including", + "flink|paimon.+", + null, + Arrays.asList("flink", "paimon_1", "paimon_2"), + Collections.singletonList("ignored")); + } + + @Test + @Timeout(60) + public void testExcludingTables() throws Exception { + includingAndExcludingTablesImpl( + "paimon_sync_database_excluding", + null, + "flink|paimon.+", + Collections.singletonList("sync"), + Arrays.asList("flink", "paimon_1", "paimon_2")); + } + + @Test + @Timeout(60) + public void testIncludingAndExcludingTables() throws Exception { + includingAndExcludingTablesImpl( + "paimon_sync_database_in_excluding", + "flink|paimon.+", + "paimon_1", + Arrays.asList("flink", "paimon_2"), + Arrays.asList("paimon_1", "test")); + } + + private void includingAndExcludingTablesImpl( + String databaseName, + @Nullable String includingTables, + @Nullable String excludingTables, + List existedTables, + List notExistedTables) + throws Exception { + // try synchronization + Map sqlServerConfig = getBasicSqlServerConfig(); + sqlServerConfig.put("database-name", databaseName); + SqlServerSyncDatabaseAction action = + syncDatabaseActionBuilder(sqlServerConfig) + .withTableConfig(getBasicTableConfig()) + .includingTables(includingTables) + .excludingTables(excludingTables) + .build(); + runActionWithDefaultEnv(action); + + // check paimon tables + assertExactlyExistTables(existedTables); + assertTableNotExists(notExistedTables); + } + + @Test + @Timeout(60) + public void testIgnoreCase() throws Exception { + Map sqlServerConfig = getBasicSqlServerConfig(); + sqlServerConfig.put("database-name", "paimon_ignore_CASE"); + + SqlServerSyncDatabaseAction action = + syncDatabaseActionBuilder(sqlServerConfig) + .withCatalogConfig( + Collections.singletonMap( + FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + .withTableConfig(getBasicTableConfig()) + .build(); + runActionWithDefaultEnv(action); + + // check table schema + FileStoreTable table = getFileStoreTable("t"); + assertThat(JsonSerdeUtil.toFlatJson(table.schema().fields())) + .isEqualTo( + "[{\"id\":0,\"name\":\"k\",\"type\":\"INT NOT NULL\"}," + + "{\"id\":1,\"name\":\"uppercase_v0\",\"type\":\"VARCHAR(20)\"}]"); + } + + @Test + public void testCatalogAndTableConfig() { + SqlServerSyncDatabaseAction action = + syncDatabaseActionBuilder(getBasicSqlServerConfig()) + .withCatalogConfig(Collections.singletonMap("catalog-key", "catalog-value")) + .withTableConfig(Collections.singletonMap("table-key", "table-value")) + .build(); + + assertThat(action.catalogConfig()).containsEntry("catalog-key", "catalog-value"); + assertThat(action.tableConfig()) + .containsExactlyEntriesOf(Collections.singletonMap("table-key", "table-value")); + } + + @Test + @Timeout(120) + public void testMetadataColumns() throws Exception { + Map sqlServerConfig = getBasicSqlServerConfig(); + sqlServerConfig.put("database-name", "metadata"); + + MultiTablesSinkMode mode = ThreadLocalRandom.current().nextBoolean() ? DIVIDED : COMBINED; + SqlServerSyncDatabaseAction action = + syncDatabaseActionBuilder(sqlServerConfig) + .withTableConfig(getBasicTableConfig()) + .withMode(mode.configString()) + .withMetadataColumn(Arrays.asList("table_name", "database_name")) + .build(); + runActionWithDefaultEnv(action); + + try (Statement statement = getStatement()) { + statement.executeUpdate("USE metadata"); + statement.executeUpdate("INSERT INTO t1 VALUES (1, 'db1_1')"); + statement.executeUpdate("INSERT INTO t1 VALUES (2, 'db1_2')"); + + statement.executeUpdate("INSERT INTO t1 VALUES (3, 'db2_3')"); + statement.executeUpdate("INSERT INTO t1 VALUES (4, 'db2_4')"); + + FileStoreTable table = getFileStoreTable("t1"); + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.INT().notNull(), + DataTypes.VARCHAR(10), + DataTypes.STRING().notNull(), + DataTypes.STRING().notNull() + }, + new String[] {"k", "v1", "table_name", "database_name"}); + waitForResult( + Arrays.asList( + "+I[1, db1_1, t1, metadata]", + "+I[2, db1_2, t1, metadata]", + "+I[3, db2_3, t1, metadata]", + "+I[4, db2_4, t1, metadata]"), + table, + rowType, + Collections.singletonList("k")); + + statement.executeUpdate("INSERT INTO t2 VALUES (1, 'db1_1')"); + statement.executeUpdate("INSERT INTO t2 VALUES (2, 'db1_2')"); + statement.executeUpdate("INSERT INTO t2 VALUES (3, 'db1_3')"); + statement.executeUpdate("INSERT INTO t2 VALUES (4, 'db1_4')"); + table = getFileStoreTable("t2"); + rowType = + RowType.of( + new DataType[] { + DataTypes.INT().notNull(), + DataTypes.VARCHAR(10), + DataTypes.STRING().notNull(), + DataTypes.STRING().notNull() + }, + new String[] {"k", "v1", "table_name", "database_name"}); + waitForResult( + Arrays.asList( + "+I[1, db1_1, t2, metadata]", + "+I[2, db1_2, t2, metadata]", + "+I[3, db1_3, t2, metadata]", + "+I[4, db1_4, t2, metadata]"), + table, + rowType, + Collections.singletonList("k")); + } + } + + @Test + @Timeout(120) + public void testSyncMultipleShards() throws Exception { + Map sqlServerConfig = getBasicSqlServerConfig(); + // test table list + sqlServerConfig.put("database-name", "database_shard"); + sqlServerConfig.put( + "schema-name", + ThreadLocalRandom.current().nextBoolean() ? "schema_.*" : "schema_1|schema_2"); + MultiTablesSinkMode mode = ThreadLocalRandom.current().nextBoolean() ? DIVIDED : COMBINED; + SqlServerSyncDatabaseAction action = + syncDatabaseActionBuilder(sqlServerConfig) + .withTableConfig(getBasicTableConfig()) + .withMode(mode.configString()) + .build(); + runActionWithDefaultEnv(action); + + try (Statement statement = getStatement()) { + // test insert into t1 + statement.executeUpdate("USE database_shard"); + statement.executeUpdate("INSERT INTO schema_1.t1 VALUES (1, 'db1_1')"); + statement.executeUpdate("INSERT INTO schema_1.t1 VALUES (2, 'db1_2')"); + + statement.executeUpdate("INSERT INTO schema_2.t1 VALUES (3, 'db2_3', 300)"); + statement.executeUpdate("INSERT INTO schema_2.t1 VALUES (4, 'db2_4', 400)"); + + FileStoreTable table = getFileStoreTable("t1"); + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.INT().notNull(), DataTypes.VARCHAR(20), DataTypes.BIGINT() + }, + new String[] {"k", "v1", "v2"}); + waitForResult( + Arrays.asList( + "+I[1, db1_1, NULL]", + "+I[2, db1_2, NULL]", + "+I[3, db2_3, 300]", + "+I[4, db2_4, 400]"), + table, + rowType, + Collections.singletonList("k")); + + // test that database_shard_schema_2.t3 won't be synchronized + statement.executeUpdate("USE database_shard"); + statement.executeUpdate("INSERT INTO schema_2.t3 VALUES (1, 'db2_1'), (2, 'db2_2')"); + statement.executeUpdate("INSERT INTO schema_1.t3 VALUES (3, 'db1_3'), (4, 'db1_4')"); + table = getFileStoreTable("t3"); + rowType = + RowType.of( + new DataType[] {DataTypes.INT().notNull(), DataTypes.VARCHAR(10)}, + new String[] {"k", "v1"}); + waitForResult( + Arrays.asList("+I[3, db1_3]", "+I[4, db1_4]"), + table, + rowType, + Collections.singletonList("k")); + } + } + + @Test + @Timeout(120) + public void testSyncMultipleShardsWithoutMerging() throws Exception { + Map sqlServerConfig = getBasicSqlServerConfig(); + sqlServerConfig.put("database-name", "without_merging_shard"); + sqlServerConfig.put("schema-name", "schema_.*"); + + MultiTablesSinkMode mode = ThreadLocalRandom.current().nextBoolean() ? DIVIDED : COMBINED; + SqlServerSyncDatabaseAction action = + syncDatabaseActionBuilder(sqlServerConfig) + .withTableConfig(getBasicTableConfig()) + .mergeShards(false) + .withMode(mode.configString()) + .build(); + runActionWithDefaultEnv(action); + + try (Statement statement = getStatement()) { + Thread.sleep(5_000); + + assertExactlyExistTables( + "without_merging_shard_schema_1_t1", + "without_merging_shard_schema_1_t2", + "without_merging_shard_schema_2_t1"); + + // test insert into without_merging_shard_1.t1 + statement.executeUpdate("USE without_merging_shard"); + statement.executeUpdate("INSERT INTO schema_1.t1 VALUES (1, 'db1_1'), (2, 'db1_2')"); + FileStoreTable table = getFileStoreTable("without_merging_shard_schema_1_t1"); + RowType rowType = + RowType.of( + new DataType[] {DataTypes.INT().notNull(), DataTypes.VARCHAR(10)}, + new String[] {"k", "v1"}); + waitForResult( + Arrays.asList("+I[1, db1_1]", "+I[2, db1_2]"), + table, + rowType, + Collections.singletonList("k")); + + // test insert into without_merging_shard_2.t1 + statement.executeUpdate("USE without_merging_shard"); + statement.executeUpdate( + "INSERT INTO schema_2.t1 VALUES (3, 'db2_3', 300), (4, 'db2_4', 400)"); + table = getFileStoreTable("without_merging_shard_schema_2_t1"); + rowType = + RowType.of( + new DataType[] { + DataTypes.INT().notNull(), DataTypes.VARCHAR(20), DataTypes.BIGINT() + }, + new String[] {"k", "v1", "v2"}); + waitForResult( + Arrays.asList("+I[3, db2_3, 300]", "+I[4, db2_4, 400]"), + table, + rowType, + Collections.singletonList("k")); + } + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncDatabaseTableListITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncDatabaseTableListITCase.java new file mode 100644 index 000000000000..d332af64ded4 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncDatabaseTableListITCase.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action.cdc.sqlserver; + +import org.apache.paimon.flink.action.MultiTablesSinkMode; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; + +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.sql.Statement; +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; + +import static org.apache.paimon.flink.action.MultiTablesSinkMode.COMBINED; +import static org.apache.paimon.flink.action.MultiTablesSinkMode.DIVIDED; + +/** Test if the table list in {@link SqlServerSyncDatabaseAction} is correct. */ +public class SqlServerSyncDatabaseTableListITCase extends SqlServerActionITCaseBase { + + @BeforeAll + public static void startContainers() { + MSSQL_SERVER_CONTAINER.withInitScript("sqlserver/tablelist_test_setup.sql"); + start(); + } + + @Test + @Timeout(120) + public void testActionRunResult() throws Exception { + Map sqlserverConfig = getBasicSqlServerConfig(); + sqlserverConfig.put("database-name", "shard_database"); + sqlserverConfig.put( + "schema-name", + ThreadLocalRandom.current().nextBoolean() + ? ".*shard_.*" + : "shard_1|shard_2|shard_3|x_shard_1"); + MultiTablesSinkMode mode = ThreadLocalRandom.current().nextBoolean() ? DIVIDED : COMBINED; + SqlServerSyncDatabaseAction action = + syncDatabaseActionBuilder(sqlserverConfig) + .withTableConfig(getBasicTableConfig()) + .mergeShards(false) + .withMode(mode.configString()) + .includingTables("t.+|s.+") + .excludingTables("ta|sa") + .build(); + runActionWithDefaultEnv(action); + + assertExactlyExistTables( + "shard_database_shard_1_t11", + "shard_database_shard_1_t2", + "shard_database_shard_1_t3", + "shard_database_shard_1_taa", + "shard_database_shard_1_s2", + "shard_database_shard_2_t1", + "shard_database_shard_2_t22", + "shard_database_shard_2_t3", + "shard_database_shard_2_tb", + "shard_database_x_shard_1_t1"); + + try (Statement statement = getStatement()) { + // ensure the job steps into incremental phase + statement.executeUpdate("USE shard_database"); + statement.executeUpdate("INSERT INTO shard_1.t2 VALUES (1, 'A')"); + } + waitForResult( + Collections.singletonList("+I[1, A]"), + getFileStoreTable("shard_database_shard_1_t2"), + RowType.of( + new DataType[] {DataTypes.INT().notNull(), DataTypes.VARCHAR(100)}, + new String[] {"k", "name"}), + Collections.singletonList("k")); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncTableActionITCase.java new file mode 100644 index 000000000000..ec46e25b3408 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/sqlserver/SqlServerSyncTableActionITCase.java @@ -0,0 +1,731 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action.cdc.sqlserver; + +import org.apache.paimon.options.CatalogOptions; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.JsonSerdeUtil; + +import com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions; +import org.apache.flink.core.execution.JobClient; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.sql.Statement; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicLong; + +import static org.apache.paimon.testutils.assertj.PaimonAssertions.anyCauseMatches; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** IT cases for {@link SqlServerSyncTableAction}. */ +public class SqlServerSyncTableActionITCase extends SqlServerActionITCaseBase { + + private static final String DATABASE_NAME = "paimon_sync_table"; + + private static AtomicLong seq = new AtomicLong(0); + + @BeforeAll + public static void startContainers() { + MSSQL_SERVER_CONTAINER.withInitScript("sqlserver/sync_table_setup.sql"); + start(); + } + + @Test + @Timeout(300) + @Disabled + public void testSchemaEvolution() throws Exception { + Map sqlServerConfig = getBasicSqlServerConfig(); + sqlServerConfig.put(SqlServerSourceOptions.DATABASE_NAME.key(), DATABASE_NAME); + sqlServerConfig.put(SqlServerSourceOptions.TABLE_NAME.key(), "schema_evolution_\\d+"); + + SqlServerSyncTableAction action = + syncTableActionBuilder(sqlServerConfig) + .withCatalogConfig( + Collections.singletonMap( + CatalogOptions.METASTORE.key(), "test-alter-table")) + .withTableConfig(getBasicTableConfig()) + .withPartitionKeys("pt") + .withPrimaryKeys("pt", "_id") + .build(); + runActionWithDefaultEnv(action); + + checkTableSchema( + "[{\"id\":0,\"name\":\"pt\",\"type\":\"INT NOT NULL\"}," + + "{\"id\":1,\"name\":\"_id\",\"type\":\"INT NOT NULL\"}," + + "{\"id\":2,\"name\":\"v1\",\"type\":\"VARCHAR(10)\"}]"); + + try (Statement statement = getStatement()) { + testSchemaEvolutionImpl(statement); + } + } + + private void testSchemaEvolutionImpl(Statement statement) throws Exception { + FileStoreTable table = getFileStoreTable(); + statement.execute("USE " + DATABASE_NAME); + statement.executeUpdate("INSERT INTO schema_evolution_1 VALUES (1, 1, 'one')"); + statement.executeUpdate( + "INSERT INTO schema_evolution_2 VALUES (1, 2, 'two'), (2, 4, 'four')"); + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.INT().notNull(), + DataTypes.INT().notNull(), + DataTypes.VARCHAR(10) + }, + new String[] {"pt", "_id", "v1"}); + List primaryKeys = Arrays.asList("pt", "_id"); + List expected = Arrays.asList("+I[1, 1, one]", "+I[1, 2, two]", "+I[2, 4, four]"); + waitForResult(expected, table, rowType, primaryKeys); + + statement.executeUpdate("ALTER TABLE schema_evolution_1 ADD v2 INT"); + enableTableCdc( + statement, + "schema_evolution_1", + String.format("%s_%s", "schema_evolution_1", seq.incrementAndGet())); + statement.executeUpdate( + "INSERT INTO schema_evolution_1 VALUES (2, 3, 'three', 30), (1, 5, 'five', 50)"); + + statement.executeUpdate("ALTER TABLE schema_evolution_2 ADD v2 INT"); + enableTableCdc( + statement, + "schema_evolution_2", + String.format("%s_%s", "schema_evolution_2", seq.incrementAndGet())); + + statement.executeUpdate("INSERT INTO schema_evolution_2 VALUES (1, 6, 'six', 60)"); + statement.executeUpdate("UPDATE schema_evolution_2 SET v1 = 'second' WHERE _id = 2"); + + rowType = + RowType.of( + new DataType[] { + DataTypes.INT().notNull(), + DataTypes.INT().notNull(), + DataTypes.VARCHAR(10), + DataTypes.INT() + }, + new String[] {"pt", "_id", "v1", "v2"}); + expected = + Arrays.asList( + "+I[1, 1, one, NULL]", + "+I[1, 2, second, NULL]", + "+I[2, 3, three, 30]", + "+I[2, 4, four, NULL]", + "+I[1, 5, five, 50]", + "+I[1, 6, six, 60]"); + waitForResult(expected, table, rowType, primaryKeys); + + statement.executeUpdate("ALTER TABLE schema_evolution_1 ALTER COLUMN v2 BIGINT"); + enableTableCdc( + statement, + "schema_evolution_1", + String.format("%_%", "schema_evolution_1", seq.incrementAndGet())); + + statement.executeUpdate( + "INSERT INTO schema_evolution_1 VALUES (2, 7, 'seven', 70000000000)"); + statement.executeUpdate("DELETE FROM schema_evolution_1 WHERE _id = 5"); + statement.executeUpdate("UPDATE schema_evolution_1 SET v2 = 30000000000 WHERE _id = 3"); + + statement.executeUpdate("ALTER TABLE schema_evolution_2 ALTER COLUMN v2 BIGINT"); + + statement.executeUpdate( + "INSERT INTO schema_evolution_2 VALUES (2, 8, 'eight', 80000000000)"); + rowType = + RowType.of( + new DataType[] { + DataTypes.INT().notNull(), + DataTypes.INT().notNull(), + DataTypes.VARCHAR(10), + DataTypes.BIGINT() + }, + new String[] {"pt", "_id", "v1", "v2"}); + expected = + Arrays.asList( + "+I[1, 1, one, NULL]", + "+I[1, 2, second, NULL]", + "+I[2, 3, three, 30000000000]", + "+I[2, 4, four, NULL]", + "+I[1, 6, six, 60]", + "+I[2, 7, seven, 70000000000]", + "+I[2, 8, eight, 80000000000]"); + waitForResult(expected, table, rowType, primaryKeys); + + statement.executeUpdate("ALTER TABLE schema_evolution_1 ADD v3 NUMERIC(8, 3)"); + statement.executeUpdate("ALTER TABLE schema_evolution_1 ADD v4 BYTEA"); + statement.executeUpdate("ALTER TABLE schema_evolution_1 ADD v5 FLOAT"); + statement.executeUpdate("ALTER TABLE schema_evolution_1 ALTER COLUMN v1 VARCHAR(20)"); + statement.executeUpdate( + "INSERT INTO schema_evolution_1 VALUES (1, 9, 'nine', 90000000000, 99999.999, 'nine.bin', 9.9)"); + statement.executeUpdate("ALTER TABLE schema_evolution_2 ADD v3 NUMERIC(8, 3)"); + statement.executeUpdate("ALTER TABLE schema_evolution_2 ADD v4 BYTEA"); + statement.executeUpdate("ALTER TABLE schema_evolution_2 ADD v5 FLOAT"); + statement.executeUpdate("ALTER TABLE schema_evolution_2 ALTER COLUMN v1 VARCHAR(20)"); + statement.executeUpdate( + "UPDATE schema_evolution_2 SET v1 = 'very long string' WHERE _id = 8"); + rowType = + RowType.of( + new DataType[] { + DataTypes.INT().notNull(), + DataTypes.INT().notNull(), + DataTypes.VARCHAR(20), + DataTypes.BIGINT(), + DataTypes.DECIMAL(8, 3), + DataTypes.VARBINARY(10), + DataTypes.FLOAT() + }, + new String[] {"pt", "_id", "v1", "v2", "v3", "v4", "v5"}); + expected = + Arrays.asList( + "+I[1, 1, one, NULL, NULL, NULL, NULL]", + "+I[1, 2, second, NULL, NULL, NULL, NULL]", + "+I[2, 3, three, 30000000000, NULL, NULL, NULL]", + "+I[2, 4, four, NULL, NULL, NULL, NULL]", + "+I[1, 6, six, 60, NULL, NULL, NULL]", + "+I[2, 7, seven, 70000000000, NULL, NULL, NULL]", + "+I[2, 8, very long string, 80000000000, NULL, NULL, NULL]", + "+I[1, 9, nine, 90000000000, 99999.999, [110, 105, 110, 101, 46, 98, 105, 110], 9.9]"); + waitForResult(expected, table, rowType, primaryKeys); + + statement.executeUpdate("ALTER TABLE schema_evolution_1 ALTER COLUMN v5 DOUBLE PRECISION"); + statement.executeUpdate( + "UPDATE schema_evolution_1 SET v4 = 'nine.bin.long', v5 = 9.00000000009 WHERE _id = 9"); + statement.executeUpdate("ALTER TABLE schema_evolution_2 ALTER COLUMN v5 DOUBLE PRECISION"); + statement.executeUpdate( + "UPDATE schema_evolution_2 SET v4 = 'four.bin.long', v5 = 4.00000000004 WHERE _id = 4"); + rowType = + RowType.of( + new DataType[] { + DataTypes.INT().notNull(), + DataTypes.INT().notNull(), + DataTypes.VARCHAR(20), + DataTypes.BIGINT(), + DataTypes.DECIMAL(8, 3), + DataTypes.VARBINARY(20), + DataTypes.DOUBLE() + }, + new String[] {"pt", "_id", "v1", "v2", "v3", "v4", "v5"}); + expected = + Arrays.asList( + "+I[1, 1, one, NULL, NULL, NULL, NULL]", + "+I[1, 2, second, NULL, NULL, NULL, NULL]", + "+I[2, 3, three, 30000000000, NULL, NULL, NULL]", + "+I[2, 4, four, NULL, NULL, [102, 111, 117, 114, 46, 98, 105, 110, 46, 108, 111, 110, 103], 4.00000000004]", + "+I[1, 6, six, 60, NULL, NULL, NULL]", + "+I[2, 7, seven, 70000000000, NULL, NULL, NULL]", + "+I[2, 8, very long string, 80000000000, NULL, NULL, NULL]", + "+I[1, 9, nine, 90000000000, 99999.999, [110, 105, 110, 101, 46, 98, 105, 110, 46, 108, 111, 110, 103], 9.00000000009]"); + waitForResult(expected, table, rowType, primaryKeys); + + // test that catalog loader works + assertThat(getFileStoreTable().options()).containsEntry("alter-table-test", "true"); + } + + @Test + @Timeout(60) + public void testSyncSqlServerTable() throws Exception { + Map sqlServerConfig = getBasicSqlServerConfig(); + sqlServerConfig.put("database-name", DATABASE_NAME); + sqlServerConfig.put("table-name", "schema_evolution_\\d+"); + + SqlServerSyncTableAction action = + syncTableActionBuilder(sqlServerConfig) + .withCatalogConfig( + Collections.singletonMap( + CatalogOptions.METASTORE.key(), "test-alter-table")) + .withTableConfig(getBasicTableConfig()) + .withPartitionKeys("pt") + .withPrimaryKeys("pt", "_id") + .build(); + runActionWithDefaultEnv(action); + + checkTableSchema( + "[{\"id\":0,\"name\":\"pt\",\"type\":\"INT NOT NULL\"}," + + "{\"id\":1,\"name\":\"_id\",\"type\":\"INT NOT NULL\"}," + + "{\"id\":2,\"name\":\"v1\",\"type\":\"VARCHAR(10)\"}]"); + + try (Statement statement = getStatement()) { + testSyncSqlServerTableImpl(statement); + } + } + + private void checkTableSchema(String excepted) throws Exception { + FileStoreTable table = getFileStoreTable(); + assertThat(JsonSerdeUtil.toFlatJson(table.schema().fields())).isEqualTo(excepted); + } + + private void testSyncSqlServerTableImpl(Statement statement) throws Exception { + FileStoreTable table = getFileStoreTable(); + statement.executeUpdate("USE " + DATABASE_NAME); + statement.executeUpdate("INSERT INTO schema_evolution_1 VALUES (1, 1, 'one')"); + statement.executeUpdate( + "INSERT INTO schema_evolution_2 VALUES (1, 2, 'two'), (2, 4, 'four')"); + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.INT().notNull(), + DataTypes.INT().notNull(), + DataTypes.VARCHAR(10) + }, + new String[] {"pt", "_id", "v1"}); + List primaryKeys = Arrays.asList("pt", "_id"); + List expected = Arrays.asList("+I[1, 1, one]", "+I[1, 2, two]", "+I[2, 4, four]"); + waitForResult(expected, table, rowType, primaryKeys); + } + + @Test + @Timeout(120) + public void testAllTypes() throws Exception { + // the first round checks for table creation + // the second round checks for running the action on an existing table + for (int i = 0; i < 2; i++) { + testAllTypesOnce(); + } + } + + private void testAllTypesOnce() throws Exception { + Map sqlServerConfig = getBasicSqlServerConfig(); + sqlServerConfig.put("database-name", DATABASE_NAME); + sqlServerConfig.put("table-name", "all_types_table"); + + SqlServerSyncTableAction action = + syncTableActionBuilder(sqlServerConfig) + .withPartitionKeys("pt") + .withPrimaryKeys("pt", "_id") + .build(); + JobClient client = runActionWithDefaultEnv(action); + testAllTypesImpl(); + client.cancel().get(); + } + + private void testAllTypesImpl() throws Exception { + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.INT().notNull(), // _id + DataTypes.DECIMAL(2, 1).notNull(), // pt + DataTypes.BOOLEAN(), // _bit1 + DataTypes.TINYINT(), // _tinyint + DataTypes.SMALLINT(), // _smallint + DataTypes.INT(), // _int + DataTypes.BIGINT(), // _bigint + DataTypes.FLOAT(), // _float + DataTypes.DOUBLE(), // _real + DataTypes.DECIMAL(8, 0), // _numeric + DataTypes.DECIMAL(8, 3), // _numeric8_3 + DataTypes.DECIMAL(8, 0), // _decimal + DataTypes.DECIMAL(38, 10), // _big_decimal + DataTypes.DECIMAL(10, 4), // _smallmoney + DataTypes.DECIMAL(19, 4), // _money + DataTypes.DECIMAL(19, 4), // _big_money + DataTypes.DATE(), // _date + DataTypes.TIME(3), // _time3 + DataTypes.TIME(6), // _time6 + DataTypes.TIME(7), // _time7 + DataTypes.TIMESTAMP(3), // _datetime + DataTypes.TIMESTAMP(0), // _smalldatetime + DataTypes.TIMESTAMP(7), // _datetime2 + DataTypes.TIMESTAMP(3), // _datetime2_3 + DataTypes.TIMESTAMP(6), // _datetime2_6 + DataTypes.TIMESTAMP(7), // _datetime2_7 + DataTypes.CHAR(10), // _char + DataTypes.VARCHAR(20), // _varchar + DataTypes.STRING(), // _text + DataTypes.VARCHAR(10), // _nchar + DataTypes.VARCHAR(20), // _nvarchar + DataTypes.STRING(), // _ntext + DataTypes.STRING(), // _xml + DataTypes.STRING() // _datetimeoffset + }, + new String[] { + "_id", + "pt", + "_bit1", + "_tinyint", + "_smallint", + "_int", + "_bigint", + "_float", + "_real", + "_numeric", + "_numeric8_3", + "_decimal", + "_big_decimal", + "_smallmoney", + "_money", + "_big_money", + "_date", + "_time3", + "_time6", + "_time7", + "_datetime", + "_smalldatetime", + "_datetime2", + "_datetime2_3", + "_datetime2_6", + "_datetime2_7", + "_char", + "_varchar", + "_text", + "_nchar", + "_nvarchar", + "_ntext", + "_xml", + "_datetimeoffset" + }); + FileStoreTable table = getFileStoreTable(); + List expected = + Arrays.asList( + "+I[1, 1.1, true, 1, 1000, 1000000, 10000000000, 3.14159, 3.14, 12345678, 12345.678, 12345678, 12345.6789123456, 12345.6700, 12345678.0000, 12345.6789, 19439, 37815123, 37815123, 37815123, 2023-09-30T10:30:15.123, 2023-03-23T14:30, 2023-09-30T10:30:15.123456700, 2023-09-30T10:30:15.123, 2023-09-30T10:30:15.123456, 2023-09-30T10:30:15.123456700, Paimon, Apache Paimon, Apache Paimon SQLServer TEXT Test Data, Paimon , Apache Paimon, Apache Paimon NTEXT Long Test Data, NULL, 2023-02-01T10:00:00+05:00]", + "+I[2, 2.2, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL]"); + waitForResult(expected, table, rowType, Arrays.asList("pt", "_id")); + } + + @Test + public void testIncompatibleSqlServerTable() { + Map sqlServerConfig = getBasicSqlServerConfig(); + sqlServerConfig.put("database-name", DATABASE_NAME); + sqlServerConfig.put("table-name", "incompatible_field_\\d+"); + SqlServerSyncTableAction action = syncTableActionBuilder(sqlServerConfig).build(); + assertThatThrownBy(action::run) + .satisfies( + anyCauseMatches( + IllegalArgumentException.class, + "Column v1 have different types when merging schemas.")); + } + + @Test + public void testIncompatiblePaimonTable() throws Exception { + Map sqlServerConfig = getBasicSqlServerConfig(); + sqlServerConfig.put("database-name", DATABASE_NAME); + sqlServerConfig.put("table-name", "incompatible_pk_\\d+"); + createFileStoreTable( + RowType.of( + new DataType[] {DataTypes.INT(), DataTypes.BIGINT(), DataTypes.DOUBLE()}, + new String[] {"a", "b", "c"}), + Collections.emptyList(), + Collections.singletonList("a"), + new HashMap<>()); + + SqlServerSyncTableAction action = + syncTableActionBuilder(sqlServerConfig).withPrimaryKeys("a").build(); + + assertThatThrownBy(action::run) + .satisfies( + anyCauseMatches( + IllegalArgumentException.class, + "Paimon schema and source table schema are not compatible.")); + } + + @Test + public void testInvalidPrimaryKey() throws Exception { + Map sqlServerConfig = getBasicSqlServerConfig(); + sqlServerConfig.put("database-name", DATABASE_NAME); + sqlServerConfig.put("table-name", "schema_evolution_\\d+"); + + SqlServerSyncTableAction action = + syncTableActionBuilder(sqlServerConfig).withPrimaryKeys("pk").build(); + + assertThatThrownBy(action::run) + .satisfies( + anyCauseMatches( + IllegalArgumentException.class, + "Specified primary key 'pk' does not exist in source tables or computed columns [pt, _id, v1].")); + } + + @Test + public void testNoPrimaryKey() { + Map sqlServerConfig = getBasicSqlServerConfig(); + sqlServerConfig.put("database-name", DATABASE_NAME); + sqlServerConfig.put("table-name", "incompatible_pk_\\d+"); + + SqlServerSyncTableAction action = syncTableActionBuilder(sqlServerConfig).build(); + + assertThatThrownBy(action::run) + .satisfies( + anyCauseMatches( + IllegalArgumentException.class, + "Primary keys are not specified. " + + "Also, can't infer primary keys from source table schemas because " + + "source tables have no primary keys or have different primary keys.")); + } + + @Test + @Timeout(120) + public void testComputedColumn() throws Exception { + // the first round checks for table creation + // the second round checks for running the action on an existing table + for (int i = 0; i < 2; i++) { + innerTestComputedColumn(i == 0); + } + } + + private void innerTestComputedColumn(boolean executeSqlServer) throws Exception { + Map sqlServerConfig = getBasicSqlServerConfig(); + sqlServerConfig.put("database-name", DATABASE_NAME); + sqlServerConfig.put("table-name", "test_computed_column"); + + List computedColumnDefs = + Arrays.asList( + "_year_date=year(_date)", + "_year_datetime=year(_datetime)", + "_month_date=month(_date)", + "_month_datetime=month(_datetime)", + "_day_date=day(_date)", + "_day_datetime=day(_datetime)", + "_hour_date=hour(_date)", + "_hour_datetime=hour(_datetime)", + "_date_format_date=date_format(_date,yyyy)", + "_date_format_datetime=date_format(_datetime,yyyy-MM-dd)", + "_substring_date1=substring(_date,2)", + "_truncate_date=trUNcate(pk,2)"); // test case-insensitive too + + SqlServerSyncTableAction action = + syncTableActionBuilder(sqlServerConfig) + .withPartitionKeys("_year_date") + .withPrimaryKeys("pk", "_year_date") + .withComputedColumnArgs(computedColumnDefs) + .build(); + runActionWithDefaultEnv(action); + + if (executeSqlServer) { + try (Statement statement = getStatement()) { + statement.execute("USE " + DATABASE_NAME); + statement.executeUpdate( + "INSERT INTO test_computed_column VALUES (1, '2023-03-23', '2022-01-01 14:30')"); + statement.executeUpdate( + "INSERT INTO test_computed_column VALUES (2, '2023-03-23', null)"); + } + } + + FileStoreTable table = getFileStoreTable(); + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.INT().notNull(), + DataTypes.DATE(), + DataTypes.TIMESTAMP(3), + DataTypes.INT().notNull(), + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.INT().notNull() + }, + new String[] { + "pk", + "_date", + "_datetime", + "_year_date", + "_year_datetime", + "_month_date", + "_month_datetime", + "_day_date", + "_day_datetime", + "_hour_date", + "_hour_datetime", + "_date_format_date", + "_date_format_datetime", + "_substring_date1", + "_truncate_date" + }); + List expected = + Arrays.asList( + "+I[1, 19439, 2022-01-01T14:30, 2023, 2022, 3, 1, 23, 1, 0, 14, 2023, 2022-01-01, 23-03-23, 0]", + "+I[2, 19439, NULL, 2023, NULL, 3, NULL, 23, NULL, 0, NULL, 2023, NULL, 23-03-23, 2]"); + waitForResult(expected, table, rowType, Arrays.asList("pk", "_year_date")); + } + + @Test + @Timeout(60) + public void testSyncShards() throws Exception { + Map postgresConfig = getBasicSqlServerConfig(); + + // test table list + ThreadLocalRandom random = ThreadLocalRandom.current(); + String schemaPattern = random.nextBoolean() ? "shard_.+" : "shard_1|shard_2"; + String tblPattern = random.nextBoolean() ? "t.+" : "t1|t2"; + postgresConfig.put(PostgresSourceOptions.DATABASE_NAME.key(), "shard_schema"); + postgresConfig.put(PostgresSourceOptions.SCHEMA_NAME.key(), schemaPattern); + postgresConfig.put(PostgresSourceOptions.TABLE_NAME.key(), tblPattern); + + SqlServerSyncTableAction action = + syncTableActionBuilder(postgresConfig) + .withPartitionKeys("pt") + .withPrimaryKeys("pk", "pt") + .withComputedColumnArgs("pt=substring(_date,5)") + .build(); + runActionWithDefaultEnv(action); + + try (Statement statement = getStatement()) { + statement.execute("USE shard_schema"); + statement.executeUpdate("INSERT INTO shard_1.t1 VALUES (1, '2023-07-30')"); + statement.executeUpdate("INSERT INTO shard_1.t2 VALUES (2, '2023-07-30')"); + statement.executeUpdate("INSERT INTO shard_2.t1 VALUES (3, '2023-07-31')"); + statement.executeUpdate("INSERT INTO shard_2.t2 VALUES (4, '2023-07-31')"); + } + + FileStoreTable table = getFileStoreTable(); + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.INT().notNull(), + DataTypes.VARCHAR(10), + DataTypes.STRING().notNull() + }, + new String[] {"pk", "_date", "pt"}); + waitForResult( + Arrays.asList( + "+I[1, 2023-07-30, 07-30]", + "+I[2, 2023-07-30, 07-30]", + "+I[3, 2023-07-31, 07-31]", + "+I[4, 2023-07-31, 07-31]"), + table, + rowType, + Arrays.asList("pk", "pt")); + } + + @Test + @Timeout(60) + public void testOptionsChange() throws Exception { + Map sqlServerConfig = getBasicSqlServerConfig(); + + sqlServerConfig.put(SqlServerSourceOptions.DATABASE_NAME.key(), DATABASE_NAME); + sqlServerConfig.put(SqlServerSourceOptions.TABLE_NAME.key(), "test_options_change"); + Map tableConfig = new HashMap<>(); + tableConfig.put("bucket", "1"); + tableConfig.put("sink.parallelism", "1"); + + SqlServerSyncTableAction action1 = + syncTableActionBuilder(sqlServerConfig) + .withPartitionKeys("pt") + .withPrimaryKeys("pk", "pt") + .withComputedColumnArgs("pt=substring(_date,5)") + .withTableConfig(tableConfig) + .build(); + JobClient jobClient = runActionWithDefaultEnv(action1); + try (Statement statement = getStatement()) { + statement.execute("USE " + DATABASE_NAME); + statement.executeUpdate( + "INSERT INTO test_options_change VALUES (1, '2023-03-23', '2022-01-01 14:30')"); + statement.executeUpdate( + "INSERT INTO test_options_change VALUES (2, '2023-03-23', null)"); + } + waitingTables(tableName); + jobClient.cancel(); + + tableConfig.put("sink.savepoint.auto-tag", "true"); + tableConfig.put("tag.num-retained-max", "5"); + tableConfig.put("tag.automatic-creation", "process-time"); + tableConfig.put("tag.creation-period", "hourly"); + tableConfig.put("tag.creation-delay", "600000"); + tableConfig.put("snapshot.time-retained", "1h"); + tableConfig.put("snapshot.num-retained.min", "5"); + tableConfig.put("snapshot.num-retained.max", "10"); + tableConfig.put("changelog-producer", "input"); + + SqlServerSyncTableAction action2 = + syncTableActionBuilder(sqlServerConfig) + .withPartitionKeys("pt") + .withPrimaryKeys("pk", "pt") + .withComputedColumnArgs("pt=substring(_date,5)") + .withTableConfig(tableConfig) + .build(); + runActionWithDefaultEnv(action2); + + Map dynamicOptions = getFileStoreTable().options(); + assertThat(dynamicOptions).containsAllEntriesOf(tableConfig); + } + + @Test + @Timeout(120) + public void testMetadataColumns() throws Exception { + try (Statement statement = getStatement()) { + statement.execute("USE table_metadata"); + statement.executeUpdate("INSERT INTO test_metadata_columns VALUES (1, '2023-07-30')"); + statement.executeUpdate("INSERT INTO test_metadata_columns VALUES (2, '2023-07-30')"); + } + + Map sqlServerConfig = getBasicSqlServerConfig(); + sqlServerConfig.put(SqlServerSourceOptions.DATABASE_NAME.key(), "table_metadata"); + sqlServerConfig.put(SqlServerSourceOptions.TABLE_NAME.key(), "test_metadata_columns"); + + SqlServerSyncTableAction action = + syncTableActionBuilder(sqlServerConfig) + .withPrimaryKeys("pk") + .withMetadataColumns("table_name", "schema_name", "database_name", "op_ts") + .build(); + + runActionWithDefaultEnv(action); + + FileStoreTable table = getFileStoreTable(); + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.INT().notNull(), + DataTypes.VARCHAR(10), + DataTypes.STRING().notNull(), + DataTypes.STRING().notNull(), + DataTypes.STRING().notNull(), + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull() + }, + new String[] { + "pk", "_date", "table_name", "schema_name", "database_name", "op_ts" + }); + + waitForResult( + Arrays.asList( + "+I[1, 2023-07-30, test_metadata_columns, dbo, table_metadata, 1970-01-01T00:00]", + "+I[2, 2023-07-30, test_metadata_columns, dbo, table_metadata, 1970-01-01T00:00]"), + table, + rowType, + Collections.singletonList("pk")); + } + + @Test + public void testCatalogAndTableConfig() { + SqlServerSyncTableAction action = + syncTableActionBuilder(getBasicSqlServerConfig()) + .withCatalogConfig(Collections.singletonMap("catalog-key", "catalog-value")) + .withTableConfig(Collections.singletonMap("table-key", "table-value")) + .build(); + assertThat(action.catalogConfig()).containsEntry("catalog-key", "catalog-value"); + assertThat(action.tableConfig()) + .containsExactlyEntriesOf(Collections.singletonMap("table-key", "table-value")); + } + + private FileStoreTable getFileStoreTable() throws Exception { + return getFileStoreTable(tableName); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/META-INF/licenses/container-license-acceptance.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/META-INF/licenses/container-license-acceptance.txt new file mode 100644 index 000000000000..91423a10bc05 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/META-INF/licenses/container-license-acceptance.txt @@ -0,0 +1,16 @@ +-- Licensed to the Apache Software Foundation (ASF) under one +-- or more contributor license agreements. See the NOTICE file +-- distributed with this work for additional information +-- regarding copyright ownership. The ASF licenses this file +-- to you under the Apache License, Version 2.0 (the +-- "License"); you may not use this file except in compliance +-- with the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +mcr.microsoft.com/mssql/server:2019-latest \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/sqlserver/sync_database_setup.sql b/paimon-flink/paimon-flink-cdc/src/test/resources/sqlserver/sync_database_setup.sql new file mode 100644 index 000000000000..fb0440a14205 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/sqlserver/sync_database_setup.sql @@ -0,0 +1,326 @@ +-- Licensed to the Apache Software Foundation (ASF) under one +-- or more contributor license agreements. See the NOTICE file +-- distributed with this work for additional information +-- regarding copyright ownership. The ASF licenses this file +-- to you under the Apache License, Version 2.0 (the +-- "License"); you may not use this file except in compliance +-- with the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. + +-- ################################################################################ +-- SqlServerSyncDatabaseActionITCase +-- ################################################################################ + +CREATE DATABASE paimon_sync_database; +USE paimon_sync_database; +EXEC sys.sp_cdc_enable_db; + +CREATE TABLE t1 ( + k INT, + v1 VARCHAR(10), + PRIMARY KEY (k) +); + +CREATE TABLE t2 ( + k1 INT, + k2 VARCHAR(10), + v1 INT, + v2 BIGINT, + PRIMARY KEY (k1, k2) +); + +-- no primary key, should be ignored +CREATE TABLE t3 ( + v1 INT +); +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 't1', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 't2', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 't3', @role_name = NULL, @supports_net_changes = 0; + + +-- ################################################################################ +-- SqlServerSyncDatabaseActionITCase#testIgnoreIncompatibleTables +-- ################################################################################ + +CREATE DATABASE paimon_sync_database_ignore_incompatible; +USE paimon_sync_database_ignore_incompatible; +EXEC sys.sp_cdc_enable_db; + +CREATE TABLE incompatible ( + k INT, + v1 VARCHAR(10), + PRIMARY KEY (k) +); + +CREATE TABLE compatible ( + k1 INT, + k2 VARCHAR(10), + v1 INT, + v2 BIGINT, + PRIMARY KEY (k1, k2) +); + +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'incompatible', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'compatible', @role_name = NULL, @supports_net_changes = 0; + +-- ################################################################################ +-- SqlServerSyncDatabaseActionITCase#testTableAffix +-- ################################################################################ + +CREATE DATABASE paimon_sync_database_affix; +USE paimon_sync_database_affix; +EXEC sys.sp_cdc_enable_db; + +CREATE TABLE t1 ( + k1 INT, + v0 VARCHAR(10), + PRIMARY KEY (k1) +); + +CREATE TABLE t2 ( + k2 INT, + v0 VARCHAR(10), + PRIMARY KEY (k2) +); + +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 't1', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 't2', @role_name = NULL, @supports_net_changes = 0; + +-- ################################################################################ +-- SqlServerSyncDatabaseActionITCase#testIncludingTables +-- ################################################################################ + +CREATE DATABASE paimon_sync_database_including; +USE paimon_sync_database_including; +EXEC sys.sp_cdc_enable_db; + +CREATE TABLE paimon_1 ( + k INT, + PRIMARY KEY (k) +); + +CREATE TABLE paimon_2 ( + k INT, + PRIMARY KEY (k) +); + +CREATE TABLE flink ( + k INT, + PRIMARY KEY (k) +); + +CREATE TABLE ignored ( + k INT, + PRIMARY KEY (k) +); + +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'paimon_1', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'paimon_2', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'flink', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'ignored', @role_name = NULL, @supports_net_changes = 0; + + +-- ################################################################################ +-- SqlServerSyncDatabaseActionITCase#testExcludingTables +-- ################################################################################ + +CREATE DATABASE paimon_sync_database_excluding; +USE paimon_sync_database_excluding; +EXEC sys.sp_cdc_enable_db; + +CREATE TABLE paimon_1 ( + k INT, + PRIMARY KEY (k) +); + +CREATE TABLE paimon_2 ( + k INT, + PRIMARY KEY (k) +); + +CREATE TABLE flink ( + k INT, + PRIMARY KEY (k) +); + +CREATE TABLE sync ( + k INT, + PRIMARY KEY (k) +); +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'paimon_1', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'paimon_2', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'flink', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'sync', @role_name = NULL, @supports_net_changes = 0; + +-- ################################################################################ +-- SqlServerSyncDatabaseActionITCase#testIncludingAndExcludingTables +-- ################################################################################ + +CREATE DATABASE paimon_sync_database_in_excluding; +USE paimon_sync_database_in_excluding; +EXEC sys.sp_cdc_enable_db; + +CREATE TABLE paimon_1 ( + k INT, + PRIMARY KEY (k) +); + +CREATE TABLE paimon_2 ( + k INT, + PRIMARY KEY (k) +); + +CREATE TABLE flink ( + k INT, + PRIMARY KEY (k) +); + +CREATE TABLE test ( + k INT, + PRIMARY KEY (k) +); +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'paimon_1', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'paimon_2', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'flink', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'test', @role_name = NULL, @supports_net_changes = 0; + +-- ################################################################################ +-- SqlServerSyncDatabaseActionITCase#testIgnoreCase +-- ################################################################################ + +CREATE DATABASE paimon_ignore_CASE; +USE paimon_ignore_CASE; +EXEC sys.sp_cdc_enable_db; + +CREATE TABLE T ( + k INT, + UPPERCASE_V0 VARCHAR(20), + PRIMARY KEY (k) +); +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'T', @role_name = NULL, @supports_net_changes = 0; + + +-- ################################################################################ +-- testSyncMultipleShards +-- ################################################################################ + +CREATE DATABASE database_shard; +USE database_shard; +EXEC sys.sp_cdc_enable_db; + +create schema schema_1; +CREATE TABLE schema_1.t1 ( + k INT, + v1 VARCHAR(10), + PRIMARY KEY (k) +); + +CREATE TABLE schema_1.t2 ( + k BIGINT, + v1 FLOAT, + PRIMARY KEY (k) +); + +CREATE TABLE schema_1.t3 ( + k INT, + v1 VARCHAR(10), + PRIMARY KEY (k) +); + +EXEC sys.sp_cdc_enable_table @source_schema = 'schema_1', @source_name = 't1', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'schema_1', @source_name = 't2', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'schema_1', @source_name = 't3', @role_name = NULL, @supports_net_changes = 0; + +create schema schema_2; +-- test schema merging +CREATE TABLE schema_2.t1 ( + k INT, + v1 VARCHAR(20), + v2 BIGINT, + PRIMARY KEY (k) +); + +-- test schema evolution +CREATE TABLE schema_2.t2 ( + k BIGINT, + v1 FLOAT, + PRIMARY KEY (k) +); + +-- test some shard doesn't have primary key +CREATE TABLE schema_2.t3 ( + k INT, + v1 VARCHAR(10) +); +EXEC sys.sp_cdc_enable_table @source_schema = 'schema_2', @source_name = 't1', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'schema_2', @source_name = 't2', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'schema_2', @source_name = 't3', @role_name = NULL, @supports_net_changes = 0; + +-- ################################################################################ +-- testSyncMultipleShardsWithoutMerging +-- ################################################################################ + +CREATE DATABASE without_merging_shard; +USE without_merging_shard; +EXEC sys.sp_cdc_enable_db; + +CREATE SCHEMA schema_1; +CREATE TABLE schema_1.t1 ( + k INT, + v1 VARCHAR(10), + PRIMARY KEY (k) +); + +CREATE TABLE schema_1.t2 ( + k INT, + v1 VARCHAR(10), + PRIMARY KEY (k) +); + +EXEC sys.sp_cdc_enable_table @source_schema = 'schema_1', @source_name = 't1', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'schema_1', @source_name = 't2', @role_name = NULL, @supports_net_changes = 0; + +CREATE SCHEMA schema_2; +CREATE TABLE schema_2.t1 ( + k INT, + v1 VARCHAR(20), + v2 BIGINT, + PRIMARY KEY (k) +); + +-- test some shard doesn't have primary key +CREATE TABLE schema_2.t2 ( + k INT, + v1 VARCHAR(10) +); +EXEC sys.sp_cdc_enable_table @source_schema = 'schema_2', @source_name = 't1', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'schema_2', @source_name = 't2', @role_name = NULL, @supports_net_changes = 0; + +-- ################################################################################ +-- testMetadataColumns +-- ################################################################################ + +CREATE DATABASE metadata; +USE metadata; +EXEC sys.sp_cdc_enable_db; + +CREATE TABLE t1 ( + k INT, + v1 VARCHAR(10), + PRIMARY KEY (k) +); + +CREATE TABLE t2 ( + k INT, + v1 VARCHAR(10), + PRIMARY KEY (k) +); +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 't1', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 't2', @role_name = NULL, @supports_net_changes = 0; diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/sqlserver/sync_table_setup.sql b/paimon-flink/paimon-flink-cdc/src/test/resources/sqlserver/sync_table_setup.sql new file mode 100644 index 000000000000..40dac2f5632e --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/sqlserver/sync_table_setup.sql @@ -0,0 +1,258 @@ +-- 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. +-- +-- ################################################################################ +-- SqlServerSyncTableActionITCase +-- ################################################################################ + +CREATE DATABASE paimon_sync_table; +USE paimon_sync_table; +EXEC sys.sp_cdc_enable_db; + +CREATE TABLE schema_evolution_1 ( + pt INT, + _id INT, + v1 VARCHAR(10), + PRIMARY KEY (_id) +); +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'schema_evolution_1', @role_name = NULL, @supports_net_changes = 0; + + +CREATE TABLE schema_evolution_2 ( + pt INT, + _id INT, + v1 VARCHAR(10), + PRIMARY KEY (_id) +); +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'schema_evolution_2', @role_name = NULL, @supports_net_changes = 0; + + +CREATE TABLE all_types_table ( + _id INT, + pt DECIMAL(2, 1), + -- BIT + _bit1 BIT, + -- TINYINT + _tinyint TINYINT, + -- SMALLINT + _smallint SMALLINT, + -- INT + _int INT, + -- BIGINT + _bigint BIGINT, + -- FLOAT + _float FLOAT, + -- REAL + _real REAL, + -- NUMERIC + _numeric NUMERIC(8), + _numeric8_3 NUMERIC(8, 3), + -- DECIMAL + _decimal DECIMAL(8), + _big_decimal DECIMAL(38,10), + -- SMALLMONEY + _smallmoney SMALLMONEY, + -- MONEY + _money MONEY, + _big_money MONEY, + -- DATE + _date DATE, + -- TIME + _time3 TIME(3), + _time6 TIME(6), + _time7 TIME(7), + -- DATETIME + _datetime DATETIME, + -- SMALLDATETIME + _smalldatetime SMALLDATETIME, + -- DATETIME2 + _datetime2 DATETIME2, + _datetime2_3 DATETIME2(3), + _datetime2_6 DATETIME2(6), + _datetime2_7 DATETIME2(7), + -- string + _char CHAR(10), + _varchar VARCHAR(20), + _text TEXT, + _nchar NCHAR(10), + _nvarchar NVARCHAR(20), + _ntext NTEXT, + _xml XML, + _datetimeoffset DATETIMEOFFSET(7) + PRIMARY KEY (_id) +); + + +INSERT INTO all_types_table VALUES ( + 1, + 1.1, + -- BIT + 255, + -- TINYINT + 1, + -- SMALLINT + 1000, + -- INT + 1000000, + -- BIGINT + 10000000000, + -- FLOAT + 3.14159, + -- REAL + 3.14, + -- NUMERIC + 12345678, 12345.678, + -- DECIMAL + 12345678, 12345.6789123456, + -- SMALLMONEY + 12345.6700, + -- MONEY + 12345678.0000, 12345.6789, + -- DATE + '2023-03-23', + -- TIME + '10:30:15.123','10:30:15.123456','10:30:15.1234567', + -- DATETIME + '2023-09-30 10:30:15.123', + -- SMALLDATETIME + '2023-03-23 14:30:05', + -- DATETIME2 + '2023-09-30 10:30:15.1234567', '2023-09-30 10:30:15.123','2023-09-30 10:30:15.123456', '2023-09-30 10:30:15.1234567', + -- string + 'Paimon','Apache Paimon','Apache Paimon SQLServer TEXT Test Data','Paimon','Apache Paimon','Apache Paimon NTEXT Long Test Data', + -- XML + NULL, + -- DATETIMEOFFSET + '2023-02-01T10:00:00+05:00' +), ( + 2,2.2, + NULL, NULL, NULL, + NULL, NULL, NULL, + NULL, NULL, NULL, + NULL, NULL, NULL, + NULL, NULL, NULL, + NULL, NULL, NULL, + NULL, NULL, NULL, + NULL, NULL, NULL, + NULL, NULL, NULL, + NULL, NULL, NULL, + NULL, NULL +); +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'all_types_table', @role_name = NULL, @supports_net_changes = 0; + +CREATE TABLE incompatible_field_1 ( + _id INT, + v1 SMALLDATETIME, + PRIMARY KEY (_id) +); +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'incompatible_field_1', @role_name = NULL, @supports_net_changes = 0; + + +CREATE TABLE incompatible_field_2 ( + _id INT, + v1 INT, + PRIMARY KEY (_id) +); +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'incompatible_field_2', @role_name = NULL, @supports_net_changes = 0; + + +CREATE TABLE incompatible_pk_1 ( + a INT, + b BIGINT, + c VARCHAR(20), + PRIMARY KEY (a, b) +); +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'incompatible_pk_1', @role_name = NULL, @supports_net_changes = 0; + + +CREATE TABLE incompatible_pk_2 ( + a INT, + b BIGINT, + c VARCHAR(20), + PRIMARY KEY (a) +); +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'incompatible_pk_2', @role_name = NULL, @supports_net_changes = 0; + +CREATE TABLE test_computed_column ( + pk INT, + _date DATE, + _datetime DATETIME, + PRIMARY KEY (pk) +); +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'test_computed_column', @role_name = NULL, @supports_net_changes = 0; + +CREATE TABLE test_options_change ( + pk INT, + _date DATE, + _datetime DATETIME, + PRIMARY KEY (pk) +); +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'test_options_change', @role_name = NULL, @supports_net_changes = 0; + +-- ################################################################################ +-- testSyncShardsMultipleSchemas +-- ################################################################################ + +CREATE DATABASE shard_schema; +USE shard_schema; +EXEC sys.sp_cdc_enable_db; + +CREATE SCHEMA shard_1; +CREATE TABLE shard_1.t1 ( + pk INT, + _date VARCHAR(10), + PRIMARY KEY (pk) +); + +CREATE TABLE shard_1.t2 ( + pk INT, + _date VARCHAR(10), + PRIMARY KEY (pk) +); + +CREATE SCHEMA shard_2; +CREATE TABLE shard_2.t1 ( + pk INT, + _date VARCHAR(10), + PRIMARY KEY (pk) +); + +CREATE TABLE shard_2.t2 ( + pk INT, + _date VARCHAR(10), + PRIMARY KEY (pk) +); + +EXEC sys.sp_cdc_enable_table @source_schema = 'shard_1', @source_name = 't1', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'shard_1', @source_name = 't2', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'shard_2', @source_name = 't1', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'shard_2', @source_name = 't2', @role_name = NULL, @supports_net_changes = 0; + +-- ################################################################################ +-- testMetadataColumns +-- ################################################################################ + +CREATE DATABASE table_metadata; +USE table_metadata; +EXEC sys.sp_cdc_enable_db; + +CREATE TABLE test_metadata_columns ( + pk INT, + _date VARCHAR(10), + PRIMARY KEY (pk) +); + +EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'test_metadata_columns', @role_name = NULL, @supports_net_changes = 0; \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/sqlserver/tablelist_test_setup.sql b/paimon-flink/paimon-flink-cdc/src/test/resources/sqlserver/tablelist_test_setup.sql new file mode 100644 index 000000000000..dca76b392c3a --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/sqlserver/tablelist_test_setup.sql @@ -0,0 +1,75 @@ +-- Licensed to the Apache Software Foundation (ASF) under one +-- or more contributor license agreements. See the NOTICE file +-- distributed with this work for additional information +-- regarding copyright ownership. The ASF licenses this file +-- to you under the Apache License, Version 2.0 (the +-- "License"); you may not use this file except in compliance +-- with the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- +-- ################################################################################ +-- SqlServerSyncDatabaseTableListITCase +-- ################################################################################ + +-- captured databases +CREATE DATABASE shard_database; + +-- create tables + +USE shard_database; +EXEC sys.sp_cdc_enable_db; + +CREATE SCHEMA shard_1; + +CREATE TABLE shard_1.t1 (k INT, name VARCHAR(100)); -- ignored because of pk absence +CREATE TABLE shard_1.t11 (k INT, name VARCHAR(100), PRIMARY KEY (k)); -- captured +CREATE TABLE shard_1.t2 (k INT, name VARCHAR(100), PRIMARY KEY (k)); -- captured +CREATE TABLE shard_1.t3 (k INT, name VARCHAR(100), PRIMARY KEY (k)); -- captured +CREATE TABLE shard_1.ta (k INT, name VARCHAR(100), PRIMARY KEY (k)); -- ignored +CREATE TABLE shard_1.taa (k INT, name VARCHAR(100), PRIMARY KEY (k)); -- captured +CREATE TABLE shard_1.s1 (k INT, name VARCHAR(100)); -- ignored because of pk absence +CREATE TABLE shard_1.s2 (k INT, name VARCHAR(100), PRIMARY KEY (k)); -- captured +CREATE TABLE shard_1.sa (k INT, name VARCHAR(100), PRIMARY KEY (k)); -- ignored +CREATE TABLE shard_1.m (k INT, name VARCHAR(100), PRIMARY KEY (k)); -- ignored +EXEC sys.sp_cdc_enable_table @source_schema = 'shard_1', @source_name = 't1', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'shard_1', @source_name = 't11', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'shard_1', @source_name = 't2', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'shard_1', @source_name = 't3', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'shard_1', @source_name = 'ta', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'shard_1', @source_name = 'taa', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'shard_1', @source_name = 's1', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'shard_1', @source_name = 's2', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'shard_1', @source_name = 'sa', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'shard_1', @source_name = 'm', @role_name = NULL, @supports_net_changes = 0; + + +CREATE SCHEMA shard_2; +CREATE TABLE shard_2.t1 (k INT, name VARCHAR(100), PRIMARY KEY (k)); -- captured +CREATE TABLE shard_2.t2 (k INT, name VARCHAR(100)); -- ignored because of pk absence +CREATE TABLE shard_2.t22 (k INT, name VARCHAR(100), PRIMARY KEY (k)); -- captured +CREATE TABLE shard_2.t3 (k INT, name VARCHAR(100), PRIMARY KEY (k)); -- captured +CREATE TABLE shard_2.tb (k INT, name VARCHAR(100), PRIMARY KEY (k)); -- captured + +EXEC sys.sp_cdc_enable_table @source_schema = 'shard_2', @source_name = 't1', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'shard_2', @source_name = 't2', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'shard_2', @source_name = 't22', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'shard_2', @source_name = 't3', @role_name = NULL, @supports_net_changes = 0; +EXEC sys.sp_cdc_enable_table @source_schema = 'shard_2', @source_name = 'tb', @role_name = NULL, @supports_net_changes = 0; + +CREATE SCHEMA x_shard_1; + +CREATE TABLE x_shard_1.t1 (k INT, name VARCHAR(100), PRIMARY KEY (k)); -- captured +EXEC sys.sp_cdc_enable_table @source_schema = 'x_shard_1', @source_name = 't1', @role_name = NULL, @supports_net_changes = 0; + + +CREATE SCHEMA ignored; + +CREATE TABLE ignored.t1 (k INT, name VARCHAR(100), PRIMARY KEY (k)); -- ignored +EXEC sys.sp_cdc_enable_table @source_schema = 'ignored', @source_name = 't1', @role_name = NULL, @supports_net_changes = 0;