From 0577cd0a0dc257848525bd6473a0d470e0e662c2 Mon Sep 17 00:00:00 2001 From: gongzhongqiang Date: Fri, 24 May 2024 11:29:32 +0800 Subject: [PATCH] [FLINK-35431][docs] Migrate references in Flink CDC documentation from Debezium 1.9 to 2.0 This closes #3354. --- .dlc.json | 35 ++++++++++++++++++ .github/workflows/build_docs.yml | 36 ++++++++++++++++--- README.md | 2 +- docs/content.zh/_index.md | 2 +- .../docs/connectors/flink-sources/db2-cdc.md | 12 +++---- .../connectors/flink-sources/mysql-cdc.md | 14 ++++---- .../connectors/flink-sources/oracle-cdc.md | 6 ++-- .../docs/connectors/flink-sources/overview.md | 2 +- .../connectors/flink-sources/postgres-cdc.md | 6 ++-- .../connectors/flink-sources/sqlserver-cdc.md | 4 +-- .../connectors/flink-sources/vitess-cdc.md | 2 +- .../connectors/pipeline-connectors/mysql.md | 4 +-- .../contribute-to-flink-cdc.md | 8 ++--- docs/content/_index.md | 2 +- .../docs/connectors/flink-sources/db2-cdc.md | 12 +++---- .../connectors/flink-sources/mysql-cdc.md | 14 ++++---- .../connectors/flink-sources/oracle-cdc.md | 6 ++-- .../docs/connectors/flink-sources/overview.md | 2 +- .../connectors/flink-sources/postgres-cdc.md | 6 ++-- .../connectors/flink-sources/sqlserver-cdc.md | 4 +-- .../connectors/flink-sources/vitess-cdc.md | 2 +- .../connectors/pipeline-connectors/mysql.md | 4 +-- 22 files changed, 124 insertions(+), 61 deletions(-) create mode 100644 .dlc.json diff --git a/.dlc.json b/.dlc.json new file mode 100644 index 000000000..a55469383 --- /dev/null +++ b/.dlc.json @@ -0,0 +1,35 @@ +{ + "ignorePatterns": [ + { + "pattern": "^http://localhost" + }, + { + "pattern": "^#" + }, + { + "pattern": "^{" + }, + { + "pattern": "^https://repo1.maven.org/maven2/org/apache/flink.*SNAPSHOT.*" + }, + { + "pattern": "^https://mvnrepository.com" + }, + { + "pattern": "^https://img.shields.io" + }, + { + "pattern": "^https://twitter.com*" + } + ], + "timeout": "30s", + "retryOn429": true, + "retryCount": 10, + "fallbackRetryDelay": "1000s", + "aliveStatusCodes": [ + 0, + 200, + 401, + 403 + ] +} diff --git a/.github/workflows/build_docs.yml b/.github/workflows/build_docs.yml index f8c2a283f..55dbcd46f 100644 --- a/.github/workflows/build_docs.yml +++ b/.github/workflows/build_docs.yml @@ -16,6 +16,12 @@ name: "Build documentation" on: + pull_request: + branches: + - master + - release-* + paths: + - 'docs/**' push: paths: - 'docs/**' @@ -24,21 +30,38 @@ on: - release-* schedule: - cron: '0 0 * * *' # Deploy every day + workflow_dispatch: concurrency: - group: ${{ github.workflow }}-${{ github.ref }} + group: ${{ github.workflow }}-${{ github.event_name }}-${{ github.event.number || github.run_id }} cancel-in-progress: true jobs: - build-documentation: + check-doc: if: github.repository == 'apache/flink-cdc' runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v3 + - name: Check dead links + uses: gaurav-nelson/github-action-markdown-link-check@1.0.15 + with: + config-file: '.dlc.json' + - name: Build doc + run: | + docker run --rm --volume "$PWD:/root/flink-cdc" chesnay/flink-ci:java_8_11_17_21_maven_386 bash -c "cd /root/flink-cdc && chmod +x ./.github/workflows/docs.sh && ./.github/workflows/docs.sh" + + build-documentation: + if: github.event_name != 'pull_request' && github.repository == 'apache/flink-cdc' + name: "Check and build doc" + runs-on: ubuntu-latest strategy: + fail-fast: false max-parallel: 1 matrix: branch: - master - release-3.0 + - release-3.1 steps: - uses: actions/checkout@v3 @@ -52,11 +75,16 @@ jobs: echo "flink_branch=${currentBranch}" >> ${GITHUB_ENV} if [ "${currentBranch}" = "master" ]; then - echo "flink_alias=release-3.1" >> ${GITHUB_ENV} - elif [ "${currentBranch}" = "release-3.0" ]; then + echo "flink_alias=release-3.2" >> ${GITHUB_ENV} + elif [ "${currentBranch}" = "release-3.1" ]; then echo "flink_alias=stable" >> ${GITHUB_ENV} fi + - name: Check dead links + uses: gaurav-nelson/github-action-markdown-link-check@1.0.15 + with: + config-file: '.dlc.json' + - name: Build documentation run: | docker run --rm --volume "$PWD:/root/flink-cdc" chesnay/flink-ci:java_8_11_17_21_maven_386 bash -c "cd /root/flink-cdc && chmod +x ./.github/workflows/docs.sh && ./.github/workflows/docs.sh" diff --git a/README.md b/README.md index 6b567555d..c4f54a07f 100644 --- a/README.md +++ b/README.md @@ -1,5 +1,5 @@

