Skip to content

Commit

Permalink
Supports SQLServer cdc
Browse files Browse the repository at this point in the history
  • Loading branch information
sunxiaojian committed Apr 15, 2024
1 parent ffb9032 commit 8fa0754
Show file tree
Hide file tree
Showing 32 changed files with 4,237 additions and 11 deletions.
2 changes: 2 additions & 0 deletions docs/content/flink/cdc-ingestion/overview.md
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down
211 changes: 211 additions & 0 deletions docs/content/flink/cdc-ingestion/sqlserver-cdc.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,211 @@
---
title: "SQLServer CDC"
weight: 2
type: docs
aliases:
- /cdc-ingestion/sqlserver-cdc.html
---
<!--
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.
-->

# 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
<FLINK_HOME>/bin/flink run \
/path/to/paimon-flink-action-{{< version >}}.jar \
sqlserver-sync-table
--warehouse <warehouse-path> \
--database <database-name> \
--table <table-name> \
[--partition-keys <partition-keys>] \
[--primary-keys <primary-keys>] \
[--type-mapping <option1,option2...>] \
[--computed-column <'column-name=expr-name(args[, ...])'> [--computed-column ...]] \
[--metadata-column <metadata-column>] \
[--sqlserver-conf <sqlserver-cdc-source-conf> [--sqlserver-conf <sqlserver-cdc-source-conf> ...]] \
[--catalog-conf <paimon-catalog-conf> [--catalog-conf <paimon-catalog-conf> ...]] \
[--table-conf <paimon-table-sink-conf> [--table-conf <paimon-table-sink-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
<FLINK_HOME>/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
<FLINK_HOME>/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
<FLINK_HOME>/bin/flink run \
/path/to/paimon-flink-action-{{< version >}}.jar \
sqlserver-sync-database
--warehouse <warehouse-path> \
--database <database-name> \
[--ignore-incompatible <true/false>] \
[--merge-shards <true/false>] \
[--table-prefix <paimon-table-prefix>] \
[--table-suffix <paimon-table-suffix>] \
[--including-tables <sqlserver-table-name|name-regular-expr>] \
[--excluding-tables <sqlserver-table-name|name-regular-expr>] \
[--mode <sync-mode>] \
[--metadata-column <metadata-column>] \
[--type-mapping <option1,option2...>] \
[--sqlserver-conf <sqlserver-cdc-source-conf> [--sqlserver-conf <sqlserver-cdc-source-conf> ...]] \
[--catalog-conf <paimon-catalog-conf> [--catalog-conf <paimon-catalog-conf> ...]] \
[--table-conf <paimon-table-sink-conf> [--table-conf <paimon-table-sink-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
<FLINK_HOME>/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
<FLINK_HOME>/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).
93 changes: 93 additions & 0 deletions docs/layouts/shortcodes/generated/sqlserver_sync_database.html
Original file line number Diff line number Diff line change
@@ -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" }}
<table class="configuration table table-bordered">
<thead>
<tr>
<th class="text-left" style="width: 15%">Configuration</th>
<th class="text-left" style="width: 85%">Description</th>
</tr>
</thead>
<tbody>
<tr>
<td><h5>--warehouse</h5></td>
<td>The path to Paimon warehouse.</td>
</tr>
<tr>
<td><h5>--database</h5></td>
<td>The database name in Paimon catalog.</td>
</tr>
<tr>
<td><h5>--ignore-incompatible</h5></td>
<td>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.</td>
</tr>
<tr>
<td><h5>--merge-shards</h5></td>
<td>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.</td>
</tr>
<tr>
<td><h5>--table-prefix</h5></td>
<td>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_".</td>
</tr>
<tr>
<td><h5>--table-suffix</h5></td>
<td>The suffix of all Paimon tables to be synchronized. The usage is same as "--table-prefix".</td>
</tr>
<tr>
<td><h5>--including-tables</h5></td>
<td>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'.</td>
</tr>
<tr>
<td><h5>--excluding-tables</h5></td>
<td>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.</td>
</tr>
<tr>
<td><h5>--mode</h5></td>
<td>It is used to specify synchronization mode.<br />Possible values:<ul><li>"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.</li><li>"combined": start a single combined sink for all tables, the new table will be automatically synchronized.</li></ul></td>
</tr>
<tr>
<td><h5>--metadata-column</h5></td>
<td>--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 <a href="https://ververica.github.io/flink-cdc-connectors/master/content/connectors/sqlserver-cdc.html#available-metadata">document</a> for a complete list of available metadata.</td>
</tr>
<tr>
<td><h5>--type-mapping</h5></td>
<td>It is used to specify how to map SQLServer data type to Paimon type.<br />
Supported options:
<ul>
<li>"tinyint1-not-bool": maps SQLServer TINYINT(1) to TINYINT instead of BOOLEAN.</li>
<li>"to-nullable": ignores all NOT NULL constraints (except for primary keys).</li>
<li>"to-string": maps all SQLServer types to STRING.</li>
<li>"char-to-string": maps SQLServer CHAR(length)/VARCHAR(length)/NCHAR(length)/NVARCHAR(length) types to STRING.</li>
</ul>
</td>
</tr>
<tr>
<td><h5>--sqlserver-conf</h5></td>
<td>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 <a href="https://ververica.github.io/flink-cdc-connectors/master/content/connectors/sqlserver-cdc.html#connector-options">document</a> for a complete list of configurations.</td>
</tr>
<tr>
<td><h5>--catalog-conf</h5></td>
<td>The configuration for Paimon catalog. Each configuration should be specified in the format "key=value". See <a href="{{ $ref }}">here</a> for a complete list of catalog configurations.</td>
</tr>
<tr>
<td><h5>--table-conf</h5></td>
<td>The configuration for Paimon table sink. Each configuration should be specified in the format "key=value". See <a href="{{ $ref }}">here</a> for a complete list of table configurations.</td>
</tr>
</tbody>
</table>
81 changes: 81 additions & 0 deletions docs/layouts/shortcodes/generated/sqlserver_sync_table.html
Original file line number Diff line number Diff line change
@@ -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" }}
<table class="configuration table table-bordered">
<thead>
<tr>
<th class="text-left" style="width: 15%">Configuration</th>
<th class="text-left" style="width: 85%">Description</th>
</tr>
</thead>
<tbody>
<tr>
<td><h5>--warehouse</h5></td>
<td>The path to Paimon warehouse.</td>
</tr>
<tr>
<td><h5>--database</h5></td>
<td>The database name in Paimon catalog.</td>
</tr>
<tr>
<td><h5>--table</h5></td>
<td>The Paimon table name.</td>
</tr>
<tr>
<td><h5>--partition-keys</h5></td>
<td>The partition keys for Paimon table. If there are multiple partition keys, connect them with comma, for example "dt,hh,mm".</td>
</tr>
<tr>
<td><h5>--primary-keys</h5></td>
<td>The primary keys for Paimon table. If there are multiple primary keys, connect them with comma, for example "buyer_id,seller_id".</td>
</tr>
<tr>
<td><h5>--type-mapping</h5></td>
<td>It is used to specify how to map SQLServer data type to Paimon type.<br />
Supported options:
<ul>
<li>"tinyint1-not-bool": maps SQLServer TINYINT(1) to TINYINT instead of BOOLEAN.</li>
<li>"to-nullable": ignores all NOT NULL constraints (except for primary keys).</li>
<li>"to-string": maps all SQLServer types to STRING.</li>
<li>"char-to-string": maps SQLServer CHAR(length)/VARCHAR(length)/NCHAR(length)/NVARCHAR(length) types to STRING.</li>
</ul>
</td>
</tr>
<tr>
<td><h5>--computed-column</h5></td>
<td>The definitions of computed columns. The argument field is from SQLServer table field name. See <a href="../overview/#computed-functions">here</a> for a complete list of configurations. </td>
</tr>
<tr>
<td><h5>--metadata-column</h5></td>
<td>--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 <a href="https://ververica.github.io/flink-cdc-connectors/master/content/connectors/sqlserver-cdc.html#available-metadata">document</a> for a complete list of available metadata.</td>
</tr>
<tr>
<td><h5>--sqlserver-conf</h5></td>
<td>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 <a href="https://ververica.github.io/flink-cdc-connectors/master/content/connectors/sqlserver-cdc.html#connector-options">document</a> for a complete list of configurations.</td>
</tr>
<tr>
<td><h5>--catalog-conf</h5></td>
<td>The configuration for Paimon catalog. Each configuration should be specified in the format "key=value". See <a href="{{ $ref }}">here</a> for a complete list of catalog configurations.</td>
</tr>
<tr>
<td><h5>--table-conf</h5></td>
<td>The configuration for Paimon table sink. Each configuration should be specified in the format "key=value". See <a href="{{ $ref }}">here</a> for a complete list of table configurations.</td>
</tr>
</tbody>
</table>
Loading

0 comments on commit 8fa0754

Please sign in to comment.