- Flink CDC + Flink CDC

diff --git a/docs/content.zh/_index.md b/docs/content.zh/_index.md index fc2c11f6f..f439ee43b 100644 --- a/docs/content.zh/_index.md +++ b/docs/content.zh/_index.md @@ -165,7 +165,7 @@ under the License.

Try Flink CDC

Flink CDC provides a series of quick start demos without any dependencies or java code. A Linux or MacOS computer with Docker installed is enough. - Please check out our Quick Start for more information. + Please check out our }}">Quick Start for more information.

diff --git a/docs/content.zh/docs/connectors/flink-sources/db2-cdc.md b/docs/content.zh/docs/connectors/flink-sources/db2-cdc.md index ef06c1b3e..9c9154704 100644 --- a/docs/content.zh/docs/connectors/flink-sources/db2-cdc.md +++ b/docs/content.zh/docs/connectors/flink-sources/db2-cdc.md @@ -32,9 +32,9 @@ describes how to setup the db2 CDC connector to run SQL queries against Db2 data ## Supported Databases -| Connector | Database | Driver | -|-----------------------|----------------------------------------------------|----------------------| -| [Db2-cdc](../db2-cdc) |
  • [Db2](https://www.ibm.com/products/db2): 11.5 | Db2 Driver: 11.5.0.0 | +| Connector | Database | Driver | +|-----------|----------------------------------------------------|----------------------| +| Db2-cdc |
  • [Db2](https://www.ibm.com/products/db2): 11.5 | Db2 Driver: 11.5.0.0 | Dependencies ------------ @@ -56,7 +56,7 @@ put it under `/lib/`. Setup Db2 server ---------------- -Follow the steps in the [Debezium Db2 Connector](https://debezium.io/documentation/reference/1.9/connectors/db2.html#setting-up-db2). +Follow the steps in the [Debezium Db2 Connector](https://debezium.io/documentation/reference/2.0/connectors/db2.html#setting-up-db2). Notes @@ -184,7 +184,7 @@ for more detailed information. String The session time zone in database server, e.g. "Asia/Shanghai". It controls how the TIMESTAMP type in Db2 converted to STRING. - See more here. + See more here. If not set, then ZoneId.systemDefault() is used to determine the server time zone. @@ -196,7 +196,7 @@ for more detailed information. Pass-through Debezium's properties to Debezium Embedded Engine which is used to capture data changes from Db2 server. For example: 'debezium.snapshot.mode' = 'never'. - See more about the Debezium's Db2 Connector properties + See more about the Debezium's Db2 Connector properties diff --git a/docs/content.zh/docs/connectors/flink-sources/mysql-cdc.md b/docs/content.zh/docs/connectors/flink-sources/mysql-cdc.md index e42728730..b1fd48c20 100644 --- a/docs/content.zh/docs/connectors/flink-sources/mysql-cdc.md +++ b/docs/content.zh/docs/connectors/flink-sources/mysql-cdc.md @@ -30,9 +30,9 @@ MySQL CDC 连接器允许从 MySQL 数据库读取快照数据和增量数据。 ## 支持的数据库 -| Connector | Database | Driver | -|----------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|-------------------------| -| mysql-cdc |
  • [MySQL](https://dev.mysql.com/doc): 5.6, 5.7, 8.0.x
  • [RDS MySQL](https://www.aliyun.com/product/rds/mysql): 5.6, 5.7, 8.0.x
  • [PolarDB MySQL](https://www.aliyun.com/product/polardb): 5.6, 5.7, 8.0.x
  • [Aurora MySQL](https://aws.amazon.com/cn/rds/aurora): 5.6, 5.7, 8.0.x
  • [MariaDB](https://mariadb.org): 10.x
  • [PolarDB X](https://github.com/ApsaraDB/galaxysql): 2.0.1 | JDBC Driver: 8.0.27 | +| Connector | Database | Driver | +|-----------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|---------------------| +| mysql-cdc |
  • [MySQL](https://dev.mysql.com/doc): 5.6, 5.7, 8.0.x
  • [RDS MySQL](https://www.aliyun.com/product/rds/mysql): 5.6, 5.7, 8.0.x
  • [PolarDB MySQL](https://www.aliyun.com/product/polardb): 5.6, 5.7, 8.0.x
  • [Aurora MySQL](https://aws.amazon.com/cn/rds/aurora): 5.6, 5.7, 8.0.x
  • [MariaDB](https://mariadb.org): 10.x
  • [PolarDB X](https://github.com/ApsaraDB/galaxysql): 2.0.1 | JDBC Driver: 8.0.27 | 依赖 ------------ @@ -80,7 +80,7 @@ mysql> GRANT SELECT, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT ON *. mysql> FLUSH PRIVILEGES; ``` -查看更多用户权限问题请参考 [权限说明](https://debezium.io/documentation/reference/1.9/connectors/mysql.html#mysql-creating-user). +查看更多用户权限问题请参考 [权限说明](https://debezium.io/documentation/reference/2.0/connectors/mysql.html#mysql-creating-user). 注意事项 @@ -284,7 +284,7 @@ Flink SQL> SELECT * FROM orders; String 数据库服务器中的会话时区, 例如: "Asia/Shanghai". 它控制 MYSQL 中的时间戳类型如何转换为字符串。 - 更多请参考 这里. + 更多请参考 这里. 如果没有设置,则使用ZoneId.systemDefault()来确定服务器时区。 @@ -339,7 +339,7 @@ Flink SQL> SELECT * FROM orders; String 将 Debezium 的属性传递给 Debezium 嵌入式引擎,该引擎用于从 MySQL 服务器捕获数据更改。 For example: 'debezium.snapshot.mode' = 'never'. - 查看更多关于 Debezium 的 MySQL 连接器属性 + 查看更多关于 Debezium 的 MySQL 连接器属性 scan.incremental.close-idle-reader.enabled @@ -738,7 +738,7 @@ $ ./bin/flink run \ - + diff --git a/docs/content.zh/docs/connectors/flink-sources/oracle-cdc.md b/docs/content.zh/docs/connectors/flink-sources/oracle-cdc.md index 9f17cd7af..992b7b440 100644 --- a/docs/content.zh/docs/connectors/flink-sources/oracle-cdc.md +++ b/docs/content.zh/docs/connectors/flink-sources/oracle-cdc.md @@ -206,7 +206,7 @@ Overall, the steps for configuring CDB database is quite similar to non-CDB data exit ``` -See more about the [Setting up Oracle](https://debezium.io/documentation/reference/1.9/connectors/oracle.html#setting-up-oracle) +See more about the [Setting up Oracle](https://debezium.io/documentation/reference/2.0/connectors/oracle.html#setting-up-oracle) How to create an Oracle CDC table ---------------- @@ -373,7 +373,7 @@ Connector Options + See more about the Debezium's Oracle Connector properties @@ -481,7 +481,7 @@ Features ### Exactly-Once Processing -The Oracle CDC connector is a Flink Source connector which will read database snapshot first and then continues to read change events with **exactly-once processing** even failures happen. Please read [How the connector works](https://debezium.io/documentation/reference/1.9/connectors/oracle.html#how-the-oracle-connector-works). +The Oracle CDC connector is a Flink Source connector which will read database snapshot first and then continues to read change events with **exactly-once processing** even failures happen. Please read [How the connector works](https://debezium.io/documentation/reference/2.0/connectors/oracle.html#how-the-oracle-connector-works). ### Startup Reading Position diff --git a/docs/content.zh/docs/connectors/flink-sources/overview.md b/docs/content.zh/docs/connectors/flink-sources/overview.md index 0e8fe1052..b3a0c8741 100644 --- a/docs/content.zh/docs/connectors/flink-sources/overview.md +++ b/docs/content.zh/docs/connectors/flink-sources/overview.md @@ -186,7 +186,7 @@ The following JSON data show the change event in JSON format. "transaction": null } ``` -**Note:** Please refer [Debezium documentation](https://debezium.io/documentation/reference/1.9/connectors/mysql.html#mysql-events +**Note:** Please refer [Debezium documentation](https://debezium.io/documentation/reference/2.0/connectors/mysql.html#mysql-events ) to know the meaning of each field. In some cases, users can use the `JsonDebeziumDeserializationSchema(true)` Constructor to enabled include schema in the message. Then the Debezium JSON message may look like this: diff --git a/docs/content.zh/docs/connectors/flink-sources/postgres-cdc.md b/docs/content.zh/docs/connectors/flink-sources/postgres-cdc.md index 3e54855a4..9e089af02 100644 --- a/docs/content.zh/docs/connectors/flink-sources/postgres-cdc.md +++ b/docs/content.zh/docs/connectors/flink-sources/postgres-cdc.md @@ -184,7 +184,7 @@ Connector Options + See more about the Debezium's Postgres Connector properties @@ -239,7 +239,7 @@ Connector Options
    -Note: `slot.name` is recommended to set for different tables to avoid the potential `PSQLException: ERROR: replication slot "flink" is active for PID 974` error. See more [here](https://debezium.io/documentation/reference/1.9/connectors/postgresql.html#postgresql-property-slot-name). +Note: `slot.name` is recommended to set for different tables to avoid the potential `PSQLException: ERROR: replication slot "flink" is active for PID 974` error. See more [here](https://debezium.io/documentation/reference/2.0/connectors/postgresql.html#postgresql-property-slot-name). ### Incremental Snapshot Options @@ -430,7 +430,7 @@ and then PostgreSQL CDC Source assigns the chunks to multiple readers to read th ### Exactly-Once Processing -The Postgres CDC connector is a Flink Source connector which will read database snapshot first and then continues to read binlogs with **exactly-once processing** even failures happen. Please read [How the connector works](https://debezium.io/documentation/reference/1.9/connectors/postgresql.html#how-the-postgresql-connector-works). +The Postgres CDC connector is a Flink Source connector which will read database snapshot first and then continues to read binlogs with **exactly-once processing** even failures happen. Please read [How the connector works](https://debezium.io/documentation/reference/2.0/connectors/postgresql.html#how-the-postgresql-connector-works). ### DataStream Source diff --git a/docs/content.zh/docs/connectors/flink-sources/sqlserver-cdc.md b/docs/content.zh/docs/connectors/flink-sources/sqlserver-cdc.md index abc2779ef..155c289e2 100644 --- a/docs/content.zh/docs/connectors/flink-sources/sqlserver-cdc.md +++ b/docs/content.zh/docs/connectors/flink-sources/sqlserver-cdc.md @@ -215,7 +215,7 @@ Connector Options
    + See more about the Debezium's SQLServer Connector properties @@ -317,7 +317,7 @@ Features ### Exactly-Once Processing -The SQLServer CDC connector is a Flink Source connector which will read database snapshot first and then continues to read change events with **exactly-once processing** even failures happen. Please read [How the connector works](https://debezium.io/documentation/reference/1.9/connectors/sqlserver.html#how-the-sqlserver-connector-works). +The SQLServer CDC connector is a Flink Source connector which will read database snapshot first and then continues to read change events with **exactly-once processing** even failures happen. Please read [How the connector works](https://debezium.io/documentation/reference/2.0/connectors/sqlserver.html#how-the-sqlserver-connector-works). ### Startup Reading Position diff --git a/docs/content.zh/docs/connectors/flink-sources/vitess-cdc.md b/docs/content.zh/docs/connectors/flink-sources/vitess-cdc.md index 94bbf1311..12ad11989 100644 --- a/docs/content.zh/docs/connectors/flink-sources/vitess-cdc.md +++ b/docs/content.zh/docs/connectors/flink-sources/vitess-cdc.md @@ -257,7 +257,7 @@ Data Type Mapping
    MySQL typeMySQL type Flink SQL type NOTE
    String Pass-through Debezium's properties to Debezium Embedded Engine which is used to capture data changes from Oracle server. For example: 'debezium.snapshot.mode' = 'never'. - See more about the Debezium's Oracle Connector properties
    scan.incremental.close-idle-reader.enabledString Pass-through Debezium's properties to Debezium Embedded Engine which is used to capture data changes from Postgres server. For example: 'debezium.snapshot.mode' = 'never'. - See more about the Debezium's Postgres Connector properties
    debezium.snapshot.select.statement.overrides String Pass-through Debezium's properties to Debezium Embedded Engine which is used to capture data changes from SQLServer. For example: 'debezium.snapshot.mode' = 'initial_only'. - See more about the Debezium's SQLServer Connector properties
    scan.incremental.close-idle-reader.enabled
    - + diff --git a/docs/content.zh/docs/connectors/pipeline-connectors/mysql.md b/docs/content.zh/docs/connectors/pipeline-connectors/mysql.md index 133bc6983..ca44085cf 100644 --- a/docs/content.zh/docs/connectors/pipeline-connectors/mysql.md +++ b/docs/content.zh/docs/connectors/pipeline-connectors/mysql.md @@ -223,7 +223,7 @@ pipeline: + 查看更多关于 Debezium 的 MySQL 连接器属性 @@ -253,7 +253,7 @@ pipeline:
    MySQL typeMySQL type Flink SQL type
    String 将 Debezium 的属性传递给 Debezium 嵌入式引擎,该引擎用于从 MySQL 服务器捕获数据更改。 例如: 'debezium.snapshot.mode' = 'never'. - 查看更多关于 Debezium 的 MySQL 连接器属性
    scan.incremental.close-idle-reader.enabled
    - + diff --git a/docs/content.zh/docs/developer-guide/contribute-to-flink-cdc.md b/docs/content.zh/docs/developer-guide/contribute-to-flink-cdc.md index 29d24a86d..1ec1a42ff 100644 --- a/docs/content.zh/docs/developer-guide/contribute-to-flink-cdc.md +++ b/docs/content.zh/docs/developer-guide/contribute-to-flink-cdc.md @@ -33,12 +33,12 @@ Bug报告,提议新的功能,加入社区邮件列表的讨论,贡献代 Flink CDC 社区的贡献不仅限于为项目贡献代码,下面列举了一些可以在社区贡献的内容。 -| 贡献方式 | 更多信息 | +| 贡献方式 | 更多信息 | |:------|:-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| 提交BUG | 为了提交问题,您需要首先在 [Flink jira](https://issues.apache.org/jira/projects/FLINK/issues) 建立对应的issue,并在`Component/s`选择`Flink CDC`。然后在问题描述中详细描述遇到的问题的信息,如果可能的话,最好提供一下能够复现问题的操作步骤。 | +| 提交BUG | 为了提交问题,您需要首先在 [Flink jira](https://issues.apache.org/jira/projects/FLINK/issues) 建立对应的issue,并在`Component/s`选择`Flink CDC`。然后在问题描述中详细描述遇到的问题的信息,如果可能的话,最好提供一下能够复现问题的操作步骤。 | | 贡献代码 | 请阅读 贡献代码指导 | | 代码评审 | 请阅读 代码评审指导 | -| 用户支持 | 通过 [Flink 用户邮件列表](https://flink.apache.org/what-is-flink/community/#mailing-lists) 来帮助回复用户问题,在 [Flink jira](https://issues.
    apache.org/jira/projects/FLINK/issues) 可以查询到最新的已知问题。 | +| 用户支持 | 通过 [Flink 用户邮件列表](https://flink.apache.org/what-is-flink/community/#mailing-lists) 来帮助回复用户问题,在 [Flink jira](https://issues.apache.org/jira/projects/FLINK/issues) 可以查询到最新的已知问题。 | 如果还有其他问题,可以通过 Flink Dev 邮件列表寻求帮助。 @@ -50,7 +50,7 @@ Flink CDC 项目通过众多贡献者的代码贡献来维护,改进和拓展 如果您想要为 Flink CDC 贡献代码,可以通过如下的方式。 -1. 首先在 [Flink jira](https://issues.
    apache.org/jira/projects/FLINK/issues) 的想要负责的 issue +1. 首先在 [Flink jira](https://issues.apache.org/jira/projects/FLINK/issues) 的想要负责的 issue 下评论(最好在评论中解释下对于这个问题的理解,和后续的设计,如果可能的话也可以提供下 POC 的代码)。 2. 在这个 issue 被分配给你后,开始进行开发实现(提交信息请遵循`[FLINK-xxx][xxx] xxxxxxx`的格式)。 3. 开发完成后可以向 [Flink CDC](https://github.com/apache/flink-cdc) 项目提交 PR(请确保 Clone 的项目 committer 有操作权限)。 diff --git a/docs/content/_index.md b/docs/content/_index.md index b2a10cb03..15b4a314a 100644 --- a/docs/content/_index.md +++ b/docs/content/_index.md @@ -165,7 +165,7 @@ under the License.

    Try Flink CDC

    Flink CDC provides a series of quick start demos without any dependencies or java code. A Linux or MacOS computer with Docker installed is enough. - Please check out our Quick Start for more information. + Please check out our }}">Quick Start for more information.

    diff --git a/docs/content/docs/connectors/flink-sources/db2-cdc.md b/docs/content/docs/connectors/flink-sources/db2-cdc.md index ef06c1b3e..59cde68af 100644 --- a/docs/content/docs/connectors/flink-sources/db2-cdc.md +++ b/docs/content/docs/connectors/flink-sources/db2-cdc.md @@ -32,9 +32,9 @@ describes how to setup the db2 CDC connector to run SQL queries against Db2 data ## Supported Databases -| Connector | Database | Driver | -|-----------------------|----------------------------------------------------|----------------------| -| [Db2-cdc](../db2-cdc) |
  • [Db2](https://www.ibm.com/products/db2): 11.5 | Db2 Driver: 11.5.0.0 | +| Connector | Database | Driver | +|-----------|----------------------------------------------------|----------------------| +| [Db2-cdc] |
  • [Db2](https://www.ibm.com/products/db2): 11.5 | Db2 Driver: 11.5.0.0 | Dependencies ------------ @@ -56,7 +56,7 @@ put it under `/lib/`. Setup Db2 server ---------------- -Follow the steps in the [Debezium Db2 Connector](https://debezium.io/documentation/reference/1.9/connectors/db2.html#setting-up-db2). +Follow the steps in the [Debezium Db2 Connector](https://debezium.io/documentation/reference/2.0/connectors/db2.html#setting-up-db2). Notes @@ -184,7 +184,7 @@ for more detailed information.
  • @@ -196,7 +196,7 @@ for more detailed information. + See more about the Debezium's Db2 Connector properties
    MySQL typeMySQL type CDC type NOTE
    String The session time zone in database server, e.g. "Asia/Shanghai". It controls how the TIMESTAMP type in Db2 converted to STRING. - See more here. + See more here. If not set, then ZoneId.systemDefault() is used to determine the server time zone.
    Pass-through Debezium's properties to Debezium Embedded Engine which is used to capture data changes from Db2 server. For example: 'debezium.snapshot.mode' = 'never'. - See more about the Debezium's Db2 Connector properties
    diff --git a/docs/content/docs/connectors/flink-sources/mysql-cdc.md b/docs/content/docs/connectors/flink-sources/mysql-cdc.md index ec28c1cca..11b69abf3 100644 --- a/docs/content/docs/connectors/flink-sources/mysql-cdc.md +++ b/docs/content/docs/connectors/flink-sources/mysql-cdc.md @@ -31,9 +31,9 @@ The MySQL CDC connector allows for reading snapshot data and incremental data fr ## Supported Databases -| Connector | Database | Driver | -|---------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|-------------------------| -| [mysql-cdc](../mysql-cdc) |
  • [MySQL](https://dev.mysql.com/doc): 5.6, 5.7, 8.0.x
  • [RDS MySQL](https://www.aliyun.com/product/rds/mysql): 5.6, 5.7, 8.0.x
  • [PolarDB MySQL](https://www.aliyun.com/product/polardb): 5.6, 5.7, 8.0.x
  • [Aurora MySQL](https://aws.amazon.com/cn/rds/aurora): 5.6, 5.7, 8.0.x
  • [MariaDB](https://mariadb.org): 10.x
  • [PolarDB X](https://github.com/ApsaraDB/galaxysql): 2.0.1 | JDBC Driver: 8.0.27 | +| Connector | Database | Driver | +|-----------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|---------------------| +| mysql-cdc |
  • [MySQL](https://dev.mysql.com/doc): 5.6, 5.7, 8.0.x
  • [RDS MySQL](https://www.aliyun.com/product/rds/mysql): 5.6, 5.7, 8.0.x
  • [PolarDB MySQL](https://www.aliyun.com/product/polardb): 5.6, 5.7, 8.0.x
  • [Aurora MySQL](https://aws.amazon.com/cn/rds/aurora): 5.6, 5.7, 8.0.x
  • [MariaDB](https://mariadb.org): 10.x
  • [PolarDB X](https://github.com/ApsaraDB/galaxysql): 2.0.1 | JDBC Driver: 8.0.27 | Dependencies ------------ @@ -74,7 +74,7 @@ mysql> GRANT SELECT, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT ON *. mysql> FLUSH PRIVILEGES; ``` -See more about the [permission explanation](https://debezium.io/documentation/reference/1.9/connectors/mysql.html#mysql-creating-user). +See more about the [permission explanation](https://debezium.io/documentation/reference/2.0/connectors/mysql.html#mysql-creating-user). Notes @@ -282,7 +282,7 @@ Connector Options String The session time zone in database server, e.g. "Asia/Shanghai". It controls how the TIMESTAMP type in MYSQL converted to STRING. - See more here. + See more here. If not set, then ZoneId.systemDefault() is used to determine the server time zone. @@ -337,7 +337,7 @@ During a snapshot operation, the connector will query each included table to pro String Pass-through Debezium's properties to Debezium Embedded Engine which is used to capture data changes from MySQL server. For example: 'debezium.snapshot.mode' = 'never'. - See more about the Debezium's MySQL Connector properties + See more about the Debezium's MySQL Connector properties scan.incremental.close-idle-reader.enabled @@ -800,7 +800,7 @@ Data Type Mapping - + diff --git a/docs/content/docs/connectors/flink-sources/oracle-cdc.md b/docs/content/docs/connectors/flink-sources/oracle-cdc.md index 9f17cd7af..992b7b440 100644 --- a/docs/content/docs/connectors/flink-sources/oracle-cdc.md +++ b/docs/content/docs/connectors/flink-sources/oracle-cdc.md @@ -206,7 +206,7 @@ Overall, the steps for configuring CDB database is quite similar to non-CDB data exit ``` -See more about the [Setting up Oracle](https://debezium.io/documentation/reference/1.9/connectors/oracle.html#setting-up-oracle) +See more about the [Setting up Oracle](https://debezium.io/documentation/reference/2.0/connectors/oracle.html#setting-up-oracle) How to create an Oracle CDC table ---------------- @@ -373,7 +373,7 @@ Connector Options + See more about the Debezium's Oracle Connector properties @@ -481,7 +481,7 @@ Features ### Exactly-Once Processing -The Oracle CDC connector is a Flink Source connector which will read database snapshot first and then continues to read change events with **exactly-once processing** even failures happen. Please read [How the connector works](https://debezium.io/documentation/reference/1.9/connectors/oracle.html#how-the-oracle-connector-works). +The Oracle CDC connector is a Flink Source connector which will read database snapshot first and then continues to read change events with **exactly-once processing** even failures happen. Please read [How the connector works](https://debezium.io/documentation/reference/2.0/connectors/oracle.html#how-the-oracle-connector-works). ### Startup Reading Position diff --git a/docs/content/docs/connectors/flink-sources/overview.md b/docs/content/docs/connectors/flink-sources/overview.md index 13b33d7f3..12ce1eb9f 100644 --- a/docs/content/docs/connectors/flink-sources/overview.md +++ b/docs/content/docs/connectors/flink-sources/overview.md @@ -186,7 +186,7 @@ The following JSON data show the change event in JSON format. "transaction": null } ``` -**Note:** Please refer [Debezium documentation](https://debezium.io/documentation/reference/1.9/connectors/mysql.html#mysql-events +**Note:** Please refer [Debezium documentation](https://debezium.io/documentation/reference/2.0/connectors/mysql.html#mysql-events ) to know the meaning of each field. In some cases, users can use the `JsonDebeziumDeserializationSchema(true)` Constructor to enabled include schema in the message. Then the Debezium JSON message may look like this: diff --git a/docs/content/docs/connectors/flink-sources/postgres-cdc.md b/docs/content/docs/connectors/flink-sources/postgres-cdc.md index 3e54855a4..9e089af02 100644 --- a/docs/content/docs/connectors/flink-sources/postgres-cdc.md +++ b/docs/content/docs/connectors/flink-sources/postgres-cdc.md @@ -184,7 +184,7 @@ Connector Options + See more about the Debezium's Postgres Connector properties @@ -239,7 +239,7 @@ Connector Options
    -Note: `slot.name` is recommended to set for different tables to avoid the potential `PSQLException: ERROR: replication slot "flink" is active for PID 974` error. See more [here](https://debezium.io/documentation/reference/1.9/connectors/postgresql.html#postgresql-property-slot-name). +Note: `slot.name` is recommended to set for different tables to avoid the potential `PSQLException: ERROR: replication slot "flink" is active for PID 974` error. See more [here](https://debezium.io/documentation/reference/2.0/connectors/postgresql.html#postgresql-property-slot-name). ### Incremental Snapshot Options @@ -430,7 +430,7 @@ and then PostgreSQL CDC Source assigns the chunks to multiple readers to read th ### Exactly-Once Processing -The Postgres CDC connector is a Flink Source connector which will read database snapshot first and then continues to read binlogs with **exactly-once processing** even failures happen. Please read [How the connector works](https://debezium.io/documentation/reference/1.9/connectors/postgresql.html#how-the-postgresql-connector-works). +The Postgres CDC connector is a Flink Source connector which will read database snapshot first and then continues to read binlogs with **exactly-once processing** even failures happen. Please read [How the connector works](https://debezium.io/documentation/reference/2.0/connectors/postgresql.html#how-the-postgresql-connector-works). ### DataStream Source diff --git a/docs/content/docs/connectors/flink-sources/sqlserver-cdc.md b/docs/content/docs/connectors/flink-sources/sqlserver-cdc.md index abc2779ef..155c289e2 100644 --- a/docs/content/docs/connectors/flink-sources/sqlserver-cdc.md +++ b/docs/content/docs/connectors/flink-sources/sqlserver-cdc.md @@ -215,7 +215,7 @@ Connector Options
    + See more about the Debezium's SQLServer Connector properties @@ -317,7 +317,7 @@ Features ### Exactly-Once Processing -The SQLServer CDC connector is a Flink Source connector which will read database snapshot first and then continues to read change events with **exactly-once processing** even failures happen. Please read [How the connector works](https://debezium.io/documentation/reference/1.9/connectors/sqlserver.html#how-the-sqlserver-connector-works). +The SQLServer CDC connector is a Flink Source connector which will read database snapshot first and then continues to read change events with **exactly-once processing** even failures happen. Please read [How the connector works](https://debezium.io/documentation/reference/2.0/connectors/sqlserver.html#how-the-sqlserver-connector-works). ### Startup Reading Position diff --git a/docs/content/docs/connectors/flink-sources/vitess-cdc.md b/docs/content/docs/connectors/flink-sources/vitess-cdc.md index 94bbf1311..12ad11989 100644 --- a/docs/content/docs/connectors/flink-sources/vitess-cdc.md +++ b/docs/content/docs/connectors/flink-sources/vitess-cdc.md @@ -257,7 +257,7 @@ Data Type Mapping
    MySQL typeMySQL type Flink SQL type NOTE
    String Pass-through Debezium's properties to Debezium Embedded Engine which is used to capture data changes from Oracle server. For example: 'debezium.snapshot.mode' = 'never'. - See more about the Debezium's Oracle Connector properties
    scan.incremental.close-idle-reader.enabledString Pass-through Debezium's properties to Debezium Embedded Engine which is used to capture data changes from Postgres server. For example: 'debezium.snapshot.mode' = 'never'. - See more about the Debezium's Postgres Connector properties
    debezium.snapshot.select.statement.overrides String Pass-through Debezium's properties to Debezium Embedded Engine which is used to capture data changes from SQLServer. For example: 'debezium.snapshot.mode' = 'initial_only'. - See more about the Debezium's SQLServer Connector properties
    scan.incremental.close-idle-reader.enabled
    - + diff --git a/docs/content/docs/connectors/pipeline-connectors/mysql.md b/docs/content/docs/connectors/pipeline-connectors/mysql.md index a2971dcf4..7afd5d0a1 100644 --- a/docs/content/docs/connectors/pipeline-connectors/mysql.md +++ b/docs/content/docs/connectors/pipeline-connectors/mysql.md @@ -226,7 +226,7 @@ pipeline: + See more about the Debezium's MySQL Connector properties @@ -262,7 +262,7 @@ The config option `scan.startup.mode` specifies the startup mode for MySQL CDC c
    MySQL typeMySQL type Flink SQL type
    String Pass-through Debezium's properties to Debezium Embedded Engine which is used to capture data changes from MySQL server. For example: 'debezium.snapshot.mode' = 'never'. - See more about the Debezium's MySQL Connector properties
    scan.incremental.close-idle-reader.enabled
    - +
    MySQL typeMySQL type Flink CDC type Note