diff --git a/docs/content.zh/docs/connectors/flink-sources/oceanbase-cdc.md b/docs/content.zh/docs/connectors/flink-sources/oceanbase-cdc.md index 70d8e6a1229..b6be7a20d99 100644 --- a/docs/content.zh/docs/connectors/flink-sources/oceanbase-cdc.md +++ b/docs/content.zh/docs/connectors/flink-sources/oceanbase-cdc.md @@ -36,9 +36,7 @@ OceanBase CDC 连接器允许从 OceanBase 读取快照数据和增量数据。 - *OceanBase CE*: OceanBase 社区版。OceanBase 的开源版本,兼容 MySQL https://github.com/oceanbase/oceanbase 。 - *OceanBase EE*: OceanBase 企业版。OceanBase 的商业版本,支持 MySQL 和 Oracle 两种兼容模式 https://www.oceanbase.com 。 - *OceanBase Cloud*: OceanBase 云数据库 https://www.oceanbase.com/product/cloud 。 -- *Log Proxy CE*: OceanBase 日志代理服务社区版。单独使用时支持 CDC 模式,是一个获取 OceanBase 社区版事务日志(commit log)的代理服务 https://github.com/oceanbase/oblogproxy 。 -- *Log Proxy EE*: OceanBase 日志代理服务企业版。单独使用时支持 CDC 模式,是一个获取 OceanBase 企业版事务日志(commit log)的代理服务,目前仅在 OceanBase Cloud 上提供有限的支持, 详情请咨询相关技术支持。 -- *Binlog Service CE*: OceanBase Binlog 服务社区版。OceanBase 社区版的一个兼容 MySQL 复制协议的解决方案,详情参考 Log Proxy CE Binlog 模式的文档。 +- *Binlog Service CE*: OceanBase Binlog 服务社区版。OceanBase 社区版的一个兼容 MySQL 复制协议的解决方案,详情见[文档](https://www.oceanbase.com/docs/oblogproxy-doc)。 - *Binlog Service EE*: OceanBase Binlog 服务企业版。OceanBase 企业版 MySQL 模式的一个兼容 MySQL 复制协议的解决方案,仅可在阿里云使用,详情见[操作指南](https://www.alibabacloud.com/help/zh/apsaradb-for-oceanbase/latest/binlog-overview)。 - *MySQL Driver*: `mysql-connector-java`,可用于 OceanBase 社区版和 OceanBase 企业版 MySQL 模式。 - *OceanBase Driver*: OceanBase JDBC 驱动,支持所有版本的 MySQL 和 Oracle 兼容模式 https://github.com/oceanbase/obconnector-j 。 @@ -59,11 +57,10 @@ OceanBase CDC 源端读取方案: OceanBase CE - MySQL Driver: 5.1.4x, 8.0.x
- OceanBase Driver: 2.4.x + MySQL Driver: 8.0.x
OceanBase CDC Connector - Log Proxy CE + Binlog Service CE MySQL Driver: 8.0.x @@ -73,11 +70,10 @@ OceanBase CDC 源端读取方案: OceanBase EE (MySQL 模式) - MySQL Driver: 5.1.4x, 8.0.x
- OceanBase Driver: 2.4.x + MySQL Driver: 8.0.x
OceanBase CDC Connector - Log Proxy EE + Binlog Service EE MySQL Driver: 8.0.x @@ -86,15 +82,20 @@ OceanBase CDC 源端读取方案: OceanBase EE (Oracle 模式) - OceanBase Driver: 2.4.x - OceanBase CDC Connector - Log Proxy EE (CDC 模式) + + 暂不支持 OceanBase Oracle 兼容模式下的增量订阅服务,请联系企业技术支持。 + -注意: 对于使用 OceanBase 社区版或 OceanBase 企业版 MySQL 模式的用户,我们推荐参考 [MySQL CDC 的文档](mysql-cdc.md),使用 MySQL CDC 连接器搭配 Binlog 服务。 +**注意事项**: oceanbase-cdc 连接器 从 3.5 版本开始进行了以下大的改动: +- 先前基于 OceanBase Log Proxy 服务实现的连接器已被正式移除,当前将仅支持连接到 OceanBase Binlog 服务。 +- 当前版本 oceanbase-cdc 将基于 mysql-cdc 连接器实现,主要改进了对 OceanBase Binlog 服务的兼容性,包含一些 Bug 修复,推荐使用。 +- 由于 OceanBase Binlog 服务兼容 MySQL 复制协议,仍然支持使用 [MySQL CDC](mysql-cdc.md) 连接器连接到 OceanBase Binlog 服务。 +- 暂不支持 OceanBase Oracle 兼容模式下的增量订阅服务,请联系企业技术支持。 + 依赖 ------------ @@ -113,7 +114,7 @@ OceanBase CDC 源端读取方案: **注意:** 参考 [flink-sql-connector-oceanbase-cdc](https://mvnrepository.com/artifact/org.apache.flink/flink-sql-connector-oceanbase-cdc) 当前已发布的所有版本都可以在 Maven 中央仓库获取。 -由于 MySQL Driver 和 OceanBase Driver 使用的开源协议都与 Flink CDC 项目不兼容,我们无法在 jar 包中提供驱动。 您可能需要手动配置以下依赖: +由于 MySQL Driver 使用的开源协议与 Flink CDC 项目不兼容,我们无法在 jar 包中提供驱动。 您可能需要手动配置以下依赖:
@@ -129,357 +130,396 @@ OceanBase CDC 源端读取方案: - - - - - -
用于连接到 OceanBase 数据库的 MySQL 租户。
com.oceanbase:oceanbase-client:2.4.9用于连接到 OceanBase 数据库的 MySQL 或 Oracle 租户。
-配置 OceanBase 数据库和 Log Proxy 服务 +部署 OceanBase 数据库和 Binlog 服务 ---------------------- -1. 按照 [文档](https://github.com/oceanbase/oceanbase#quick-start) 配置 OceanBase 集群。 -2. 在 sys 租户中,为 oblogproxy 创建一个带密码的用户。 - - ```shell - mysql -h${host} -P${port} -uroot - - mysql> SHOW TENANT; - mysql> CREATE USER ${sys_username} IDENTIFIED BY '${sys_password}'; - mysql> GRANT ALL PRIVILEGES ON *.* TO ${sys_username} WITH GRANT OPTION; - ``` +1. 按照 [文档](https://www.oceanbase.com/docs/common-oceanbase-database-cn-1000000002012734) 部署 OceanBase 数据库。 +2. 按照 [文档](https://www.oceanbase.com/docs/common-oblogproxy-doc-1000000003053708) 部署 OceanBase Binlog 服务。 -3. 为你想要监控的租户创建一个用户,这个用户用来读取快照数据和变化事件数据。 -4. OceanBase 社区版用户需要获取`rootserver-list`,可以使用以下命令获取: - ```bash - mysql> SHOW PARAMETERS LIKE 'rootservice_list'; - ``` - OceanBase 企业版用户需要获取 `config-url`,可以使用以下命令获取: - - ```shell - mysql> show parameters like 'obconfig_url'; - ``` +使用文档 +---------------- -5. 设置 OceanBase LogProxy。 对于OceanBase社区版的用户,您可以按照[此文档](https://www.oceanbase.com/docs/community-oblogproxy-doc-1000000000531984)进行操作。 +**注意事项**: +- 当前版本 oceanbase-cdc 连接器基于 mysql-cdc 连接器实现,只修改了部分内部实现。外部接口、使用参数等 和 mysql-cdc 基本保持一致,使用文档也可参考[MySQL CDC 使用文档](mysql-cdc.md)。 -创建 OceanBase CDC 表 ----------------- +### 创建 OceanBase CDC 表 -使用以下命令,创建 OceanBase CDC 表: +OceanBase CDC 表可以定义如下: ```sql --- 每 3 秒做一次 checkpoint,用于测试,生产配置建议 5 到 10 分钟 -Flink SQL> SET 'execution.checkpointing.interval' = '3s'; +-- 每 3 秒做一次 checkpoint,用于测试,生产配置建议5到10分钟 +Flink SQL> SET 'execution.checkpointing.interval' = '3s'; --- 在 Flink SQL 中创建 OceanBase 表 `orders` +-- 在 Flink SQL中注册 OceanBase 表 'orders' Flink SQL> CREATE TABLE orders ( - order_id INT, - order_date TIMESTAMP(0), - customer_name STRING, - price DECIMAL(10, 5), - product_id INT, - order_status BOOLEAN, - PRIMARY KEY (order_id) NOT ENFORCED -) WITH ( - 'connector' = 'oceanbase-cdc', - 'scan.startup.mode' = 'initial', - 'username' = 'user@test_tenant#cluster_name', - 'password' = 'pswd', - 'tenant-name' = 'test_tenant', - 'database-name' = '^test_db$', - 'table-name' = '^orders$', - 'hostname' = '127.0.0.1', - 'port' = '2881', - 'rootserver-list' = '127.0.0.1:2882:2881', - 'logproxy.host' = '127.0.0.1', - 'logproxy.port' = '2983', - 'working-mode' = 'memory' -); - --- 从表 orders 中读取快照数据和 binlog 数据 + order_id INT, + order_date TIMESTAMP(0), + customer_name STRING, + price DECIMAL(10, 5), + product_id INT, + order_status BOOLEAN, + PRIMARY KEY(order_id) NOT ENFORCED + ) WITH ( + 'connector' = 'oceanbase-cdc', + 'hostname' = 'localhost', + 'port' = '2881', + 'username' = 'root', + 'password' = '123456', + 'database-name' = 'mydb', + 'table-name' = 'orders'); + +-- 从订单表读取全量数据(快照)和增量数据(binlog) Flink SQL> SELECT * FROM orders; ``` -如果您使用的是企业版的 OceanBase Oracle 模式,您需要先添加 OceanBase 的官方 JDBC 驱动 jar 包到 Flink 环境,并且部署企业版的 oblogproxy 服务,然后通过以下命令创建 OceanBase CDC 表: - -```sql -Flink SQL> CREATE TABLE orders ( - order_id INT, - order_date TIMESTAMP(0), - customer_name STRING, - price DECIMAL(10, 5), - product_id INT, - order_status BOOLEAN, - PRIMARY KEY (order_id) NOT ENFORCED -) WITH ( - 'connector' = 'oceanbase-cdc', - 'scan.startup.mode' = 'initial', - 'username' = 'user@test_tenant#cluster_name', - 'password' = 'pswd', - 'tenant-name' = 'test_tenant', - 'database-name' = '^test_db$', - 'table-name' = '^orders$', - 'hostname' = '127.0.0.1', - 'port' = '2881', - 'compatible-mode' = 'oracle', - 'jdbc.driver' = 'com.oceanbase.jdbc.Driver', - 'config-url' = 'http://127.0.0.1:8080/services?Action=ObRootServiceInfo&User_ID=xxx&UID=xxx&ObRegion=xxx', - 'logproxy.host' = '127.0.0.1', - 'logproxy.port' = '2983', - 'working-mode' = 'memory' -); -``` - -您也可以访问 Flink CDC 官网文档,快速体验将数据从 OceanBase 导入到 Elasticsearch。更多信息,参考 [Flink CDC 官网文档]({{< ref "docs/connectors/flink-sources/tutorials/oceanbase-tutorial" >}})。 - -OceanBase CDC 连接器选项 +连接器选项 ---------------- -OceanBase CDC 连接器包括用于 SQL 和 DataStream API 的选项,如下表所示。 - -*注意*:连接器支持两种方式来指定需要监听的表,两种方式同时使用时会监听两种方式匹配的所有表。 -1. 使用 `database-name` 和 `table-name` 匹配正则表达式中的数据库和表名。 -2. 使用 `table-list` 去匹配数据库名和表名的准确列表。 -
- - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
配置项是否必选默认值类型描述
connectorString指定要使用的连接器,此处为 'oceanbase-cdc'
scan.startup.modeinitialString指定 OceanBase CDC 消费者的启动模式。可取值为 - 'initial''latest-offset''timestamp''snapshot'。 -
scan.startup.timestampLong起始点的时间戳,单位为秒。仅在启动模式为 'timestamp' 时可用。
usernameString连接 OceanBase 数据库的用户的名称。
passwordString连接 OceanBase 数据库时使用的密码。
tenant-nameString待监控 OceanBase 数据库的租户名,应该填入精确值。
database-nameString待监控 OceanBase 数据库的数据库名,应该是正则表达式。
table-nameString待监控 OceanBase 数据库的表名,应该是正则表达式。
table-listString待监控 OceanBase 数据库的全路径的表名列表,逗号分隔,如:"db1.table1, db2.table2"。
hostnameStringOceanBase 数据库或 OceanBbase 代理 ODP 的 IP 地址或主机名。
portInteger - OceanBase 数据库服务器的整数端口号。可以是 OceanBase 服务器的 SQL 端口号(默认值为 2881)
- 或 OceanBase代理服务的端口号(默认值为 2883)
connect.timeout30sDuration连接器在尝试连接到 OceanBase 数据库服务器超时前的最长时间。
server-time-zone+00:00String - 数据库服务器中的会话时区,用户控制 OceanBase 的时间类型如何转换为 STRING。
- 合法的值可以是格式为"±hh:mm"的 UTC 时区偏移量,
- 如果 mysql 数据库中的时区信息表已创建,合法的值则可以是创建的时区。 -
logproxy.hostStringOceanBase 日志代理服务 的 IP 地址或主机名。
logproxy.portIntegerOceanBase 日志代理服务 的端口号。
logproxy.client.id规则生成StringOceanBase日志代理服务的客户端连接 ID,默认值的生成规则是 {flink_ip}_{process_id}_{timestamp}_{thread_id}_{tenant}。
rootserver-listStringOceanBase root 服务器列表,服务器格式为 `ip:rpc_port:sql_port`,
多个服务器地址使用英文分号 `;` 隔开,OceanBase 社区版本必填。
config-urlString从配置服务器获取服务器信息的 url, OceanBase 企业版本必填。
working-modestorageString日志代理中 `libobcdc` 的工作模式 , 可以是 `storage` 或 `memory`。
compatible-modemysqlStringOceanBase 的兼容模式,可以是 `mysql` 或 `oracle`。
jdbc.drivercom.mysql.cj.jdbc.DriverString全量读取时使用的 jdbc 驱动类名。
jdbc.properties.*String传递自定义 JDBC URL 属性的选项。用户可以传递自定义属性,如 'jdbc.properties.useSSL' = 'false'。
obcdc.properties.*String传递自定义 libobcdc 属性的选项,如 'obcdc.properties.sort_trans_participants' = '1'。详情参见 obcdc 配置项说明
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
OptionRequiredDefaultTypeDescription
connectorrequired(none)String指定要使用的连接器, 这里应该是 'oceanbase-cdc'.
hostnamerequired(none)String OceanBase 服务器的 IP 地址或主机名。
usernamerequired(none)String连接到 OceanBase 数据库服务器时要使用的 OceanBase 用户的名称。
passwordrequired(none)String连接 OceanBase 数据库服务器时使用的密码。
database-namerequired(none)String要监视的 OceanBase 服务器的数据库名称。数据库名称还支持正则表达式,以监视多个与正则表达式匹配的表。
table-namerequired(none)String需要监视的 OceanBase 数据库的表名。表名支持正则表达式,以监视满足正则表达式的多个表。注意:OceanBase CDC 连接器在正则匹配表名时,会把用户填写的 database-name, table-name 通过字符串 `\\.` 连接成一个全路径的正则表达式,然后使用该正则表达式和 OceanBase 数据库中表的全限定名进行正则匹配。
portoptional2881Integer OceanBase 数据库服务器的整数端口号。
server-idoptional(none)String读取数据使用的 server id,server id 可以是个整数或者一个整数范围,比如 '5400' 或 '5400-5408', + 建议在 'scan.incremental.snapshot.enabled' 参数为启用时,配置成整数范围。 默认情况下,连接器会在 5400 和 6400 之间生成一个随机数,但是我们建议用户明确指定 Server id。 +
scan.incremental.snapshot.enabledoptionaltrueBoolean增量快照是一种读取表快照的新机制,与旧的快照机制相比, + 增量快照有许多优点,包括: + (1)在快照读取期间,Source 支持并发读取, + (2)在快照读取期间,Source 支持进行 chunk 粒度的 checkpoint, + (3)在快照读取之前,Source 不需要数据库锁权限。 + 如果希望 Source 并行运行,则每个并行 Readers 都应该具有唯一的 Server id,所以 + Server id 必须是类似 `5400-6400` 的范围,并且该范围必须大于并行度。 + 请查阅 增量快照读取 章节了解更多详细信息。 +
scan.incremental.snapshot.chunk.sizeoptional8096Integer表快照的块大小(行数),读取表的快照时,捕获的表被拆分为多个块。
scan.snapshot.fetch.sizeoptional1024Integer读取表快照时每次读取数据的最大条数。
scan.incremental.snapshot.chunk.key-columnoptional(none)String表快照的分片键,在读取表的快照时,被捕获的表会按分片键拆分为多个分片。 + 默认情况下,分片键是主键的第一列。可以使用非主键列作为分片键,但这可能会导致查询性能下降。 +
+ 警告: 使用非主键列作为分片键可能会导致数据不一致。请参阅 增量快照读取 章节了解详细信息。 +
scan.startup.modeoptionalinitialString OceanBase CDC 消费者可选的启动模式, + 合法的模式为 "initial","earliest-offset","latest-offset","specific-offset","timestamp" 和 "snapshot"。 + 请查阅 启动模式 章节了解更多详细信息。
scan.startup.specific-offset.fileoptional(none)String在 "specific-offset" 启动模式下,启动位点的 binlog 文件名。
scan.startup.specific-offset.posoptional(none)Long在 "specific-offset" 启动模式下,启动位点的 binlog 文件位置。
scan.startup.specific-offset.gtid-setoptional(none)String在 "specific-offset" 启动模式下,启动位点的 GTID 集合。
scan.startup.timestamp-millisoptional(none)Long在 "timestamp" 启动模式下,启动位点的毫秒时间戳。
scan.startup.specific-offset.skip-eventsoptional(none)Long在指定的启动位点后需要跳过的事件数量。
scan.startup.specific-offset.skip-rowsoptional(none)Long在指定的启动位点后需要跳过的数据行数量。
server-time-zoneoptional(none)String数据库服务器中的会话时区, 例如: "Asia/Shanghai". + 它控制 OceanBase 中的时间戳类型如何转换为字符串。 + 更多请参考 这里. + 如果没有设置,则使用ZoneId.systemDefault()来确定服务器时区。 +
debezium.min.row. + count.to.stream.resultoptional1000Integer + 在快照操作期间,连接器将查询每个包含的表,以生成该表中所有行的读取事件。 此参数确定 OceanBase 连接是否将表的所有结果拉入内存(速度很快,但需要大量内存), 或者结果是否需要流式传输(传输速度可能较慢,但适用于非常大的表)。 该值指定了在连接器对结果进行流式处理之前,表必须包含的最小行数,默认值为1000。将此参数设置为`0`以跳过所有表大小检查,并始终在快照期间对所有结果进行流式处理。
connect.timeoutoptional30sDuration连接器在尝试连接到 OceanBase 数据库服务器后超时前应等待的最长时间。该时长不能少于250毫秒。
connect.max-retriesoptional3Integer连接器应重试以建立 OceanBase 数据库服务器连接的最大重试次数。
connection.pool.sizeoptional20Integer连接池大小。
jdbc.properties.*optional20String传递自定义 JDBC URL 属性的选项。用户可以传递自定义属性,如 'jdbc.properties.useSSL' = 'false'.
heartbeat.intervaloptional30sDuration用于跟踪最新可用 binlog 偏移的发送心跳事件的间隔。
debezium.*optional(none)String将 Debezium 的属性传递给 Debezium 嵌入式引擎,该引擎用于从 OceanBase 服务器捕获数据更改。 + For example: 'debezium.snapshot.mode' = 'never'. + 查看更多关于 Debezium 的 MySQL 连接器属性
scan.incremental.close-idle-reader.enabledoptionalfalseBoolean是否在快照结束后关闭空闲的 Reader。 此特性需要 flink 版本大于等于 1.14 并且 'execution.checkpointing.checkpoints-after-tasks-finish.enabled' 需要设置为 true。
scan.parse.online.schema.changes.enabledoptionalfalseBoolean + 是否尝试解析由 gh-ostpt-osc 工具生成的表结构变更事件。 + 这些工具会在变更表结构时,将变更语句应用到“影子表”之上,并稍后将其与主表进行交换,以达到表结构变更的目的。 +
+ 这是一项实验性功能。 +
scan.incremental.snapshot.unbounded-chunk-first.enabledoptionalfalseBoolean + 快照读取阶段是否先分配 UnboundedChunk。
+ 这有助于降低 TaskManager 在快照阶段同步最后一个chunk时遇到内存溢出 (OOM) 的风险。
+ 这是一项实验特性,默认为 false。 +
scan.read-changelog-as-append-only.enabledoptionalfalseBoolean + 是否将 changelog 数据流转换为 append-only 数据流。
+ 仅在需要保存上游表删除消息等特殊场景下开启使用,比如在逻辑删除场景下,用户不允许物理删除下游消息,此时使用该特性,并配合 row_kind 元数据字段,下游可以先保存所有明细数据,再通过 row_kind 字段判断是否进行逻辑删除。
+ 参数取值如下:
+
  • true:所有类型的消息(包括INSERT、DELETE、UPDATE_BEFORE、UPDATE_AFTER)都会转换成 INSERT 类型的消息。
  • +
  • false(默认):所有类型的消息都保持原样下发。
  • +
    scan.incremental.snapshot.backfill.skipoptionalfalseBoolean + 是否在快照读取阶段跳过 backfill 。
    + 如果跳过 backfill ,快照阶段捕获表的更改将在稍后的 binlog 读取阶段被回放,而不是合并到快照中。
    + 警告:跳过 backfill 可能会导致数据不一致,因为快照阶段发生的某些 binlog 事件可能会被重放(仅保证 at-least-once )。 + 例如,更新快照阶段已更新的值,或删除快照阶段已删除的数据。这些重放的 binlog 事件应进行特殊处理。 +
    use.legacy.json.formatoptionaltrueBoolean是否使用 legacy JSON 格式来转换 Binlog 中的 JSON 类型的数据。
    + 这代表着是否使用 legacy JSON 格式来转换 Binlog 中的 JSON 类型的数据。 + 如果用户配置 'use.legacy.json.format' = 'true',则从 Binlog 中转换 JSON 类型的数据时,会移除值之前的空格和逗号之后的空格。例如, + Binlog 中 JSON 类型的数据 {"key1": "value1", "key2": "value2"} 会被转换为 {"key1":"value1","key2":"value2"}。 + 如果设置 'use.legacy.json.format' = 'false', 这条数据会被转换为 {"key1": "value1", "key2": "value2"}, 也就是 key 和 value 前的空格都会被保留。 +
    支持的元数据 ---------------- -在创建表时,您可以使用以下格式的元数据作为只读列(VIRTUAL)。 +下表中的元数据可以在 DDL 中作为只读(虚拟)meta 列声明。 - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    列名数据类型描述
    tenant_nameSTRING当前记录所属的租户名称。
    database_nameSTRING当前记录所属的 db 名。
    schema_nameSTRING当前记录所属的 schema 名。
    table_nameSTRING NOT NULL当前记录所属的表名称。
    op_tsTIMESTAMP_LTZ(3) NOT NULL该值表示此修改在数据库中发生的时间。如果这条记录是该表在快照阶段读取的记录,则该值返回 0。
    KeyDataTypeDescription
    table_nameSTRING NOT NULL当前记录所属的表名称。
    database_nameSTRING NOT NULL当前记录所属的库名称。
    op_tsTIMESTAMP_LTZ(3) NOT NULL当前记录表在数据库中更新的时间。
    如果从表的快照而不是 binlog 读取记录,该值将始终为0。
    row_kindSTRING NOT NULL当前记录的变更类型。
    + 注意:如果 Source 算子选择为每条记录输出 row_kind 列,则下游 SQL 操作符在处理回撤时可能会由于此新添加的列而无法比较,导致出现非确定性更新问题。建议仅在简单的同步作业中使用此元数据列。
    + '+I' 表示 INSERT 消息,'-D' 表示 DELETE 消息,'-U' 表示 UPDATE_BEFORE 消息,'+U' 表示 UPDATE_AFTER 消息。
    -如下 SQL 展示了如何在表中使用这些元数据列: - +下述创建表示例展示元数据列的用法: ```sql CREATE TABLE products ( - tenant_name STRING METADATA FROM 'tenant_name' VIRTUAL, db_name STRING METADATA FROM 'database_name' VIRTUAL, table_name STRING METADATA FROM 'table_name' VIRTUAL, operation_ts TIMESTAMP_LTZ(3) METADATA FROM 'op_ts' VIRTUAL, + operation STRING METADATA FROM 'row_kind' VIRTUAL, order_id INT, order_date TIMESTAMP(0), customer_name STRING, @@ -488,389 +528,377 @@ CREATE TABLE products ( order_status BOOLEAN, PRIMARY KEY(order_id) NOT ENFORCED ) WITH ( - 'connector' = 'oceanbase-cdc', - 'scan.startup.mode' = 'initial', - 'username' = 'user@test_tenant', - 'password' = 'pswd', - 'tenant-name' = 'test_tenant', - 'database-name' = '^test_db$', - 'table-name' = '^orders$', - 'hostname' = '127.0.0.1', - 'port' = '2881', - 'rootserver-list' = '127.0.0.1:2882:2881', - 'logproxy.host' = '127.0.0.1', - 'logproxy.port' = '2983', - 'working-mode' = 'memory' + 'connector' = 'oceanbase-cdc', + 'hostname' = 'localhost', + 'port' = '2881', + 'username' = 'root', + 'password' = '123456', + 'database-name' = 'mydb', + 'table-name' = 'orders' ); ``` -特性 --------- - -### At-Least-Once 处理 - -OceanBase CDC 连接器是一个 Flink Source 连接器。它将首先读取数据库快照,然后再读取变化事件,并进行 **At-Least-Once 处理**。 - -OceanBase 数据库是一个分布式数据库,它的日志也分散在不同的服务器上。由于没有类似 MySQL binlog 偏移量的位置信息,OceanBase 数据库用时间戳作为位置标记。为确保读取完整的数据,liboblog(读取 OceanBase 日志记录的 C++ 库)可能会在给定的时间戳之前读取一些日志数据。因此,OceanBase 数据库可能会读到起始点附近时间戳的重复数据,可保证 **At-Least-Once 处理**。 - -### 启动模式 - -配置选项 `scan.startup.mode` 指定 OceanBase CDC 连接器的启动模式。可用取值包括: - -- `initial`(默认):在首次启动时对受监视的数据库表执行初始快照,并继续读取最新的提交日志。 -- `latest-offset`:首次启动时,不对受监视的数据库表执行快照,仅从连接器启动时读取提交日志。 -- `timestamp`:在首次启动时不对受监视的数据库表执行初始快照,仅从指定的 `scan.startup.timestamp` 读取最新的提交日志。 -- `snapshot`: 仅对受监视的数据库表执行初始快照。 - -### 消费提交日志 - -OceanBase CDC 连接器使用 [oblogclient](https://github.com/oceanbase/oblogclient) 消费 OceanBase日志代理服务 中的事务日志。 - -### DataStream Source +下述创建表示例展示使用正则表达式匹配多张库表的用法: +```sql +CREATE TABLE products ( + db_name STRING METADATA FROM 'database_name' VIRTUAL, + table_name STRING METADATA FROM 'table_name' VIRTUAL, + operation_ts TIMESTAMP_LTZ(3) METADATA FROM 'op_ts' VIRTUAL, + operation STRING METADATA FROM 'row_kind' VIRTUAL, + order_id INT, + order_date TIMESTAMP(0), + customer_name STRING, + price DECIMAL(10, 5), + product_id INT, + order_status BOOLEAN, + PRIMARY KEY(order_id) NOT ENFORCED +) WITH ( + 'connector' = 'oceanbase-cdc', + 'hostname' = 'localhost', + 'port' = '2881', + 'username' = 'root', + 'password' = '123456', + 'database-name' = '(^(test).*|^(tpc).*|txc|.*[p$]|t{2})', + 'table-name' = '(t[5-8]|tt)' +); +``` + + + + + + + + + + + + + + + + + + + + + + + + + +
    匹配示例表达式描述
    前缀匹配^(test).*匹配前缀为test的数据库名或表名,例如test1、test2等。
    后缀匹配.*[p$]匹配后缀为p的数据库名或表名,例如cdcp、edcp等。
    特定匹配txc匹配具体的数据库名或表名。
    -OceanBase CDC 连接器也可以作为 DataStream Source 使用。您可以按照如下创建一个 SourceFunction: +进行库表匹配时,会使用正则表达式 `database-name\\.table-name` 来与OceanBase表的全限定名做匹配,所以该例子使用 `(^(test).*|^(tpc).*|txc|.*[p$]|t{2})\\.(t[5-8]|tt)`,可以匹配到表 txc.tt、test2.test5。 -```java -import org.apache.flink.cdc.connectors.base.options.StartupOptions; -import org.apache.flink.cdc.connectors.oceanbase.OceanBaseSource; -import org.apache.flink.cdc.debezium.JsonDebeziumDeserializationSchema; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -public class OceanBaseSourceExample { - public static void main(String[] args) throws Exception { - SourceFunction oceanBaseSource = - OceanBaseSource.builder() - .startupOptions(StartupOptions.initial()) - .hostname("127.0.0.1") - .port(2881) - .username("user@test_tenant") - .password("pswd") - .compatibleMode("mysql") - .jdbcDriver("com.mysql.cj.jdbc.Driver") - .tenantName("test_tenant") - .databaseName("^test_db$") - .tableName("^test_table$") - .logProxyHost("127.0.0.1") - .logProxyPort(2983) - .rsList("127.0.0.1:2882:2881") - .serverTimeZone("+08:00") - .deserializer(new JsonDebeziumDeserializationSchema()) - .build(); +数据类型映射 +---------------- - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    OceanBase typeFlink SQL typeNOTE
    TINYINTTINYINT
    + SMALLINT
    + TINYINT UNSIGNED
    + TINYINT UNSIGNED ZEROFILL +
    SMALLINT
    + INT
    + MEDIUMINT
    + SMALLINT UNSIGNED
    + SMALLINT UNSIGNED ZEROFILL +
    INT
    + BIGINT
    + INT UNSIGNED
    + INT UNSIGNED ZEROFILL
    + MEDIUMINT UNSIGNED
    + MEDIUMINT UNSIGNED ZEROFILL +
    BIGINT
    + BIGINT UNSIGNED
    + BIGINT UNSIGNED ZEROFILL
    + SERIAL +
    DECIMAL(20, 0)
    + FLOAT
    + FLOAT UNSIGNED
    + FLOAT UNSIGNED ZEROFILL +
    FLOAT
    + REAL
    + REAL UNSIGNED
    + REAL UNSIGNED ZEROFILL
    + DOUBLE
    + DOUBLE UNSIGNED
    + DOUBLE UNSIGNED ZEROFILL
    + DOUBLE PRECISION
    + DOUBLE PRECISION UNSIGNED
    + DOUBLE PRECISION UNSIGNED ZEROFILL +
    DOUBLE
    + NUMERIC(p, s)
    + NUMERIC(p, s) UNSIGNED
    + NUMERIC(p, s) UNSIGNED ZEROFILL
    + DECIMAL(p, s)
    + DECIMAL(p, s) UNSIGNED
    + DECIMAL(p, s) UNSIGNED ZEROFILL
    + FIXED(p, s)
    + FIXED(p, s) UNSIGNED
    + FIXED(p, s) UNSIGNED ZEROFILL
    + where p <= 38
    +
    DECIMAL(p, s)
    + NUMERIC(p, s)
    + NUMERIC(p, s) UNSIGNED
    + NUMERIC(p, s) UNSIGNED ZEROFILL
    + DECIMAL(p, s)
    + DECIMAL(p, s) UNSIGNED
    + DECIMAL(p, s) UNSIGNED ZEROFILL
    + FIXED(p, s)
    + FIXED(p, s) UNSIGNED
    + FIXED(p, s) UNSIGNED ZEROFILL
    + where 38 < p <= 65
    +
    STRING在 OceanBase MySQL 兼容模式中,十进制数据类型的精度高达 65,但在 Flink 中,十进制数据类型的精度仅限于 38。所以,如果定义精度大于 38 的十进制列,则应将其映射到字符串以避免精度损失。
    + BOOLEAN
    + TINYINT(1)
    + BIT(1) +
    BOOLEAN
    DATEDATE
    TIME [(p)]TIME [(p)]
    TIMESTAMP [(p)]
    + DATETIME [(p)] +
    TIMESTAMP [(p)] +
    + CHAR(n) + CHAR(n)
    + VARCHAR(n) + VARCHAR(n)
    + BIT(n) + BINARY(⌈n/8⌉)
    + BINARY(n) + BINARY(n)
    + VARBINARY(N) + VARBINARY(N)
    + TINYTEXT
    + TEXT
    + MEDIUMTEXT
    + LONGTEXT
    +
    STRING
    + TINYBLOB
    + BLOB
    + MEDIUMBLOB
    + LONGBLOB
    +
    BYTES目前,对于 BLOB 数据类型,仅支持长度不大于 2147483647(2**31-1)的 blob。
    + YEAR + INT
    + ENUM + STRING
    + JSON + STRING JSON 数据类型将在 Flink 中转换为 JSON 格式的字符串。
    + SET + ARRAY<STRING>因为 SET 数据类型是一个字符串对象,可以有零个或多个值 + 它应该始终映射到字符串数组。 +
    + GEOMETRY
    + POINT
    + LINESTRING
    + POLYGON
    + MULTIPOINT
    + MULTILINESTRING
    + MULTIPOLYGON
    + GEOMETRYCOLLECTION
    +
    + STRING + + 空间数据类型将转换为具有固定 Json 格式的字符串。 + 请参考 空间数据类型映射 章节了解更多详细信息。 +
    +
    - // enable checkpoint - env.enableCheckpointing(3000); +### 空间数据类型映射 - env.addSource(oceanBaseSource).print().setParallelism(1); - env.execute("Print OceanBase Snapshot + Change Events"); - } -} +除`GEOMETRYCOLLECTION`之外的空间数据类型都会转换为 Json 字符串,格式固定,如:
    +```json +{"srid": 0 , "type": "xxx", "coordinates": [0, 0]} ``` +字段`srid`标识定义几何体的 SRS,如果未指定 SRID,则 SRID 0 是新几何体值的默认值。 -### 可用的指标 - -指标系统能够帮助了解分片分发的进展, 下面列举出了支持的 Flink 指标 [Flink metrics](https://nightlies.apache.org/flink/flink-docs-master/docs/ops/metrics/): - -| Group | Name | Type | Description | -|------------------------|----------------------------|-------|----------------| -| namespace.schema.table | isSnapshotting | Gauge | 表是否在快照读取阶段 | -| namespace.schema.table | isStreamReading | Gauge | 表是否在增量读取阶段 | -| namespace.schema.table | numTablesSnapshotted | Gauge | 已经被快照读取完成的表的数量 | -| namespace.schema.table | numTablesRemaining | Gauge | 还没有被快照读取的表的数据 | -| namespace.schema.table | numSnapshotSplitsProcessed | Gauge | 正在处理的分片的数量 | -| namespace.schema.table | numSnapshotSplitsRemaining | Gauge | 还没有被处理的分片的数量 | -| namespace.schema.table | numSnapshotSplitsFinished | Gauge | 已经处理完成的分片的数据 | -| namespace.schema.table | snapshotStartTime | Gauge | 快照读取阶段开始的时间 | -| namespace.schema.table | snapshotEndTime | Gauge | 快照读取阶段结束的时间 | +字段`type`标识空间数据类型,例如`POINT`/`LINESTRING`/`POLYGON`。 -注意: -1. Group 名称是 `namespace.schema.table`,这里的 `namespace` 是实际的数据库名称, `schema` 是实际的 schema 名称, `table` 是实际的表名称。 -2. 对于 OceanBase,这里的 `namespace` 会被设置成默认值 "",也就是一个空字符串,Group 名称的格式会类似于 `test_database.test_table`。 - -数据类型映射 ----------------- +字段`coordinates`表示空间数据的`坐标`。 -### Mysql 模式 - -
    - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    OceanBase 数据类型Flink SQL 类型描述
    - BOOLEAN
    - TINYINT(1)
    - BIT(1) -
    BOOLEAN
    TINYINTTINYINT
    - SMALLINT
    - TINYINT UNSIGNED -
    SMALLINT
    - INT
    - MEDIUMINT
    - SMALLINT UNSIGNED -
    INT
    - BIGINT
    - INT UNSIGNED -
    BIGINT
    BIGINT UNSIGNEDDECIMAL(20, 0)
    - REAL
    - FLOAT -
    FLOAT
    - DOUBLE - DOUBLE
    - NUMERIC(p, s)
    - DECIMAL(p, s)
    - where p <= 38
    DECIMAL(p, s)
    - NUMERIC(p, s)
    - DECIMAL(p, s)
    - where 38 < p <=65
    -
    STRING - DECIMAL 等同于 NUMERIC。在 OceanBase 数据库中,DECIMAL 数据类型的精度最高为 65。
    - 但在 Flink 中,DECIMAL 的最高精度为 38。因此,
    - 如果你定义了一个精度大于 38 的 DECIMAL 列,你应当将其映射为 STRING,以避免精度损失。 -
    DATEDATE
    TIME [(p)]TIME [(p)]
    DATETIME [(p)]TIMESTAMP [(p)]
    TIMESTAMP [(p)]TIMESTAMP_LTZ [(p)]
    CHAR(n)CHAR(n)
    VARCHAR(n)VARCHAR(n)
    BIT(n)BINARY(⌈(n + 7) / 8⌉)
    BINARY(n)BINARY(n)
    VARBINARY(N)VARBINARY(N)
    - TINYTEXT
    - TEXT
    - MEDIUMTEXT
    - LONGTEXT -
    STRING
    - TINYBLOB
    - BLOB
    - MEDIUMBLOB
    - LONGBLOB -
    BYTES
    YEARINT
    ENUMSTRING
    SETARRAY<STRING> - 因为 OceanBase 的 SET 类型是用包含一个或多个值的字符串对象表示,
    - 所以映射到 Flink 时是一个字符串数组 -
    JSONSTRINGJSON 类型的数据在 Flink 中会转化为 JSON 格式的字符串
    -
    +对于`GEOMETRYCOLLECTION`,它将转换为 Json 字符串,格式固定,如:
    +```json +{"srid": 0 , "type": "GeometryCollection", "geometries": [{"type":"Point","coordinates":[10,10]}]} +``` -### Oracle 模式 +`Geometrics`字段是一个包含所有空间数据的数组。 +不同空间数据类型映射的示例如下:
    - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    OceanBase typeFlink SQL typeNOTE
    NUMBER(1)BOOLEAN
    NUMBER(p, s <= 0), p - s < 3 TINYINT
    NUMBER(p, s <= 0), p - s < 5 SMALLINT
    NUMBER(p, s <= 0), p - s < 10 INT
    NUMBER(p, s <= 0), p - s < 19 BIGINT
    NUMBER(p, s <= 0), 19 <=p - s <=38DECIMAL(p - s, 0)
    NUMBER(p, s > 0)DECIMAL(p, s)
    NUMBER(p, s <= 0), p - s> 38 STRING
    - FLOAT
    - BINARY_FLOAT -
    FLOAT
    BINARY_DOUBLEDOUBLE
    - DATE
    - TIMESTAMP [(p)] -
    TIMESTAMP [(p)]
    - CHAR(n)
    - NCHAR(n)
    - VARCHAR(n)
    - VARCHAR2(n)
    - NVARCHAR2(n)
    - CLOB
    -
    STRING
    - RAW
    - BLOB
    - ROWID -
    BYTES
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Spatial dataJson String converted in Flink
    POINT(1 1){"coordinates":[1,1],"type":"Point","srid":0}
    LINESTRING(3 0, 3 3, 3 5){"coordinates":[[3,0],[3,3],[3,5]],"type":"LineString","srid":0}
    POLYGON((1 1, 2 1, 2 2, 1 2, 1 1)){"coordinates":[[[1,1],[2,1],[2,2],[1,2],[1,1]]],"type":"Polygon","srid":0}
    MULTIPOINT((1 1),(2 2)){"coordinates":[[1,1],[2,2]],"type":"MultiPoint","srid":0}
    MultiLineString((1 1,2 2,3 3),(4 4,5 5)){"coordinates":[[[1,1],[2,2],[3,3]],[[4,4],[5,5]]],"type":"MultiLineString","srid":0}
    MULTIPOLYGON(((0 0, 10 0, 10 10, 0 10, 0 0)), ((5 5, 7 5, 7 7, 5 7, 5 5))){"coordinates":[[[[0,0],[10,0],[10,10],[0,10],[0,0]]],[[[5,5],[7,5],[7,7],[5,7],[5,5]]]],"type":"MultiPolygon","srid":0}
    GEOMETRYCOLLECTION(POINT(10 10), POINT(30 30), LINESTRING(15 15, 20 20)){"geometries":[{"type":"Point","coordinates":[10,10]},{"type":"Point","coordinates":[30,30]},{"type":"LineString","coordinates":[[15,15],[20,20]]}],"type":"GeometryCollection","srid":0}
    {{< top >}} diff --git a/docs/content/docs/connectors/flink-sources/oceanbase-cdc.md b/docs/content/docs/connectors/flink-sources/oceanbase-cdc.md index bc390ee369d..590d3e96875 100644 --- a/docs/content/docs/connectors/flink-sources/oceanbase-cdc.md +++ b/docs/content/docs/connectors/flink-sources/oceanbase-cdc.md @@ -35,12 +35,10 @@ Glossary: - *OceanBase CE*: OceanBase Community Edition. It's compatible with MySQL and has been open sourced at https://github.com/oceanbase/oceanbase. - *OceanBase EE*: OceanBase Enterprise Edition. It supports two compatibility modes: MySQL and Oracle. See https://en.oceanbase.com. - *OceanBase Cloud*: OceanBase Enterprise Edition on Cloud. See https://en.oceanbase.com/product/cloud. -- *Log Proxy CE*: OceanBase Log Proxy Community Edition (CDC mode). It's a proxy service which can fetch the commit log data of OceanBase CE. It has been open sourced at https://github.com/oceanbase/oblogproxy. -- *Log Proxy EE*: OceanBase Log Proxy Enterprise Edition (CDC mode). It's a proxy service which can fetch the commit log data of OceanBase EE. Limited support is available on OceanBase Cloud only, you can contact the provider support for more details. -- *Binlog Service CE*: OceanBase Binlog Service Community Edition. It is a solution of OceanBase CE that is compatible with the MySQL replication protocol. See the docs of Log Proxy CE (Binlog mode) for details. -- *Binlog Service EE*: OceanBase Binlog Service Enterprise Edition. It is a solution of OceanBase EE MySQL mode that is compatible with the MySQL replication protocol, and it's only available for users of Alibaba Cloud, see [User Guide](https://www.alibabacloud.com/help/en/apsaradb-for-oceanbase/latest/binlog-overview). -- *MySQL Driver*: `mysql-connector-java` which can be used with OceanBase CE and OceanBase EE MySQL mode. -- *OceanBase Driver*: The Jdbc driver for OceanBase, which supports both MySQL mode and Oracle mode of all OceanBase versions. It's open sourced at https://github.com/oceanbase/obconnector-j. +- *Binlog Service CE*: OceanBase Binlog Service Community Edition. It is a solution of OceanBase CE that is compatible with the MySQL replication protocol. See the docs of [OceanBase Binlog Service](https://en.oceanbase.com/docs/common-ocp-10000000002168919) for details. +- *Binlog Service EE*: OceanBase Binlog Service Enterprise Edition. It is a solution of OceanBase EE MySQL compatible mode that is compatible with the MySQL replication protocol, and it's only available for users of Alibaba Cloud, see [User Guide](https://www.alibabacloud.com/help/en/apsaradb-for-oceanbase/latest/binlog-overview). +- *MySQL Driver*: `mysql-connector-java` which can be used with OceanBase CE and OceanBase EE MySQL compatible mode. +- *OceanBase Driver*: The JDBC driver for OceanBase, which supports both MySQL compatible mode and Oracle compatible mode of all OceanBase versions. It's open sourced at https://github.com/oceanbase/obconnector-j. CDC Source Solutions for OceanBase: @@ -58,11 +56,10 @@ CDC Source Solutions for OceanBase: OceanBase CE - MySQL Driver: 5.1.4x, 8.0.x
    - OceanBase Driver: 2.4.x + MySQL Driver: 8.0.x
    OceanBase CDC Connector - Log Proxy CE + Binlog Service CE MySQL Driver: 8.0.x @@ -70,13 +67,12 @@ CDC Source Solutions for OceanBase: Binlog Service CE - OceanBase EE (MySQL Mode) + OceanBase EE (MySQL Compatible Mode) - MySQL Driver: 5.1.4x, 8.0.x
    - OceanBase Driver: 2.4.x + MySQL Driver: 8.0.x
    OceanBase CDC Connector - Log Proxy EE + Binlog Service EE MySQL Driver: 8.0.x @@ -84,16 +80,20 @@ CDC Source Solutions for OceanBase: Binlog Service EE - OceanBase EE (Oracle Mode) - OceanBase Driver: 2.4.x - OceanBase CDC Connector - Log Proxy EE (CDC Mode) + OceanBase EE (Oracle Compatible Mode) + + The incremental data subscription service for OceanBase Oracle Compatibility Mode is not currently supported. Please contact Enterprise Support for assistance. + -Note: For users of OceanBase CE or OceanBase EE MySQL Mode, we recommend that you follow the [MySQL CDC documentation](mysql-cdc.md) to use the MySQL CDC source connector with the Binlog service. +**Important Notes**: The oceanbase-cdc connector has undergone significant changes starting from version 3.5: +- The previous connector implementation based on the OceanBase Log Proxy service has been officially removed. The current version now only supports connecting to the OceanBase Binlog Service. +- The current oceanbase-cdc connector is rebuilt based on the mysql-cdc connector, offering enhanced compatibility with the OceanBase Binlog Service, including critical bug fixes. Upgrading to this version is strongly recommended. +- Since the OceanBase Binlog Service is compatible with MySQL replication protocols, you may still use the MySQL CDC connector to connect to the OceanBase Binlog Service. +- The incremental data subscription service for OceanBase Oracle Compatibility Mode is not currently supported. Please contact Enterprise Support for assistance. Dependencies ------------ @@ -128,303 +128,368 @@ Since the licenses of MySQL Driver and OceanBase Driver are incompatible with Fl Used for connecting to MySQL tenant of OceanBase. - - - com.oceanbase:oceanbase-client:2.4.9 - Used for connecting to MySQL or Oracle tenant of OceanBase. - - -Setup OceanBase and LogProxy Server +Deploy OceanBase database and Binlog service ---------------------- -1. Set up the OceanBase cluster following the [doc](https://github.com/oceanbase/oceanbase#quick-start). - -2. Create a user with password in `sys` tenant, this user is used in OceanBase LogProxy. +1. Deploy the OceanBase database according to the [document](https://en.oceanbase.com/docs/common-oceanbase-database-10000000001970931). +2. Deploy the OceanBase Binlog service according to the [document](https://en.oceanbase.com/docs/common-ocp-10000000002168919). - ```shell - mysql -h${host} -P${port} -uroot - - mysql> SHOW TENANT; - mysql> CREATE USER ${sys_username} IDENTIFIED BY '${sys_password}'; - mysql> GRANT ALL PRIVILEGES ON *.* TO ${sys_username} WITH GRANT OPTION; - ``` - -3. Create a user in the tenant you want to monitor, this is used to read data for snapshot and change event. - -4. For users of OceanBase Community Edition, you need to get the `rootserver-list`. You can use the following command to get the value: - - ```shell - mysql> show parameters like 'rootservice_list'; - ``` - - For users of OceanBase Enterprise Edition, you need to get the `config-url`. You can use the following command to get the value: - - ```shell - mysql> show parameters like 'obconfig_url'; - ``` +Usage Documentation +---------------- -5. Setup OceanBase LogProxy. For users of OceanBase Community Edition, you can follow the [docs (Chinese)](https://www.oceanbase.com/docs/community-oblogproxy-doc-1000000000531984). +**Notes**: +- The current version of the **oceanbase-cdc connector** is built on top of the **mysql-cdc connector**, with **only minor internal implementation changes**. + - **External interfaces** and **configuration parameters** are **fully consistent** with those of the mysql-cdc connector. + - For detailed usage instructions, refer to the [**MySQL CDC Usage Documentation**](mysql-cdc.md). How to create a OceanBase CDC table ---------------- -The OceanBase CDC table can be defined as following: - ```sql -- checkpoint every 3000 milliseconds -Flink SQL> SET 'execution.checkpointing.interval' = '3s'; +Flink SQL> SET 'execution.checkpointing.interval' = '3s'; -- register a OceanBase table 'orders' in Flink SQL Flink SQL> CREATE TABLE orders ( - order_id INT, - order_date TIMESTAMP(0), - customer_name STRING, - price DECIMAL(10, 5), - product_id INT, - order_status BOOLEAN, - PRIMARY KEY (order_id) NOT ENFORCED -) WITH ( - 'connector' = 'oceanbase-cdc', - 'scan.startup.mode' = 'initial', - 'username' = 'user@test_tenant#cluster_name', - 'password' = 'pswd', - 'tenant-name' = 'test_tenant', - 'database-name' = '^test_db$', - 'table-name' = '^orders$', - 'hostname' = '127.0.0.1', - 'port' = '2881', - 'rootserver-list' = '127.0.0.1:2882:2881', - 'logproxy.host' = '127.0.0.1', - 'logproxy.port' = '2983', - 'working-mode' = 'memory' -); - + order_id INT, + order_date TIMESTAMP(0), + customer_name STRING, + price DECIMAL(10, 5), + product_id INT, + order_status BOOLEAN, + PRIMARY KEY(order_id) NOT ENFORCED + ) WITH ( + 'connector' = 'oceanbase-cdc', + 'hostname' = 'localhost', + 'port' = '2881', + 'username' = 'root', + 'password' = '123456', + 'database-name' = 'mydb', + 'table-name' = 'orders'); + -- read snapshot and binlogs from orders table Flink SQL> SELECT * FROM orders; ``` -If you want to use OceanBase Oracle mode, you need to add the OceanBase jdbc jar file to Flink and set up the enterprise edition of oblogproxy, then you can create a table in Flink as following: - -```sql -Flink SQL> CREATE TABLE orders ( - order_id INT, - order_date TIMESTAMP(0), - customer_name STRING, - price DECIMAL(10, 5), - product_id INT, - order_status BOOLEAN, - PRIMARY KEY (order_id) NOT ENFORCED -) WITH ( - 'connector' = 'oceanbase-cdc', - 'scan.startup.mode' = 'initial', - 'username' = 'user@test_tenant#cluster_name', - 'password' = 'pswd', - 'tenant-name' = 'test_tenant', - 'database-name' = '^test_db$', - 'table-name' = '^orders$', - 'hostname' = '127.0.0.1', - 'port' = '2881', - 'compatible-mode' = 'oracle', - 'jdbc.driver' = 'com.oceanbase.jdbc.Driver', - 'config-url' = 'http://127.0.0.1:8080/services?Action=ObRootServiceInfo&User_ID=xxx&UID=xxx&ObRegion=xxx', - 'logproxy.host' = '127.0.0.1', - 'logproxy.port' = '2983', - 'working-mode' = 'memory' -); -``` - -You can also try the quickstart tutorial that sync data from OceanBase to Elasticsearch, please refer [Flink CDC Tutorial]({{< ref "docs/connectors/flink-sources/tutorials/oceanbase-tutorial" >}}) for more information. - - Connector Options ---------------- -The OceanBase CDC Connector contains some options for both sql and stream api as the following sheet. - -*Note*: The connector supports two ways to specify the table list to listen to, and will get the union of the results when both way are used at the same time. -1. Use `database-name` and `table-name` to match database and table names in regex. -2. Use `table-list` to match the exact value of database and table names. -
    - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    OptionRequiredDefaultTypeDescription
    connectorrequired(none)StringSpecify what connector to use, here should be 'oceanbase-cdc'.
    scan.startup.modeoptionalinitialStringSpecify the startup mode for OceanBase CDC consumer, valid enumerations are - 'initial','latest-offset','timestamp' or 'snapshot'. -
    scan.startup.timestampoptional(none)LongTimestamp in seconds of the start point, only used for 'timestamp' startup mode.
    usernamerequired(none)StringUsername to be used when connecting to OceanBase.
    passwordrequired(none)StringPassword to be used when connecting to OceanBase.
    tenant-nameoptional(none)StringTenant name of OceanBase to monitor, should be exact value. Required when 'scan.startup.mode' is not 'snapshot'.
    database-nameoptional(none)StringDatabase name of OceanBase to monitor, should be regular expression.
    table-nameoptional(none)StringTable name of OceanBase to monitor, should be regular expression.
    table-listoptional(none)StringList of full names of tables, separated by commas, e.g. "db1.table1, db2.table2".
    hostnamerequired(none)StringIP address or hostname of the OceanBase database server or OceanBase Proxy server.
    portrequired(none)IntegerInteger port number to connect to OceanBase. It can be the SQL port of OceanBase server, which is 2881 by default, or the port of OceanBase proxy service, which is 2883 by default.
    connect.timeoutoptional30sDurationThe maximum time that the connector should wait after trying to connect to the OceanBase database server before timing out.
    server-time-zoneoptional+00:00StringThe session timezone which controls how temporal types are converted to STRING in OceanBase. Can be UTC offset in format "±hh:mm", or named time zones if the time zone information tables in the mysql database have been created and populated.
    logproxy.hostoptional(none)StringHostname or IP address of OceanBase log proxy service. Required when 'scan.startup.mode' is not 'snapshot'.
    logproxy.portoptional(none)IntegerPort number of OceanBase log proxy service. Required when 'scan.startup.mode' is not 'snapshot'.
    logproxy.client.idoptionalBy rule.StringId of a log proxy client connection, will be in format {flink_ip}_{process_id}_{timestamp}_{thread_id}_{tenant} by default.
    rootserver-listoptional(none)StringThe semicolon-separated list of OceanBase root servers in format `ip:rpc_port:sql_port`, required for OceanBase CE.
    config-urloptional(none)StringThe url to get the server info from the config server, required for OceanBase EE.
    working-modeoptionalstorageStringWorking mode of `obcdc` in LogProxy, can be `storage` or `memory`.
    compatible-modeoptionalmysqlStringCompatible mode of OceanBase, can be `mysql` or `oracle`.
    jdbc.driveroptionalcom.mysql.cj.jdbc.DriverStringJDBC driver class for snapshot reading.
    jdbc.properties.*optional(none)StringOption to pass custom JDBC URL properties. User can pass custom properties like 'jdbc.properties.useSSL' = 'false'.
    obcdc.properties.*optional(none)StringOption to pass custom configurations to the libobcdc, eg: 'obcdc.properties.sort_trans_participants' = '1'. Please refer to obcdc parameters for more details.
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    OptionRequiredDefaultTypeDescription
    connectorrequired(none)StringSpecify what connector to use, here should be 'oceanbase-cdc'.
    hostnamerequired(none)StringIP address or hostname of the OceanBase database server.
    usernamerequired(none)StringName of the OceanBase database to use when connecting to the OceanBase database server.
    passwordrequired(none)StringPassword to use when connecting to the OceanBase database server.
    database-namerequired(none)StringDatabase name of the OceanBase server to monitor. The database-name also supports regular expressions to monitor multiple tables matches the regular expression.
    table-namerequired(none)String + Table name of the OceanBase database to monitor. The table-name also supports regular expressions to monitor multiple tables that satisfy the regular expressions. Note: When the OceanBase CDC connector regularly matches the table name, it will concat the database-name and table-name filled in by the user through the string `\\.` to form a full-path regular expression, and then use the regular expression to match the fully qualified name of the table in the OceanBase database. +
    portoptional2881IntegerInteger port number of the OceanBase database server.
    server-idoptional(none)StringA numeric ID or a numeric ID range of this database client, The numeric ID syntax is like '5400', + the numeric ID range syntax is like '5400-5408', The numeric ID range syntax is recommended when 'scan.incremental.snapshot.enabled' enabled. + Every ID must be unique across all currently-running database processes in the OceanBase cluster. This connector joins the OceanBase cluster + as another server (with this unique ID) so it can read the binlog. By default, a random number is generated between 5400 and 6400, + though we recommend setting an explicit value. +
    scan.incremental.snapshot.enabledoptionaltrueBooleanIncremental snapshot is a new mechanism to read snapshot of a table. Compared to the old snapshot mechanism, + the incremental snapshot has many advantages, including: + (1) source can be parallel during snapshot reading, + (2) source can perform checkpoints in the chunk granularity during snapshot reading, + (3) source doesn't need to acquire global read lock (FLUSH TABLES WITH READ LOCK) before snapshot reading. + If you would like the source run in parallel, each parallel reader should have an unique server id, so + the 'server-id' must be a range like '5400-6400', and the range must be larger than the parallelism. + Please see Incremental Snapshot Readingsection for more detailed information. +
    scan.incremental.snapshot.chunk.sizeoptional8096IntegerThe chunk size (number of rows) of table snapshot, captured tables are split into multiple chunks when read the snapshot of table.
    scan.snapshot.fetch.sizeoptional1024IntegerThe maximum fetch size for per poll when read table snapshot.
    scan.incremental.snapshot.chunk.key-columnoptional(none)StringThe chunk key of table snapshot, captured tables are split into multiple chunks by a chunk key when read the snapshot of table. + By default, the chunk key is the first column of the primary key. A column that is not part of the primary key can be used as a chunk key, but this may lead to slower query performance. +
    + Warning: Using a non-primary key column as a chunk key may lead to data inconsistencies. Please see Warning for details. +
    scan.startup.modeoptionalinitialStringOptional startup mode for OceanBase CDC consumer, valid enumerations are "initial", "earliest-offset", "latest-offset", "specific-offset", "timestamp" and "snapshot". + Please see Startup Reading Position section for more detailed information.
    scan.startup.specific-offset.fileoptional(none)StringOptional binlog file name used in case of "specific-offset" startup mode
    scan.startup.specific-offset.posoptional(none)LongOptional binlog file position used in case of "specific-offset" startup mode
    scan.startup.specific-offset.gtid-setoptional(none)StringOptional GTID set used in case of "specific-offset" startup mode
    scan.startup.timestamp-millisoptional(none)LongOptional millisecond timestamp used in case of "timestamp" startup mode.
    scan.startup.specific-offset.skip-eventsoptional(none)LongOptional number of events to skip after the specific starting offset
    scan.startup.specific-offset.skip-rowsoptional(none)LongOptional number of rows to skip after the specific starting offset
    server-time-zoneoptional(none)StringThe session time zone in database server, e.g. "Asia/Shanghai". + It controls how the TIMESTAMP type in OceanBase converted to STRING. + See more here. + If not set, then ZoneId.systemDefault() is used to determine the server time zone. +
    debezium.min.row.count.to.stream.resultoptional1000Integer +During a snapshot operation, the connector will query each included table to produce a read event for all rows in that table. This parameter determines whether the OceanBase connection will pull all results for a table into memory (which is fast but requires large amounts of memory), or whether the results will instead be streamed (can be slower, but will work for very large tables). The value specifies the minimum number of rows a table must contain before the connector will stream results, and defaults to 1,000. Set this parameter to '0' to skip all table size checks and always stream all results during a snapshot.
    connect.timeoutoptional30sDurationThe maximum time that the connector should wait after trying to connect to the OceanBase database server before timing out. + This value cannot be less than 250ms.
    connect.max-retriesoptional3IntegerThe max retry times that the connector should retry to build OceanBase database server connection.
    connection.pool.sizeoptional20IntegerThe connection pool size.
    jdbc.properties.*optional20StringOption to pass custom JDBC URL properties. User can pass custom properties like 'jdbc.properties.useSSL' = 'false'.
    heartbeat.intervaloptional30sDurationThe interval of sending heartbeat event for tracing the latest available binlog offsets.
    debezium.*optional(none)StringPass-through Debezium's properties to Debezium Embedded Engine which is used to capture data changes from OceanBase server. + For example: 'debezium.snapshot.mode' = 'never'. + See more about the Debezium's MySQL Connector properties
    scan.incremental.close-idle-reader.enabledoptionalfalseBooleanWhether to close idle readers at the end of the snapshot phase.
    + The flink version is required to be greater than or equal to 1.14 when 'execution.checkpointing.checkpoints-after-tasks-finish.enabled' is set to true.
    + If the flink version is greater than or equal to 1.15, the default value of 'execution.checkpointing.checkpoints-after-tasks-finish.enabled' has been changed to true, + so it does not need to be explicitly configured 'execution.checkpointing.checkpoints-after-tasks-finish.enabled' = 'true' +
    debezium.binary.handling.modeoptional(none)Stringdebezium.binary.handling.mode can be set to one of the following values: + none: No processing is performed, and the binary data type is transmitted as a byte array (byte array). + base64: The binary data type is converted to a Base64-encoded string and transmitted. + hex: The binary data type is converted to a hexadecimal string and transmitted. + The default value is none. Depending on your requirements and data types, you can choose the appropriate processing mode. If your database contains a large number of binary data types, it is recommended to use base64 or hex mode to make it easier to handle during transmission.
    scan.parse.online.schema.changes.enabledoptionalfalseBoolean + Whether to parse "online" schema changes generated by gh-ost or pt-osc. + Schema change events are applied to a "shadow" table and then swapped with the original table later. +
    + This is an experimental feature, and subject to change in the future. +
    use.legacy.json.formatoptionaltrueBooleanWhether to use legacy JSON format to cast JSON type data in binlog.
    + It determines whether to use the legacy JSON format when retrieving JSON type data in binlog. + If the user configures 'use.legacy.json.format' = 'true', whitespace before values and after commas in the JSON type data is removed. For example, + JSON type data {"key1": "value1", "key2": "value2"} in binlog would be converted to {"key1":"value1","key2":"value2"}. + When 'use.legacy.json.format' = 'false', the data would be converted to {"key1": "value1", "key2": "value2"}, with whitespace before values and after commas preserved. +
    scan.incremental.snapshot.unbounded-chunk-first.enabledoptionalfalseBoolean + Whether to assign the unbounded chunks first during snapshot reading phase.
    + This might help reduce the risk of the TaskManager experiencing an out-of-memory (OOM) error when taking a snapshot of the largest unbounded chunk.
    + Experimental option, defaults to false. +
    scan.read-changelog-as-append-only.enabledoptionalfalseBoolean + Whether to convert the changelog stream to an append-only stream.
    + This feature is only used in special scenarios where you need to save upstream table deletion messages. For example, in a logical deletion scenario, users are not allowed to physically delete downstream messages. In this case, this feature is used in conjunction with the row_kind metadata field. Therefore, the downstream can save all detailed data at first, and then use the row_kind field to determine whether to perform logical deletion.
    + The option values are as follows:
    +
  • true: All types of messages (including INSERT, DELETE, UPDATE_BEFORE, and UPDATE_AFTER) will be converted into INSERT messages.
  • +
  • false (default): All types of messages are sent as is.
  • +
    scan.incremental.snapshot.backfill.skipoptionalfalseBoolean + Whether to skip backfill in snapshot reading phase.
    + If backfill is skipped, changes on captured tables during snapshot phase will be consumed later in change log reading phase instead of being merged into the snapshot.
    + WARNING: Skipping backfill might lead to data inconsistency because some change log events happened within the snapshot phase might be replayed (only at-least-once semantic is promised). + For example updating an already updated value in snapshot, or deleting an already deleted entry in snapshot. These replayed change log events should be handled specially. +
    Available Metadata @@ -433,434 +498,429 @@ Available Metadata The following format metadata can be exposed as read-only (VIRTUAL) columns in a table definition. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    KeyDataTypeDescription
    tenant_nameSTRINGName of the tenant that contains the row.
    database_nameSTRINGName of the database that contains the row.
    schema_nameSTRINGName of the schema that contains the row.
    table_nameSTRING NOT NULLName of the table that contains the row.
    op_tsTIMESTAMP_LTZ(3) NOT NULLIt indicates the time that the change was made in the database.
    - If the record is read from snapshot of the table instead of the change stream, the value is always 0.
    KeyDataTypeDescription
    table_nameSTRING NOT NULLName of the table that contain the row.
    database_nameSTRING NOT NULLName of the database that contain the row.
    op_tsTIMESTAMP_LTZ(3) NOT NULLIt indicates the time that the change was made in the database.
    If the record is read from snapshot of the table instead of the binlog, the value is always 0.
    row_kindSTRING NOT NULLIt indicates the row kind of the changelog,Note: The downstream SQL operator may fail to compare due to this new added column when processing the row retraction if +the source operator chooses to output the 'row_kind' column for each record. It is recommended to use this metadata column only in simple synchronization jobs. +
    '+I' means INSERT message, '-D' means DELETE message, '-U' means UPDATE_BEFORE message and '+U' means UPDATE_AFTER message.
    The extended CREATE TABLE example demonstrates the syntax for exposing these metadata fields: ```sql -CREATE TABLE products ( - tenant_name STRING METADATA FROM 'tenant_name' VIRTUAL, - db_name STRING METADATA FROM 'database_name' VIRTUAL, - table_name STRING METADATA FROM 'table_name' VIRTUAL, - operation_ts TIMESTAMP_LTZ(3) METADATA FROM 'op_ts' VIRTUAL, - order_id INT, - order_date TIMESTAMP(0), +CREATE TABLE products +( + db_name STRING METADATA FROM 'database_name' VIRTUAL, + table_name STRING METADATA FROM 'table_name' VIRTUAL, + operation_ts TIMESTAMP_LTZ(3) METADATA FROM 'op_ts' VIRTUAL, + operation STRING METADATA FROM 'row_kind' VIRTUAL, + order_id INT, + order_date TIMESTAMP(0), customer_name STRING, - price DECIMAL(10, 5), - product_id INT, - order_status BOOLEAN, - PRIMARY KEY(order_id) NOT ENFORCED + price DECIMAL(10, 5), + product_id INT, + order_status BOOLEAN, + PRIMARY KEY (order_id) NOT ENFORCED ) WITH ( - 'connector' = 'oceanbase-cdc', - 'scan.startup.mode' = 'initial', - 'username' = 'user@test_tenant', - 'password' = 'pswd', - 'tenant-name' = 'test_tenant', - 'database-name' = '^test_db$', - 'table-name' = '^orders$', - 'hostname' = '127.0.0.1', - 'port' = '2881', - 'rootserver-list' = '127.0.0.1:2882:2881', - 'logproxy.host' = '127.0.0.1', - 'logproxy.port' = '2983', - 'working-mode' = 'memory' -); + 'connector' = 'oceanbase-cdc', + 'hostname' = 'localhost', + 'port' = '2881', + 'username' = 'root', + 'password' = '123456', + 'database-name' = 'mydb', + 'table-name' = 'orders' + ); ``` -Features --------- - -### At-Least-Once Processing - -The OceanBase CDC connector is a Flink Source connector which will read database snapshot first and then continues to read change events with **at-least-once processing**. - -OceanBase is a kind of distributed database whose log files are distributed on different servers. As there is no position information like MySQL binlog offset, we can only use timestamp as the position mark. In order to ensure the completeness of reading data, `liboblog` (a C++ library to read OceanBase log record) might read some log data before the given timestamp. So in this way we may read duplicate data whose timestamp is around the start point, and only 'at-least-once' can be guaranteed. - -### Startup Reading Position - -The config option `scan.startup.mode` specifies the startup mode for OceanBase CDC consumer. The valid enumerations are: - -- `initial`: Performs an initial snapshot on the monitored table upon first startup, and continue to read the latest commit log. -- `latest-offset`: Never to perform snapshot on the monitored table upon first startup and just read the latest commit log since the connector is started. -- `timestamp`: Never to perform snapshot on the monitored table upon first startup and just read the commit log from the given `scan.startup.timestamp`. -- `snapshot`: Only perform snapshot on the monitored table. - -### Consume Commit Log +The extended CREATE TABLE example demonstrates the usage of regex to match multi-tables: -The OceanBase CDC Connector using [oblogclient](https://github.com/oceanbase/oblogclient) to consume commit log from OceanBase LogProxy. - -### DataStream Source - -The OceanBase CDC connector can also be a DataStream source. You can create a SourceFunction as the following shows: - -```java -import org.apache.flink.cdc.connectors.base.options.StartupOptions; -import org.apache.flink.cdc.connectors.oceanbase.OceanBaseSource; -import org.apache.flink.cdc.debezium.JsonDebeziumDeserializationSchema; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.SourceFunction; - -public class OceanBaseSourceExample { - public static void main(String[] args) throws Exception { - SourceFunction oceanBaseSource = - OceanBaseSource.builder() - .startupOptions(StartupOptions.initial()) - .hostname("127.0.0.1") - .port(2881) - .username("user@test_tenant") - .password("pswd") - .compatibleMode("mysql") - .jdbcDriver("com.mysql.cj.jdbc.Driver") - .tenantName("test_tenant") - .databaseName("^test_db$") - .tableName("^test_table$") - .logProxyHost("127.0.0.1") - .logProxyPort(2983) - .rsList("127.0.0.1:2882:2881") - .serverTimeZone("+08:00") - .deserializer(new JsonDebeziumDeserializationSchema()) - .build(); - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - // enable checkpoint - env.enableCheckpointing(3000); - - env.addSource(oceanBaseSource).print().setParallelism(1); - env.execute("Print OceanBase Snapshot + Change Events"); - } -} +```sql +CREATE TABLE products +( + db_name STRING METADATA FROM 'database_name' VIRTUAL, + table_name STRING METADATA FROM 'table_name' VIRTUAL, + operation_ts TIMESTAMP_LTZ(3) METADATA FROM 'op_ts' VIRTUAL, + operation STRING METADATA FROM 'row_kind' VIRTUAL, + order_id INT, + order_date TIMESTAMP(0), + customer_name STRING, + price DECIMAL(10, 5), + product_id INT, + order_status BOOLEAN, + PRIMARY KEY (order_id) NOT ENFORCED +) WITH ( + 'connector' = 'oceanbase-cdc', + 'hostname' = 'localhost', + 'port' = '2881', + 'username' = 'root', + 'password' = '123456', + 'database-name' = '(^(test).*|^(tpc).*|txc|.*[p$]|t{2})', + 'table-name' = '(t[5-8]|tt)' + ); ``` + + + + + + + + + + + + + + + + + + + + + + + + + +
    exampleexpressiondescription
    prefix match^(test).*This matches the database name or table name starts with prefix of test, e.g test1、test2.
    suffix match.*[p$]This matches the database name or table name ends with suffix of p, e.g cdcp、edcp.
    specific matchtxcThis matches the database name or table name according to a specific name, e.g txc.
    -### Available Source metrics - -Metrics can help understand the progress of assignments, and the following are the supported [Flink metrics](https://nightlies.apache.org/flink/flink-docs-master/docs/ops/metrics/): - -| Group | Name | Type | Description | -|------------------------|----------------------------|-------|-----------------------------------------------------| -| namespace.schema.table | isSnapshotting | Gauge | Weather the table is snapshotting or not | -| namespace.schema.table | isStreamReading | Gauge | Weather the table is stream reading or not | -| namespace.schema.table | numTablesSnapshotted | Gauge | The number of tables that have been snapshotted | -| namespace.schema.table | numTablesRemaining | Gauge | The number of tables that have not been snapshotted | -| namespace.schema.table | numSnapshotSplitsProcessed | Gauge | The number of splits that is being processed | -| namespace.schema.table | numSnapshotSplitsRemaining | Gauge | The number of splits that have not been processed | -| namespace.schema.table | numSnapshotSplitsFinished | Gauge | The number of splits that have been processed | -| namespace.schema.table | snapshotStartTime | Gauge | The time when the snapshot started | -| namespace.schema.table | snapshotEndTime | Gauge | The time when the snapshot ended | +It will use `database-name\\.table-name` as a pattern to match tables, as above examples using pattern `(^(test).*|^(tpc).*|txc|.*[p$]|t{2})\\.(t[5-8]|tt)` matches txc.tt、test2.test5. -Notice: -1. The group name is `namespace.schema.table`, where `namespace` is the actual database name, `schema` is the actual schema name, and `table` is the actual table name. -2. For OceanBase, the `namespace` will be set to the default value "", and the group name will be like `test_database.test_table`. Data Type Mapping ---------------- -### Mysql Mode -
    - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    OceanBase typeFlink SQL typeNOTE
    BOOLEAN
    - TINYINT(1)
    - BIT(1)
    BOOLEAN
    TINYINTTINYINT
    - SMALLINT
    - TINYINT UNSIGNED
    SMALLINT
    - INT
    - MEDIUMINT
    - SMALLINT UNSIGNED
    INT
    - BIGINT
    - INT UNSIGNED
    BIGINT
    BIGINT UNSIGNEDDECIMAL(20, 0)
    - REAL
    - FLOAT
    -
    FLOAT
    - DOUBLE - DOUBLE
    - NUMERIC(p, s)
    - DECIMAL(p, s)
    - where p <= 38
    -
    DECIMAL(p, s)
    - NUMERIC(p, s)
    - DECIMAL(p, s)
    - where 38 < p <=65
    -
    STRINGDECIMAL is equivalent to NUMERIC. The precision for DECIMAL data type is up to 65 in OceanBase, but - the precision for DECIMAL is limited to 38 in Flink. - So if you define a decimal column whose precision is greater than 38, you should map it to STRING to - avoid precision loss.
    DATEDATE
    TIME [(p)]TIME [(p)]
    DATETIME [(p)]TIMESTAMP [(p)]
    TIMESTAMP [(p)]TIMESTAMP_LTZ [(p)]
    CHAR(n)CHAR(n)
    VARCHAR(n)VARCHAR(n)
    BIT(n)BINARY(⌈(n + 7) / 8⌉)
    BINARY(n)BINARY(n)
    VARBINARY(N)VARBINARY(N)
    - TINYTEXT
    - TEXT
    - MEDIUMTEXT
    - LONGTEXT
    -
    STRING
    - TINYBLOB
    - BLOB
    - MEDIUMBLOB
    - LONGBLOB
    -
    BYTES
    YEARINT
    ENUMSTRING
    SETARRAY<STRING>As the SET data type in OceanBase is a string object that can have zero or more values, it should always be mapped to an array of string
    JSONSTRINGThe JSON data type will be converted into STRING with JSON format in Flink.
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    OceanBase typeFlink SQL typeNOTE
    TINYINTTINYINT
    + SMALLINT
    + TINYINT UNSIGNED
    + TINYINT UNSIGNED ZEROFILL +
    SMALLINT
    + INT
    + MEDIUMINT
    + SMALLINT UNSIGNED
    + SMALLINT UNSIGNED ZEROFILL +
    INT
    + BIGINT
    + INT UNSIGNED
    + INT UNSIGNED ZEROFILL
    + MEDIUMINT UNSIGNED
    + MEDIUMINT UNSIGNED ZEROFILL +
    BIGINT
    + BIGINT UNSIGNED
    + BIGINT UNSIGNED ZEROFILL
    + SERIAL +
    DECIMAL(20, 0)
    + FLOAT
    + FLOAT UNSIGNED
    + FLOAT UNSIGNED ZEROFILL +
    FLOAT
    + REAL
    + REAL UNSIGNED
    + REAL UNSIGNED ZEROFILL
    + DOUBLE
    + DOUBLE UNSIGNED
    + DOUBLE UNSIGNED ZEROFILL
    + DOUBLE PRECISION
    + DOUBLE PRECISION UNSIGNED
    + DOUBLE PRECISION UNSIGNED ZEROFILL +
    DOUBLE
    + NUMERIC(p, s)
    + NUMERIC(p, s) UNSIGNED
    + NUMERIC(p, s) UNSIGNED ZEROFILL
    + DECIMAL(p, s)
    + DECIMAL(p, s) UNSIGNED
    + DECIMAL(p, s) UNSIGNED ZEROFILL
    + FIXED(p, s)
    + FIXED(p, s) UNSIGNED
    + FIXED(p, s) UNSIGNED ZEROFILL
    + where p <= 38
    +
    DECIMAL(p, s)
    + NUMERIC(p, s)
    + NUMERIC(p, s) UNSIGNED
    + NUMERIC(p, s) UNSIGNED ZEROFILL
    + DECIMAL(p, s)
    + DECIMAL(p, s) UNSIGNED
    + DECIMAL(p, s) UNSIGNED ZEROFILL
    + FIXED(p, s)
    + FIXED(p, s) UNSIGNED
    + FIXED(p, s) UNSIGNED ZEROFILL
    + where 38 < p <= 65
    +
    STRINGThe precision for DECIMAL data type is up to 65 in OceanBase compatible mode, but the precision for DECIMAL is limited to 38 in Flink. + So if you define a decimal column whose precision is greater than 38, you should map it to STRING to avoid precision loss.
    + BOOLEAN
    + TINYINT(1)
    + BIT(1) +
    BOOLEAN
    DATEDATE
    TIME [(p)]TIME [(p)]
    TIMESTAMP [(p)]
    + DATETIME [(p)] +
    TIMESTAMP [(p)] +
    + CHAR(n) + CHAR(n)
    + VARCHAR(n) + VARCHAR(n)
    + BIT(n) + BINARY(⌈(n + 7) / 8⌉)
    + BINARY(n) + BINARY(n)
    + VARBINARY(N) + VARBINARY(N)
    + TINYTEXT
    + TEXT
    + MEDIUMTEXT
    + LONGTEXT
    +
    STRING
    + TINYBLOB
    + BLOB
    + MEDIUMBLOB
    + LONGBLOB
    +
    BYTESCurrently, for BLOB data type, only the blob whose length isn't greater than 2,147,483,647(2 ** 31 - 1) is supported.
    + YEAR + INT
    + ENUM + STRING
    + JSON + STRINGThe JSON data type will be converted into STRING with JSON format in Flink.
    + SET + ARRAY<STRING>As the SET data type is a string object that can have zero or more values, + it should always be mapped to an array of string +
    + GEOMETRY
    + POINT
    + LINESTRING
    + POLYGON
    + MULTIPOINT
    + MULTILINESTRING
    + MULTIPOLYGON
    + GEOMETRYCOLLECTION
    +
    + STRING + + The spatial data types will be converted into STRING with a fixed Json format. + Please see Spatial Data Types Mapping section for more detailed information. +
    -### Oracle Mode +### Spatial Data Types Mapping +The spatial data types except for `GEOMETRYCOLLECTION` will be converted into Json String with a fixed format like:
    +```json +{"srid": 0 , "type": "xxx", "coordinates": [0, 0]} +``` +The field `srid` identifies the SRS in which the geometry is defined, SRID 0 is the default for new geometry values if no SRID is specified. + +The field `type` identifies the spatial data type, such as `POINT`/`LINESTRING`/`POLYGON`. + +The field `coordinates` represents the `coordinates` of the spatial data. + +For `GEOMETRYCOLLECTION`, it will be converted into Json String with a fixed format like:
    +```json +{"srid": 0 , "type": "GeometryCollection", "geometries": [{"type":"Point","coordinates":[10,10]}]} +``` + +The field `geometries` is an array contains all spatial data. +The example for different spatial data types mapping is as follows:
    - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    OceanBase typeFlink SQL typeNOTE
    NUMBER(1)BOOLEAN
    NUMBER(p, s <= 0), p - s < 3 TINYINT
    NUMBER(p, s <= 0), p - s < 5 SMALLINT
    NUMBER(p, s <= 0), p - s < 10 INT
    NUMBER(p, s <= 0), p - s < 19 BIGINT
    NUMBER(p, s <= 0), 19 <=p - s <=38DECIMAL(p - s, 0)
    NUMBER(p, s > 0)DECIMAL(p, s)
    NUMBER(p, s <= 0), p - s> 38 STRING
    - FLOAT
    - BINARY_FLOAT -
    FLOAT
    BINARY_DOUBLEDOUBLE
    - DATE
    - TIMESTAMP [(p)] -
    TIMESTAMP [(p)]
    - CHAR(n)
    - NCHAR(n)
    - VARCHAR(n)
    - VARCHAR2(n)
    - NVARCHAR2(n)
    - CLOB
    -
    STRING
    - RAW
    - BLOB
    - ROWID -
    BYTES
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Spatial dataJson String converted in Flink
    POINT(1 1){"coordinates":[1,1],"type":"Point","srid":0}
    LINESTRING(3 0, 3 3, 3 5){"coordinates":[[3,0],[3,3],[3,5]],"type":"LineString","srid":0}
    POLYGON((1 1, 2 1, 2 2, 1 2, 1 1)){"coordinates":[[[1,1],[2,1],[2,2],[1,2],[1,1]]],"type":"Polygon","srid":0}
    MULTIPOINT((1 1),(2 2)){"coordinates":[[1,1],[2,2]],"type":"MultiPoint","srid":0}
    MultiLineString((1 1,2 2,3 3),(4 4,5 5)){"coordinates":[[[1,1],[2,2],[3,3]],[[4,4],[5,5]]],"type":"MultiLineString","srid":0}
    MULTIPOLYGON(((0 0, 10 0, 10 10, 0 10, 0 0)), ((5 5, 7 5, 7 7, 5 7, 5 5))){"coordinates":[[[[0,0],[10,0],[10,10],[0,10],[0,0]]],[[[5,5],[7,5],[7,7],[5,7],[5,5]]]],"type":"MultiPolygon","srid":0}
    GEOMETRYCOLLECTION(POINT(10 10), POINT(30 30), LINESTRING(15 15, 20 20)){"geometries":[{"type":"Point","coordinates":[10,10]},{"type":"Point","coordinates":[30,30]},{"type":"LineString","coordinates":[[15,15],[20,20]]}],"type":"GeometryCollection","srid":0}
    {{< top >}} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/pom.xml b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/pom.xml index c8ce12ca3df..5aa51097aeb 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/pom.xml +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/pom.xml @@ -36,30 +36,10 @@ limitations under the License. org.apache.flink - flink-cdc-base + flink-connector-mysql-cdc ${project.version} - - - org.apache.flink - flink-connector-debezium - ${project.version} - - - kafka-log4j-appender - org.apache.kafka - - - - - - - com.oceanbase - oblogclient-logproxy - ${oblogclient.version} - - mysql diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java new file mode 100644 index 00000000000..1d0f27106b2 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java @@ -0,0 +1,1580 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ + +package io.debezium.connector.mysql; + +import com.github.shyiko.mysql.binlog.BinaryLogClient; +import com.github.shyiko.mysql.binlog.BinaryLogClient.LifecycleListener; +import com.github.shyiko.mysql.binlog.event.DeleteRowsEventData; +import com.github.shyiko.mysql.binlog.event.Event; +import com.github.shyiko.mysql.binlog.event.EventData; +import com.github.shyiko.mysql.binlog.event.EventHeader; +import com.github.shyiko.mysql.binlog.event.EventHeaderV4; +import com.github.shyiko.mysql.binlog.event.EventType; +import com.github.shyiko.mysql.binlog.event.GtidEventData; +import com.github.shyiko.mysql.binlog.event.QueryEventData; +import com.github.shyiko.mysql.binlog.event.RotateEventData; +import com.github.shyiko.mysql.binlog.event.RowsQueryEventData; +import com.github.shyiko.mysql.binlog.event.TableMapEventData; +import com.github.shyiko.mysql.binlog.event.UpdateRowsEventData; +import com.github.shyiko.mysql.binlog.event.WriteRowsEventData; +import com.github.shyiko.mysql.binlog.event.deserialization.EventDataDeserializationException; +import com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer; +import com.github.shyiko.mysql.binlog.event.deserialization.GtidEventDataDeserializer; +import com.github.shyiko.mysql.binlog.io.ByteArrayInputStream; +import com.github.shyiko.mysql.binlog.network.AuthenticationException; +import com.github.shyiko.mysql.binlog.network.DefaultSSLSocketFactory; +import com.github.shyiko.mysql.binlog.network.SSLMode; +import com.github.shyiko.mysql.binlog.network.SSLSocketFactory; +import com.github.shyiko.mysql.binlog.network.ServerException; +import io.debezium.DebeziumException; +import io.debezium.annotation.SingleThreadAccess; +import io.debezium.config.CommonConnectorConfig.EventProcessingFailureHandlingMode; +import io.debezium.config.Configuration; +import io.debezium.connector.mysql.MySqlConnectorConfig.GtidNewChannelPosition; +import io.debezium.connector.mysql.MySqlConnectorConfig.SecureConnectionMode; +import io.debezium.connector.mysql.util.ErrorMessageUtils; +import io.debezium.data.Envelope.Operation; +import io.debezium.function.BlockingConsumer; +import io.debezium.pipeline.ErrorHandler; +import io.debezium.pipeline.EventDispatcher; +import io.debezium.pipeline.source.spi.StreamingChangeEventSource; +import io.debezium.relational.TableId; +import io.debezium.schema.SchemaChangeEvent; +import io.debezium.util.Clock; +import io.debezium.util.Metronome; +import io.debezium.util.Strings; +import io.debezium.util.Threads; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.slf4j.event.Level; + +import javax.net.ssl.KeyManager; +import javax.net.ssl.KeyManagerFactory; +import javax.net.ssl.SSLContext; +import javax.net.ssl.TrustManager; +import javax.net.ssl.TrustManagerFactory; +import javax.net.ssl.X509TrustManager; + +import java.io.IOException; +import java.security.GeneralSecurityException; +import java.security.KeyStore; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; +import java.security.UnrecoverableKeyException; +import java.security.cert.CertificateException; +import java.security.cert.X509Certificate; +import java.sql.SQLException; +import java.time.Duration; +import java.time.Instant; +import java.util.EnumMap; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Predicate; + +import static io.debezium.util.Strings.isNullOrEmpty; + +/** + * Copied from mysql-cdc to be compatible with OceanBase Binlog Service. + * + *

    Copied from Debezium project(1.9.8.Final) to fix + * https://github.com/ververica/flink-cdc-connectors/issues/1944. + * + *

    Line 1427-1433 : Adjust GTID merging logic to support recovering from job which previously + * specifying starting offset on start. + * + *

    Line 1485 : Add more error details for some exceptions. + */ +public class MySqlStreamingChangeEventSource + implements StreamingChangeEventSource { + + private static final Logger LOGGER = + LoggerFactory.getLogger(MySqlStreamingChangeEventSource.class); + + private static final String KEEPALIVE_THREAD_NAME = "blc-keepalive"; + + private final EnumMap> eventHandlers = + new EnumMap<>(EventType.class); + private final BinaryLogClient client; + private final MySqlStreamingChangeEventSourceMetrics metrics; + private final Clock clock; + private final EventProcessingFailureHandlingMode eventDeserializationFailureHandlingMode; + private final EventProcessingFailureHandlingMode inconsistentSchemaHandlingMode; + + private int startingRowNumber = 0; + private long initialEventsToSkip = 0L; + private boolean skipEvent = false; + private boolean ignoreDmlEventByGtidSource = false; + private final Predicate gtidDmlSourceFilter; + private final AtomicLong totalRecordCounter = new AtomicLong(); + private volatile Map lastOffset = null; + private com.github.shyiko.mysql.binlog.GtidSet gtidSet; + private final float heartbeatIntervalFactor = 0.8f; + private final Map binaryLogClientThreads = new ConcurrentHashMap<>(4); + private final MySqlTaskContext taskContext; + private final MySqlConnectorConfig connectorConfig; + private final MySqlConnection connection; + private final EventDispatcher eventDispatcher; + private final ErrorHandler errorHandler; + + @SingleThreadAccess("binlog client thread") + private Instant eventTimestamp; + + /** Describe binlog position. */ + public static class BinlogPosition { + final String filename; + final long position; + + public BinlogPosition(String filename, long position) { + assert filename != null; + + this.filename = filename; + this.position = position; + } + + public String getFilename() { + return filename; + } + + public long getPosition() { + return position; + } + + @Override + public String toString() { + return filename + "/" + position; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + filename.hashCode(); + result = prime * result + (int) (position ^ (position >>> 32)); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + BinlogPosition other = (BinlogPosition) obj; + if (!filename.equals(other.filename)) { + return false; + } + if (position != other.position) { + return false; + } + return true; + } + } + + @FunctionalInterface + private interface BinlogChangeEmitter { + void emit(TableId tableId, T data) throws InterruptedException; + } + + public MySqlStreamingChangeEventSource( + MySqlConnectorConfig connectorConfig, + MySqlConnection connection, + EventDispatcher dispatcher, + ErrorHandler errorHandler, + Clock clock, + MySqlTaskContext taskContext, + MySqlStreamingChangeEventSourceMetrics metrics) { + + this.taskContext = taskContext; + this.connectorConfig = connectorConfig; + this.connection = connection; + this.clock = clock; + this.eventDispatcher = dispatcher; + this.errorHandler = errorHandler; + this.metrics = metrics; + + eventDeserializationFailureHandlingMode = + connectorConfig.getEventProcessingFailureHandlingMode(); + inconsistentSchemaHandlingMode = connectorConfig.inconsistentSchemaFailureHandlingMode(); + + // Set up the log reader ... + client = taskContext.getBinaryLogClient(); + // BinaryLogClient will overwrite thread names later + client.setThreadFactory( + Threads.threadFactory( + MySqlConnector.class, + connectorConfig.getLogicalName(), + "binlog-client", + false, + false, + x -> binaryLogClientThreads.put(x.getName(), x))); + client.setServerId(connectorConfig.serverId()); + client.setSSLMode(sslModeFor(connectorConfig.sslMode())); + if (connectorConfig.sslModeEnabled()) { + SSLSocketFactory sslSocketFactory = + getBinlogSslSocketFactory(connectorConfig, connection); + if (sslSocketFactory != null) { + client.setSslSocketFactory(sslSocketFactory); + } + } + Configuration configuration = connectorConfig.getConfig(); + client.setKeepAlive(configuration.getBoolean(MySqlConnectorConfig.KEEP_ALIVE)); + final long keepAliveInterval = + configuration.getLong(MySqlConnectorConfig.KEEP_ALIVE_INTERVAL_MS); + client.setKeepAliveInterval(keepAliveInterval); + // Considering heartbeatInterval should be less than keepAliveInterval, we use the + // heartbeatIntervalFactor + // multiply by keepAliveInterval and set the result value to heartbeatInterval.The default + // value of heartbeatIntervalFactor + // is 0.8, and we believe the left time (0.2 * keepAliveInterval) is enough to process the + // packet received from the MySQL server. + client.setHeartbeatInterval((long) (keepAliveInterval * heartbeatIntervalFactor)); + + boolean filterDmlEventsByGtidSource = + configuration.getBoolean(MySqlConnectorConfig.GTID_SOURCE_FILTER_DML_EVENTS); + gtidDmlSourceFilter = + filterDmlEventsByGtidSource ? connectorConfig.gtidSourceFilter() : null; + + // Set up the event deserializer with additional type(s) ... + final Map tableMapEventByTableId = + new HashMap(); + EventDeserializer eventDeserializer = + new EventDeserializer() { + @Override + public Event nextEvent(ByteArrayInputStream inputStream) throws IOException { + try { + // Delegate to the superclass ... + Event event = super.nextEvent(inputStream); + + // We have to record the most recent TableMapEventData for each table + // number for our custom deserializers ... + if (event.getHeader().getEventType() == EventType.TABLE_MAP) { + TableMapEventData tableMapEvent = event.getData(); + tableMapEventByTableId.put( + tableMapEvent.getTableId(), tableMapEvent); + } + + // DBZ-5126 Clean cache on rotate event to prevent it from growing + // indefinitely. + if (event.getHeader().getEventType() == EventType.ROTATE + && event.getHeader().getTimestamp() != 0) { + tableMapEventByTableId.clear(); + } + return event; + } + // DBZ-217 In case an event couldn't be read we create a pseudo-event for + // the sake of logging + catch (EventDataDeserializationException edde) { + // DBZ-3095 As of Java 15, when reaching EOF in the binlog stream, the + // polling loop in + // BinaryLogClient#listenForEventPackets() keeps returning values != -1 + // from peek(); + // this causes the loop to never finish + // Propagating the exception (either EOF or socket closed) causes the + // loop to be aborted + // in this case + if (edde.getCause() instanceof IOException) { + throw edde; + } + + EventHeaderV4 header = new EventHeaderV4(); + header.setEventType(EventType.INCIDENT); + header.setTimestamp(edde.getEventHeader().getTimestamp()); + header.setServerId(edde.getEventHeader().getServerId()); + + if (edde.getEventHeader() instanceof EventHeaderV4) { + header.setEventLength( + ((EventHeaderV4) edde.getEventHeader()).getEventLength()); + header.setNextPosition( + ((EventHeaderV4) edde.getEventHeader()).getNextPosition()); + header.setFlags(((EventHeaderV4) edde.getEventHeader()).getFlags()); + } + + EventData data = new EventDataDeserializationExceptionData(edde); + return new Event(header, data); + } + } + }; + + // Add our custom deserializers ... + eventDeserializer.setEventDataDeserializer(EventType.STOP, new StopEventDataDeserializer()); + eventDeserializer.setEventDataDeserializer(EventType.GTID, new GtidEventDataDeserializer()); + eventDeserializer.setEventDataDeserializer( + EventType.WRITE_ROWS, + new RowDeserializers.WriteRowsDeserializer(tableMapEventByTableId)); + eventDeserializer.setEventDataDeserializer( + EventType.UPDATE_ROWS, + new RowDeserializers.UpdateRowsDeserializer(tableMapEventByTableId)); + eventDeserializer.setEventDataDeserializer( + EventType.DELETE_ROWS, + new RowDeserializers.DeleteRowsDeserializer(tableMapEventByTableId)); + eventDeserializer.setEventDataDeserializer( + EventType.EXT_WRITE_ROWS, + new RowDeserializers.WriteRowsDeserializer(tableMapEventByTableId) + .setMayContainExtraInformation(true)); + eventDeserializer.setEventDataDeserializer( + EventType.EXT_UPDATE_ROWS, + new RowDeserializers.UpdateRowsDeserializer(tableMapEventByTableId) + .setMayContainExtraInformation(true)); + eventDeserializer.setEventDataDeserializer( + EventType.EXT_DELETE_ROWS, + new RowDeserializers.DeleteRowsDeserializer(tableMapEventByTableId) + .setMayContainExtraInformation(true)); + client.setEventDeserializer(eventDeserializer); + } + + protected void onEvent(MySqlOffsetContext offsetContext, Event event) { + long ts = 0; + + if (event.getHeader().getEventType() == EventType.HEARTBEAT) { + // HEARTBEAT events have no timestamp but are fired only when + // there is no traffic on the connection which means we are caught-up + // https://dev.mysql.com/doc/internals/en/heartbeat-event.html + metrics.setMilliSecondsBehindSource(ts); + return; + } + + // MySQL has seconds resolution but mysql-binlog-connector-java returns + // a value in milliseconds + long eventTs = event.getHeader().getTimestamp(); + + if (eventTs == 0) { + LOGGER.trace("Received unexpected event with 0 timestamp: {}", event); + return; + } + + ts = clock.currentTimeInMillis() - eventTs; + LOGGER.trace("Current milliseconds behind source: {} ms", ts); + metrics.setMilliSecondsBehindSource(ts); + } + + protected void ignoreEvent(MySqlOffsetContext offsetContext, Event event) { + LOGGER.trace("Ignoring event due to missing handler: {}", event); + } + + protected void handleEvent( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) { + if (event == null) { + return; + } + + final EventHeader eventHeader = event.getHeader(); + // Update the source offset info. Note that the client returns the value in *milliseconds*, + // even though the binlog + // contains only *seconds* precision ... + // HEARTBEAT events have no timestamp; only set the timestamp if the event is not a + // HEARTBEAT + eventTimestamp = + !eventHeader.getEventType().equals(EventType.HEARTBEAT) + ? Instant.ofEpochMilli(eventHeader.getTimestamp()) + : null; + offsetContext.setBinlogServerId(eventHeader.getServerId()); + + final EventType eventType = eventHeader.getEventType(); + if (eventType == EventType.ROTATE) { + EventData eventData = event.getData(); + RotateEventData rotateEventData; + if (eventData instanceof EventDeserializer.EventDataWrapper) { + rotateEventData = + (RotateEventData) + ((EventDeserializer.EventDataWrapper) eventData).getInternal(); + } else { + rotateEventData = (RotateEventData) eventData; + } + offsetContext.setBinlogStartPoint( + rotateEventData.getBinlogFilename(), rotateEventData.getBinlogPosition()); + } else if (eventHeader instanceof EventHeaderV4) { + EventHeaderV4 trackableEventHeader = (EventHeaderV4) eventHeader; + offsetContext.setEventPosition( + trackableEventHeader.getPosition(), trackableEventHeader.getEventLength()); + } + + // If there is a handler for this event, forward the event to it ... + try { + // Forward the event to the handler ... + eventHandlers + .getOrDefault(eventType, (e) -> ignoreEvent(offsetContext, e)) + .accept(event); + + // Generate heartbeat message if the time is right + eventDispatcher.dispatchHeartbeatEvent(partition, offsetContext); + + // Capture that we've completed another event ... + // Skip HeratBeat event for OceanBase Binlog Service + if (eventType != EventType.HEARTBEAT) { + offsetContext.completeEvent(); + } + + // update last offset used for logging + lastOffset = offsetContext.getOffset(); + + if (skipEvent) { + // We're in the mode of skipping events and we just skipped this one, so decrement + // our skip count ... + --initialEventsToSkip; + skipEvent = initialEventsToSkip > 0; + } + } catch (RuntimeException e) { + // There was an error in the event handler, so propagate the failure to Kafka Connect + // ... + logStreamingSourceState(); + errorHandler.setProducerThrowable( + new DebeziumException("Error processing binlog event", e)); + // Do not stop the client, since Kafka Connect should stop the connector on it's own + // (and doing it here may cause problems the second time it is stopped). + // We can clear the listeners though so that we ignore all future events ... + eventHandlers.clear(); + LOGGER.info( + "Error processing binlog event, and propagating to Kafka Connect so it stops this connector. Future binlog events read before connector is shutdown will be ignored."); + } catch (InterruptedException e) { + // Most likely because this reader was stopped and our thread was interrupted ... + Thread.currentThread().interrupt(); + eventHandlers.clear(); + LOGGER.info("Stopped processing binlog events due to thread interruption"); + } + } + + @SuppressWarnings("unchecked") + protected T unwrapData(Event event) { + EventData eventData = event.getData(); + if (eventData instanceof EventDeserializer.EventDataWrapper) { + eventData = ((EventDeserializer.EventDataWrapper) eventData).getInternal(); + } + return (T) eventData; + } + + /** + * Handle the supplied event that signals that mysqld has stopped. + * + * @param event the server stopped event to be processed; may not be null + */ + protected void handleServerStop(MySqlOffsetContext offsetContext, Event event) { + LOGGER.debug("Server stopped: {}", event); + } + + /** + * Handle the supplied event that is sent by a primary to a replica to let the replica know that + * the primary is still alive. Not written to a binary log. + * + * @param event the server stopped event to be processed; may not be null + */ + protected void handleServerHeartbeat( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) + throws InterruptedException { + LOGGER.trace("Server heartbeat: {}", event); + eventDispatcher.dispatchServerHeartbeatEvent(partition, offsetContext); + } + + /** + * Handle the supplied event that signals that an out of the ordinary event that occurred on the + * master. It notifies the replica that something happened on the primary that might cause data + * to be in an inconsistent state. + * + * @param event the server stopped event to be processed; may not be null + */ + protected void handleServerIncident( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) { + if (event.getData() instanceof EventDataDeserializationExceptionData) { + metrics.onErroneousEvent(partition, "source = " + event); + EventDataDeserializationExceptionData data = event.getData(); + + EventHeaderV4 eventHeader = + (EventHeaderV4) + data.getCause() + .getEventHeader(); // safe cast, instantiated that ourselves + + // logging some additional context but not the exception itself, this will happen in + // handleEvent() + if (eventDeserializationFailureHandlingMode + == EventProcessingFailureHandlingMode.FAIL) { + LOGGER.error( + "Error while deserializing binlog event at offset {}.{}" + + "Use the mysqlbinlog tool to view the problematic event: mysqlbinlog --start-position={} --stop-position={} --verbose {}", + offsetContext.getOffset(), + System.lineSeparator(), + eventHeader.getPosition(), + eventHeader.getNextPosition(), + offsetContext.getSource().binlogFilename()); + + throw new RuntimeException(data.getCause()); + } else if (eventDeserializationFailureHandlingMode + == EventProcessingFailureHandlingMode.WARN) { + LOGGER.warn( + "Error while deserializing binlog event at offset {}.{}" + + "This exception will be ignored and the event be skipped.{}" + + "Use the mysqlbinlog tool to view the problematic event: mysqlbinlog --start-position={} --stop-position={} --verbose {}", + offsetContext.getOffset(), + System.lineSeparator(), + System.lineSeparator(), + eventHeader.getPosition(), + eventHeader.getNextPosition(), + offsetContext.getSource().binlogFilename(), + data.getCause()); + } + } else { + LOGGER.error("Server incident: {}", event); + } + } + + /** + * Handle the supplied event with a {@link RotateEventData} that signals the logs are being + * rotated. This means that either the server was restarted, or the binlog has transitioned to a + * new file. In either case, subsequent table numbers will be different than those seen to this + * point. + * + * @param event the database change data event to be processed; may not be null + */ + protected void handleRotateLogsEvent(MySqlOffsetContext offsetContext, Event event) { + LOGGER.debug("Rotating logs: {}", event); + RotateEventData command = unwrapData(event); + assert command != null; + taskContext.getSchema().clearTableMappings(); + } + + /** + * Handle the supplied event with a {@link GtidEventData} that signals the beginning of a GTID + * transaction. We don't yet know whether this transaction contains any events we're interested + * in, but we have to record it so that we know the position of this event and know we've + * processed the binlog to this point. + * + *

    Note that this captures the current GTID and complete GTID set, regardless of whether the + * connector is {@link MySqlTaskContext#gtidSourceFilter() filtering} the GTID set upon + * connection. We do this because we actually want to capture all GTID set values found in the + * binlog, whether or not we process them. However, only when we connect do we actually want to + * pass to MySQL only those GTID ranges that are applicable per the configuration. + * + * @param event the GTID event to be processed; may not be null + */ + protected void handleGtidEvent(MySqlOffsetContext offsetContext, Event event) { + LOGGER.debug("GTID transaction: {}", event); + GtidEventData gtidEvent = unwrapData(event); + String gtid = gtidEvent.getGtid(); + gtidSet.add(gtid); + offsetContext.startGtid(gtid, gtidSet.toString()); // rather than use the client's GTID set + ignoreDmlEventByGtidSource = false; + if (gtidDmlSourceFilter != null && gtid != null) { + String uuid = gtid.trim().substring(0, gtid.indexOf(":")); + if (!gtidDmlSourceFilter.test(uuid)) { + ignoreDmlEventByGtidSource = true; + } + } + metrics.onGtidChange(gtid); + } + + /** + * Handle the supplied event with an {@link RowsQueryEventData} by recording the original SQL + * query that generated the event. + * + * @param event the database change data event to be processed; may not be null + */ + protected void handleRowsQuery(MySqlOffsetContext offsetContext, Event event) { + // Unwrap the RowsQueryEvent + final RowsQueryEventData lastRowsQueryEventData = unwrapData(event); + + // Set the query on the source + offsetContext.setQuery(lastRowsQueryEventData.getQuery()); + } + + /** + * Handle the supplied event with an {@link QueryEventData} by possibly recording the DDL + * statements as changes in the MySQL schemas. + * + * @param partition the partition in which the even occurred + * @param event the database change data event to be processed; may not be null + * @throws InterruptedException if this thread is interrupted while recording the DDL statements + */ + protected void handleQueryEvent( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) + throws InterruptedException { + QueryEventData command = unwrapData(event); + LOGGER.debug("Received query command: {}", event); + String sql = command.getSql().trim(); + if (sql.equalsIgnoreCase("BEGIN")) { + // We are starting a new transaction ... + offsetContext.startNextTransaction(); + eventDispatcher.dispatchTransactionStartedEvent( + partition, offsetContext.getTransactionId(), offsetContext); + offsetContext.setBinlogThread(command.getThreadId()); + if (initialEventsToSkip != 0) { + LOGGER.debug( + "Restarting partially-processed transaction; change events will not be created for the first {} events plus {} more rows in the next event", + initialEventsToSkip, + startingRowNumber); + // We are restarting, so we need to skip the events in this transaction that we + // processed previously... + skipEvent = true; + } + return; + } + if (sql.equalsIgnoreCase("COMMIT")) { + handleTransactionCompletion(partition, offsetContext, event); + return; + } + + String upperCasedStatementBegin = Strings.getBegin(sql, 7).toUpperCase(); + + if (upperCasedStatementBegin.startsWith("XA ")) { + // This is an XA transaction, and we currently ignore these and do nothing ... + return; + } + if (connectorConfig.getDdlFilter().test(sql)) { + LOGGER.debug("DDL '{}' was filtered out of processing", sql); + return; + } + if (upperCasedStatementBegin.equals("INSERT ") + || upperCasedStatementBegin.equals("UPDATE ") + || upperCasedStatementBegin.equals("DELETE ")) { + LOGGER.warn( + "Received DML '" + + sql + + "' for processing, binlog probably contains events generated with statement or mixed based replication format"); + return; + } + if (sql.equalsIgnoreCase("ROLLBACK")) { + // We have hit a ROLLBACK which is not supported + LOGGER.warn( + "Rollback statements cannot be handled without binlog buffering, the connector will fail. Please check '{}' to see how to enable buffering", + MySqlConnectorConfig.BUFFER_SIZE_FOR_BINLOG_READER.name()); + } + + final List schemaChangeEvents = + taskContext + .getSchema() + .parseStreamingDdl( + partition, + sql, + command.getDatabase(), + offsetContext, + clock.currentTimeAsInstant()); + try { + for (SchemaChangeEvent schemaChangeEvent : schemaChangeEvents) { + if (taskContext.getSchema().skipSchemaChangeEvent(schemaChangeEvent)) { + continue; + } + + final TableId tableId = + schemaChangeEvent.getTables().isEmpty() + ? null + : schemaChangeEvent.getTables().iterator().next().id(); + eventDispatcher.dispatchSchemaChangeEvent( + partition, + tableId, + (receiver) -> { + try { + receiver.schemaChangeEvent(schemaChangeEvent); + } catch (Exception e) { + throw new DebeziumException(e); + } + }); + } + } catch (InterruptedException e) { + LOGGER.info("Processing interrupted"); + } + } + + private void handleTransactionCompletion( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) + throws InterruptedException { + // We are completing the transaction ... + eventDispatcher.dispatchTransactionCommittedEvent(partition, offsetContext); + offsetContext.commitTransaction(); + offsetContext.setBinlogThread(-1L); + skipEvent = false; + ignoreDmlEventByGtidSource = false; + } + + /** + * Handle a change in the table metadata. + * + *

    This method should be called whenever we consume a TABLE_MAP event, and every transaction + * in the log should include one of these for each table affected by the transaction. Each table + * map event includes a monotonically-increasing numeric identifier, and this identifier is used + * within subsequent events within the same transaction. This table identifier can change when: + * + *

      + *
    1. the table structure is modified (e.g., via an {@code ALTER TABLE ...} command); or + *
    2. MySQL rotates to a new binary log file, even if the table structure does not change. + *
    + * + * @param event the update event; never null + */ + protected void handleUpdateTableMetadata( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) + throws InterruptedException { + TableMapEventData metadata = unwrapData(event); + long tableNumber = metadata.getTableId(); + String databaseName = metadata.getDatabase(); + String tableName = metadata.getTable(); + TableId tableId = new TableId(databaseName, null, tableName); + if (taskContext.getSchema().assignTableNumber(tableNumber, tableId)) { + LOGGER.debug("Received update table metadata event: {}", event); + } else { + informAboutUnknownTableIfRequired(partition, offsetContext, event, tableId); + } + } + + /** + * If we receive an event for a table that is monitored but whose metadata we don't know, either + * ignore that event or raise a warning or error as per the {@link + * MySqlConnectorConfig#INCONSISTENT_SCHEMA_HANDLING_MODE} configuration. + */ + private void informAboutUnknownTableIfRequired( + MySqlPartition partition, + MySqlOffsetContext offsetContext, + Event event, + TableId tableId, + Operation operation) + throws InterruptedException { + if (tableId != null + && connectorConfig.getTableFilters().dataCollectionFilter().isIncluded(tableId)) { + metrics.onErroneousEvent( + partition, "source = " + tableId + ", event " + event, operation); + EventHeaderV4 eventHeader = event.getHeader(); + + if (inconsistentSchemaHandlingMode == EventProcessingFailureHandlingMode.FAIL) { + LOGGER.error( + "Encountered change event '{}' at offset {} for table {} whose schema isn't known to this connector. One possible cause is an incomplete database history topic. Take a new snapshot in this case.{}" + + "Use the mysqlbinlog tool to view the problematic event: mysqlbinlog --start-position={} --stop-position={} --verbose {}", + event, + offsetContext.getOffset(), + tableId, + System.lineSeparator(), + eventHeader.getPosition(), + eventHeader.getNextPosition(), + offsetContext.getSource().binlogFilename()); + throw new DebeziumException( + "Encountered change event for table " + + tableId + + " whose schema isn't known to this connector"); + } else if (inconsistentSchemaHandlingMode == EventProcessingFailureHandlingMode.WARN) { + LOGGER.warn( + "Encountered change event '{}' at offset {} for table {} whose schema isn't known to this connector. One possible cause is an incomplete database history topic. Take a new snapshot in this case.{}" + + "The event will be ignored.{}" + + "Use the mysqlbinlog tool to view the problematic event: mysqlbinlog --start-position={} --stop-position={} --verbose {}", + event, + offsetContext.getOffset(), + tableId, + System.lineSeparator(), + System.lineSeparator(), + eventHeader.getPosition(), + eventHeader.getNextPosition(), + offsetContext.getSource().binlogFilename()); + } else { + LOGGER.debug( + "Encountered change event '{}' at offset {} for table {} whose schema isn't known to this connector. One possible cause is an incomplete database history topic. Take a new snapshot in this case.{}" + + "The event will be ignored.{}" + + "Use the mysqlbinlog tool to view the problematic event: mysqlbinlog --start-position={} --stop-position={} --verbose {}", + event, + offsetContext.getOffset(), + tableId, + System.lineSeparator(), + System.lineSeparator(), + eventHeader.getPosition(), + eventHeader.getNextPosition(), + offsetContext.getSource().binlogFilename()); + } + } else { + if (tableId == null) { + EventData eventData = unwrapData(event); + if (eventData instanceof WriteRowsEventData) { + tableId = + taskContext + .getSchema() + .getExcludeTableId( + ((WriteRowsEventData) eventData).getTableId()); + } else if (eventData instanceof UpdateRowsEventData) { + tableId = + taskContext + .getSchema() + .getExcludeTableId( + ((UpdateRowsEventData) eventData).getTableId()); + } else if (eventData instanceof DeleteRowsEventData) { + tableId = + taskContext + .getSchema() + .getExcludeTableId( + ((DeleteRowsEventData) eventData).getTableId()); + } + } + LOGGER.trace("Filtered {} event for {}", event.getHeader().getEventType(), tableId); + metrics.onFilteredEvent(partition, "source = " + tableId, operation); + eventDispatcher.dispatchFilteredEvent(partition, offsetContext); + } + } + + private void informAboutUnknownTableIfRequired( + MySqlPartition partition, + MySqlOffsetContext offsetContext, + Event event, + TableId tableId) + throws InterruptedException { + informAboutUnknownTableIfRequired(partition, offsetContext, event, tableId, null); + } + + /** + * Generate source records for the supplied event with an {@link WriteRowsEventData}. + * + * @param partition the partition in which the even occurred + * @param event the database change data event to be processed; may not be null + * @throws InterruptedException if this thread is interrupted while blocking + */ + protected void handleInsert( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) + throws InterruptedException { + handleChange( + partition, + offsetContext, + event, + Operation.CREATE, + WriteRowsEventData.class, + x -> taskContext.getSchema().getTableId(x.getTableId()), + WriteRowsEventData::getRows, + (tableId, row) -> + eventDispatcher.dispatchDataChangeEvent( + partition, + tableId, + new MySqlChangeRecordEmitter( + partition, + offsetContext, + clock, + Operation.CREATE, + null, + row))); + } + + /** + * Generate source records for the supplied event with an {@link UpdateRowsEventData}. + * + * @param partition the partition in which the even occurred + * @param event the database change data event to be processed; may not be null + * @throws InterruptedException if this thread is interrupted while blocking + */ + protected void handleUpdate( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) + throws InterruptedException { + handleChange( + partition, + offsetContext, + event, + Operation.UPDATE, + UpdateRowsEventData.class, + x -> taskContext.getSchema().getTableId(x.getTableId()), + UpdateRowsEventData::getRows, + (tableId, row) -> + eventDispatcher.dispatchDataChangeEvent( + partition, + tableId, + new MySqlChangeRecordEmitter( + partition, + offsetContext, + clock, + Operation.UPDATE, + row.getKey(), + row.getValue()))); + } + + /** + * Generate source records for the supplied event with an {@link DeleteRowsEventData}. + * + * @param partition the partition in which the even occurred + * @param event the database change data event to be processed; may not be null + * @throws InterruptedException if this thread is interrupted while blocking + */ + protected void handleDelete( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) + throws InterruptedException { + handleChange( + partition, + offsetContext, + event, + Operation.DELETE, + DeleteRowsEventData.class, + x -> taskContext.getSchema().getTableId(x.getTableId()), + DeleteRowsEventData::getRows, + (tableId, row) -> + eventDispatcher.dispatchDataChangeEvent( + partition, + tableId, + new MySqlChangeRecordEmitter( + partition, + offsetContext, + clock, + Operation.DELETE, + row, + null))); + } + + private void handleChange( + MySqlPartition partition, + MySqlOffsetContext offsetContext, + Event event, + Operation operation, + Class eventDataClass, + TableIdProvider tableIdProvider, + RowsProvider rowsProvider, + BinlogChangeEmitter changeEmitter) + throws InterruptedException { + if (skipEvent) { + // We can skip this because we should already be at least this far ... + LOGGER.info("Skipping previously processed row event: {}", event); + return; + } + if (ignoreDmlEventByGtidSource) { + LOGGER.debug("Skipping DML event because this GTID source is filtered: {}", event); + return; + } + final T data = unwrapData(event); + final TableId tableId = tableIdProvider.getTableId(data); + final List rows = rowsProvider.getRows(data); + String changeType = operation.name(); + + if (tableId != null && taskContext.getSchema().schemaFor(tableId) != null) { + int count = 0; + int numRows = rows.size(); + if (startingRowNumber < numRows) { + for (int row = startingRowNumber; row != numRows; ++row) { + offsetContext.setRowNumber(row, numRows); + offsetContext.event(tableId, eventTimestamp); + changeEmitter.emit(tableId, rows.get(row)); + count++; + } + if (LOGGER.isDebugEnabled()) { + if (startingRowNumber != 0) { + LOGGER.debug( + "Emitted {} {} record(s) for last {} row(s) in event: {}", + count, + changeType, + numRows - startingRowNumber, + event); + } else { + LOGGER.debug( + "Emitted {} {} record(s) for event: {}", count, changeType, event); + } + } + offsetContext.changeEventCompleted(); + } else { + // All rows were previously processed ... + LOGGER.debug("Skipping previously processed {} event: {}", changeType, event); + } + } else { + informAboutUnknownTableIfRequired(partition, offsetContext, event, tableId, operation); + } + startingRowNumber = 0; + } + + /** + * Handle a {@link EventType#VIEW_CHANGE} event. + * + * @param event the database change data event to be processed; may not be null + * @throws InterruptedException if this thread is interrupted while blocking + */ + protected void viewChange(MySqlOffsetContext offsetContext, Event event) + throws InterruptedException { + LOGGER.debug("View Change event: {}", event); + // do nothing + } + + /** + * Handle a {@link EventType#XA_PREPARE} event. + * + * @param event the database change data event to be processed; may not be null + * @throws InterruptedException if this thread is interrupted while blocking + */ + protected void prepareTransaction(MySqlOffsetContext offsetContext, Event event) + throws InterruptedException { + LOGGER.debug("XA Prepare event: {}", event); + // do nothing + } + + private SSLMode sslModeFor(SecureConnectionMode mode) { + switch (mode) { + case DISABLED: + return SSLMode.DISABLED; + case PREFERRED: + return SSLMode.PREFERRED; + case REQUIRED: + return SSLMode.REQUIRED; + case VERIFY_CA: + return SSLMode.VERIFY_CA; + case VERIFY_IDENTITY: + return SSLMode.VERIFY_IDENTITY; + } + return null; + } + + @Override + public void execute( + ChangeEventSourceContext context, + MySqlPartition partition, + MySqlOffsetContext offsetContext) + throws InterruptedException { + if (!connectorConfig.getSnapshotMode().shouldStream()) { + LOGGER.info( + "Streaming is disabled for snapshot mode {}", + connectorConfig.getSnapshotMode()); + return; + } + if (connectorConfig.getSnapshotMode() != MySqlConnectorConfig.SnapshotMode.NEVER) { + taskContext.getSchema().assureNonEmptySchema(); + } + final Set skippedOperations = connectorConfig.getSkippedOperations(); + + final MySqlOffsetContext effectiveOffsetContext = + offsetContext != null ? offsetContext : MySqlOffsetContext.initial(connectorConfig); + + // Register our event handlers ... + eventHandlers.put( + EventType.STOP, (event) -> handleServerStop(effectiveOffsetContext, event)); + eventHandlers.put( + EventType.HEARTBEAT, + (event) -> handleServerHeartbeat(partition, effectiveOffsetContext, event)); + eventHandlers.put( + EventType.INCIDENT, + (event) -> handleServerIncident(partition, effectiveOffsetContext, event)); + eventHandlers.put( + EventType.ROTATE, (event) -> handleRotateLogsEvent(effectiveOffsetContext, event)); + eventHandlers.put( + EventType.TABLE_MAP, + (event) -> handleUpdateTableMetadata(partition, effectiveOffsetContext, event)); + eventHandlers.put( + EventType.QUERY, + (event) -> handleQueryEvent(partition, effectiveOffsetContext, event)); + + if (!skippedOperations.contains(Operation.CREATE)) { + eventHandlers.put( + EventType.WRITE_ROWS, + (event) -> handleInsert(partition, effectiveOffsetContext, event)); + eventHandlers.put( + EventType.EXT_WRITE_ROWS, + (event) -> handleInsert(partition, effectiveOffsetContext, event)); + } + + if (!skippedOperations.contains(Operation.UPDATE)) { + eventHandlers.put( + EventType.UPDATE_ROWS, + (event) -> handleUpdate(partition, effectiveOffsetContext, event)); + eventHandlers.put( + EventType.EXT_UPDATE_ROWS, + (event) -> handleUpdate(partition, effectiveOffsetContext, event)); + } + + if (!skippedOperations.contains(Operation.DELETE)) { + eventHandlers.put( + EventType.DELETE_ROWS, + (event) -> handleDelete(partition, effectiveOffsetContext, event)); + eventHandlers.put( + EventType.EXT_DELETE_ROWS, + (event) -> handleDelete(partition, effectiveOffsetContext, event)); + } + + eventHandlers.put( + EventType.VIEW_CHANGE, (event) -> viewChange(effectiveOffsetContext, event)); + eventHandlers.put( + EventType.XA_PREPARE, (event) -> prepareTransaction(effectiveOffsetContext, event)); + eventHandlers.put( + EventType.XID, + (event) -> handleTransactionCompletion(partition, effectiveOffsetContext, event)); + + // Conditionally register ROWS_QUERY handler to parse SQL statements. + if (connectorConfig.includeSqlQuery()) { + eventHandlers.put( + EventType.ROWS_QUERY, + (event) -> handleRowsQuery(effectiveOffsetContext, event)); + } + + BinaryLogClient.EventListener listener; + if (connectorConfig.bufferSizeForStreamingChangeEventSource() == 0) { + listener = (event) -> handleEvent(partition, effectiveOffsetContext, event); + } else { + EventBuffer buffer = + new EventBuffer( + connectorConfig.bufferSizeForStreamingChangeEventSource(), + this, + context); + listener = (event) -> buffer.add(partition, effectiveOffsetContext, event); + } + client.registerEventListener(listener); + + client.registerLifecycleListener(new ReaderThreadLifecycleListener(effectiveOffsetContext)); + client.registerEventListener((event) -> onEvent(effectiveOffsetContext, event)); + if (LOGGER.isDebugEnabled()) { + client.registerEventListener((event) -> logEvent(effectiveOffsetContext, event)); + } + + final boolean isGtidModeEnabled = connection.isGtidModeEnabled(); + metrics.setIsGtidModeEnabled(isGtidModeEnabled); + + // Get the current GtidSet from MySQL so we can get a filtered/merged GtidSet based off of + // the last Debezium checkpoint. + String availableServerGtidStr = connection.knownGtidSet(); + if (isGtidModeEnabled) { + // The server is using GTIDs, so enable the handler ... + eventHandlers.put( + EventType.GTID, (event) -> handleGtidEvent(effectiveOffsetContext, event)); + + // Now look at the GTID set from the server and what we've previously seen ... + GtidSet availableServerGtidSet = new GtidSet(availableServerGtidStr); + + // also take into account purged GTID logs + GtidSet purgedServerGtidSet = connection.purgedGtidSet(); + LOGGER.info("GTID set purged on server: {}", purgedServerGtidSet); + + GtidSet filteredGtidSet = + filterGtidSet( + effectiveOffsetContext, availableServerGtidSet, purgedServerGtidSet); + if (filteredGtidSet != null) { + // We've seen at least some GTIDs, so start reading from the filtered GTID set ... + LOGGER.info("Registering binlog reader with GTID set: {}", filteredGtidSet); + String filteredGtidSetStr = filteredGtidSet.toString(); + client.setGtidSet(filteredGtidSetStr); + effectiveOffsetContext.setCompletedGtidSet(filteredGtidSetStr); + gtidSet = new com.github.shyiko.mysql.binlog.GtidSet(filteredGtidSetStr); + } else { + // We've not yet seen any GTIDs, so that means we have to start reading the binlog + // from the beginning ... + client.setBinlogFilename(effectiveOffsetContext.getSource().binlogFilename()); + client.setBinlogPosition(effectiveOffsetContext.getSource().binlogPosition()); + gtidSet = new com.github.shyiko.mysql.binlog.GtidSet(""); + } + } else { + // The server is not using GTIDs, so start reading the binlog based upon where we last + // left off ... + client.setBinlogFilename(effectiveOffsetContext.getSource().binlogFilename()); + client.setBinlogPosition(effectiveOffsetContext.getSource().binlogPosition()); + } + + // We may be restarting in the middle of a transaction, so see how far into the transaction + // we have already processed... + initialEventsToSkip = effectiveOffsetContext.eventsToSkipUponRestart(); + LOGGER.info("Skip {} events on streaming start", initialEventsToSkip); + + // Set the starting row number, which is the next row number to be read ... + startingRowNumber = effectiveOffsetContext.rowsToSkipUponRestart(); + LOGGER.info("Skip {} rows on streaming start", startingRowNumber); + + // Only when we reach the first BEGIN event will we start to skip events ... + skipEvent = false; + + try { + // Start the log reader, which starts background threads ... + if (context.isRunning()) { + long timeout = connectorConfig.getConnectionTimeout().toMillis(); + long started = clock.currentTimeInMillis(); + try { + LOGGER.debug( + "Attempting to establish binlog reader connection with timeout of {} ms", + timeout); + client.connect(timeout); + // Need to wait for keepalive thread to be running, otherwise it can be left + // orphaned + // The problem is with timing. When the close is called too early after connect + // then + // the keepalive thread is not terminated + if (client.isKeepAlive()) { + LOGGER.info("Waiting for keepalive thread to start"); + final Metronome metronome = Metronome.parker(Duration.ofMillis(100), clock); + int waitAttempts = 50; + boolean keepAliveThreadRunning = false; + while (!keepAliveThreadRunning && waitAttempts-- > 0) { + for (Thread t : binaryLogClientThreads.values()) { + if (t.getName().startsWith(KEEPALIVE_THREAD_NAME) && t.isAlive()) { + LOGGER.info("Keepalive thread is running"); + keepAliveThreadRunning = true; + } + } + metronome.pause(); + } + } + } catch (TimeoutException e) { + // If the client thread is interrupted *before* the client could connect, the + // client throws a timeout exception + // The only way we can distinguish this is if we get the timeout exception + // before the specified timeout has + // elapsed, so we simply check this (within 10%) ... + long duration = clock.currentTimeInMillis() - started; + if (duration > (0.9 * timeout)) { + double actualSeconds = TimeUnit.MILLISECONDS.toSeconds(duration); + throw new DebeziumException( + "Timed out after " + + actualSeconds + + " seconds while waiting to connect to MySQL at " + + connectorConfig.hostname() + + ":" + + connectorConfig.port() + + " with user '" + + connectorConfig.username() + + "'", + e); + } + // Otherwise, we were told to shutdown, so we don't care about the timeout + // exception + } catch (AuthenticationException e) { + throw new DebeziumException( + "Failed to authenticate to the MySQL database at " + + connectorConfig.hostname() + + ":" + + connectorConfig.port() + + " with user '" + + connectorConfig.username() + + "'", + e); + } catch (Throwable e) { + throw new DebeziumException( + "Unable to connect to the MySQL database at " + + connectorConfig.hostname() + + ":" + + connectorConfig.port() + + " with user '" + + connectorConfig.username() + + "': " + + e.getMessage(), + e); + } + } + while (context.isRunning()) { + Thread.sleep(100); + } + } finally { + try { + client.disconnect(); + } catch (Exception e) { + LOGGER.info("Exception while stopping binary log client", e); + } + } + } + + private SSLSocketFactory getBinlogSslSocketFactory( + MySqlConnectorConfig connectorConfig, MySqlConnection connection) { + String acceptedTlsVersion = connection.getSessionVariableForSslVersion(); + if (!isNullOrEmpty(acceptedTlsVersion)) { + SSLMode sslMode = sslModeFor(connectorConfig.sslMode()); + LOGGER.info( + "Enable ssl " + + sslMode + + " mode for connector " + + connectorConfig.getLogicalName()); + + final char[] keyPasswordArray = connection.connectionConfig().sslKeyStorePassword(); + final String keyFilename = connection.connectionConfig().sslKeyStore(); + final char[] trustPasswordArray = connection.connectionConfig().sslTrustStorePassword(); + final String trustFilename = connection.connectionConfig().sslTrustStore(); + KeyManager[] keyManagers = null; + if (keyFilename != null) { + try { + KeyStore ks = connection.loadKeyStore(keyFilename, keyPasswordArray); + + KeyManagerFactory kmf = KeyManagerFactory.getInstance("NewSunX509"); + kmf.init(ks, keyPasswordArray); + + keyManagers = kmf.getKeyManagers(); + } catch (KeyStoreException + | NoSuchAlgorithmException + | UnrecoverableKeyException e) { + throw new DebeziumException("Could not load keystore", e); + } + } + TrustManager[] trustManagers; + try { + KeyStore ks = null; + if (trustFilename != null) { + ks = connection.loadKeyStore(trustFilename, trustPasswordArray); + } + + if (ks == null && (sslMode == SSLMode.PREFERRED || sslMode == SSLMode.REQUIRED)) { + trustManagers = + new TrustManager[] { + new X509TrustManager() { + + @Override + public void checkClientTrusted( + X509Certificate[] x509Certificates, String s) + throws CertificateException {} + + @Override + public void checkServerTrusted( + X509Certificate[] x509Certificates, String s) + throws CertificateException {} + + @Override + public X509Certificate[] getAcceptedIssuers() { + return new X509Certificate[0]; + } + } + }; + } else { + TrustManagerFactory tmf = + TrustManagerFactory.getInstance( + TrustManagerFactory.getDefaultAlgorithm()); + tmf.init(ks); + trustManagers = tmf.getTrustManagers(); + } + } catch (KeyStoreException | NoSuchAlgorithmException e) { + throw new DebeziumException("Could not load truststore", e); + } + // DBZ-1208 Resembles the logic from the upstream BinaryLogClient, only that + // the accepted TLS version is passed to the constructed factory + final KeyManager[] finalKMS = keyManagers; + return new DefaultSSLSocketFactory(acceptedTlsVersion) { + + @Override + protected void initSSLContext(SSLContext sc) throws GeneralSecurityException { + sc.init(finalKMS, trustManagers, null); + } + }; + } + + return null; + } + + private void logStreamingSourceState() { + logStreamingSourceState(Level.ERROR); + } + + protected void logEvent(MySqlOffsetContext offsetContext, Event event) { + LOGGER.trace("Received event: {}", event); + } + + private void logStreamingSourceState(Level severity) { + final Object position = + client == null + ? "N/A" + : client.getBinlogFilename() + "/" + client.getBinlogPosition(); + final String message = + "Error during binlog processing. Last offset stored = {}, binlog reader near position = {}"; + switch (severity) { + case WARN: + LOGGER.warn(message, lastOffset, position); + break; + case DEBUG: + LOGGER.debug(message, lastOffset, position); + break; + default: + LOGGER.error(message, lastOffset, position); + } + } + + /** + * Apply the include/exclude GTID source filters to the current {@link #source() GTID set} and + * merge them onto the currently available GTID set from a MySQL server. + * + *

    The merging behavior of this method might seem a bit strange at first. It's required in + * order for Debezium to consume a MySQL binlog that has multi-source replication enabled, if a + * failover has to occur. In such a case, the server that Debezium is failed over to might have + * a different set of sources, but still include the sources required for Debezium to continue + * to function. MySQL does not allow downstream replicas to connect if the GTID set does not + * contain GTIDs for all channels that the server is replicating from, even if the server does + * have the data needed by the client. To get around this, we can have Debezium merge its GTID + * set with whatever is on the server, so that MySQL will allow it to connect. See DBZ-143 for details. + * + *

    This method does not mutate any state in the context. + * + * @param availableServerGtidSet the GTID set currently available in the MySQL server + * @param purgedServerGtid the GTID set already purged by the MySQL server + * @return A GTID set meant for consuming from a MySQL binlog; may return null if the SourceInfo + * has no GTIDs and therefore none were filtered + */ + public GtidSet filterGtidSet( + MySqlOffsetContext offsetContext, + GtidSet availableServerGtidSet, + GtidSet purgedServerGtid) { + String gtidStr = offsetContext.gtidSet(); + if (gtidStr == null) { + return null; + } + LOGGER.info("Attempting to generate a filtered GTID set"); + LOGGER.info("GTID set from previous recorded offset: {}", gtidStr); + GtidSet filteredGtidSet = new GtidSet(gtidStr); + Predicate gtidSourceFilter = connectorConfig.gtidSourceFilter(); + if (gtidSourceFilter != null) { + filteredGtidSet = filteredGtidSet.retainAll(gtidSourceFilter); + LOGGER.info( + "GTID set after applying GTID source includes/excludes to previous recorded offset: {}", + filteredGtidSet); + } + LOGGER.info("GTID set available on server: {}", availableServerGtidSet); + + GtidSet mergedGtidSet; + + if (connectorConfig.gtidNewChannelPosition() == GtidNewChannelPosition.EARLIEST) { + final GtidSet knownGtidSet = filteredGtidSet; + LOGGER.info("Using first available positions for new GTID channels"); + final GtidSet relevantAvailableServerGtidSet = + (gtidSourceFilter != null) + ? availableServerGtidSet.retainAll(gtidSourceFilter) + : availableServerGtidSet; + LOGGER.info( + "Relevant GTID set available on server: {}", relevantAvailableServerGtidSet); + + // Since the GTID recorded in the checkpoint represents the CDC-executed records, in + // certain scenarios + // (such as when the startup mode is earliest/timestamp/binlogfile), the recorded GTID + // may not start from + // the beginning. For example, A:300-500. However, during job recovery, we usually only + // need to focus on + // the last consumed point instead of consuming A:1-299. Therefore, some adjustments + // need to be made to the + // recorded offset in the checkpoint, and the available GTID for other MySQL instances + // should be completed. + mergedGtidSet = + GtidUtils.fixRestoredGtidSet( + GtidUtils.mergeGtidSetInto( + relevantAvailableServerGtidSet.retainAll( + uuid -> knownGtidSet.forServerWithId(uuid) != null), + purgedServerGtid), + filteredGtidSet); + } else { + mergedGtidSet = availableServerGtidSet.with(filteredGtidSet); + } + + LOGGER.info("Final merged GTID set to use when connecting to MySQL: {}", mergedGtidSet); + return mergedGtidSet; + } + + MySqlStreamingChangeEventSourceMetrics getMetrics() { + return metrics; + } + + void rewindBinaryLogClient(ChangeEventSourceContext context, BinlogPosition position) { + try { + if (context.isRunning()) { + LOGGER.debug("Rewinding binlog to position {}", position); + client.disconnect(); + client.setBinlogFilename(position.getFilename()); + client.setBinlogPosition(position.getPosition()); + client.connect(); + } + } catch (IOException e) { + LOGGER.error("Unexpected error when re-connecting to the MySQL binary log reader", e); + } + } + + BinlogPosition getCurrentBinlogPosition() { + return new BinlogPosition(client.getBinlogFilename(), client.getBinlogPosition()); + } + + /** + * Wraps the specified exception in a {@link DebeziumException}, ensuring that all useful state + * is captured inside the new exception's message. + * + * @param error the exception; may not be null + * @return the wrapped Kafka Connect exception + */ + protected DebeziumException wrap(Throwable error) { + assert error != null; + String msg = error.getMessage(); + if (error instanceof ServerException) { + ServerException e = (ServerException) error; + msg = msg + " Error code: " + e.getErrorCode() + "; SQLSTATE: " + e.getSqlState() + "."; + } else if (error instanceof SQLException) { + SQLException e = (SQLException) error; + msg = + e.getMessage() + + " Error code: " + + e.getErrorCode() + + "; SQLSTATE: " + + e.getSQLState() + + "."; + } + msg = ErrorMessageUtils.optimizeErrorMessage(msg); + return new DebeziumException(msg, error); + } + + /** LifecycleListener for Reader Thread. */ + protected final class ReaderThreadLifecycleListener implements LifecycleListener { + private final MySqlOffsetContext offsetContext; + + ReaderThreadLifecycleListener(MySqlOffsetContext offsetContext) { + this.offsetContext = offsetContext; + } + + @Override + public void onDisconnect(BinaryLogClient client) { + if (LOGGER.isInfoEnabled()) { + taskContext.temporaryLoggingContext( + connectorConfig, + "binlog", + () -> { + Map offset = lastOffset; + if (offset != null) { + LOGGER.info( + "Stopped reading binlog after {} events, last recorded offset: {}", + totalRecordCounter, + offset); + } else { + LOGGER.info( + "Stopped reading binlog after {} events, no new offset was recorded", + totalRecordCounter); + } + }); + } + } + + @Override + public void onConnect(BinaryLogClient client) { + // Set up the MDC logging context for this thread ... + taskContext.configureLoggingContext("binlog"); + + // The event row number will be used when processing the first event ... + LOGGER.info( + "Connected to MySQL binlog at {}:{}, starting at {}", + connectorConfig.hostname(), + connectorConfig.port(), + offsetContext); + } + + @Override + public void onCommunicationFailure(BinaryLogClient client, Exception ex) { + LOGGER.debug("A communication failure event arrived", ex); + logStreamingSourceState(); + try { + // Stop BinaryLogClient background threads + client.disconnect(); + } catch (final Exception e) { + LOGGER.debug("Exception while closing client", e); + } + errorHandler.setProducerThrowable(wrap(ex)); + } + + @Override + public void onEventDeserializationFailure(BinaryLogClient client, Exception ex) { + if (eventDeserializationFailureHandlingMode + == EventProcessingFailureHandlingMode.FAIL) { + LOGGER.debug("A deserialization failure event arrived", ex); + logStreamingSourceState(); + errorHandler.setProducerThrowable(wrap(ex)); + } else if (eventDeserializationFailureHandlingMode + == EventProcessingFailureHandlingMode.WARN) { + LOGGER.warn("A deserialization failure event arrived", ex); + logStreamingSourceState(Level.WARN); + } else { + LOGGER.debug("A deserialization failure event arrived", ex); + logStreamingSourceState(Level.DEBUG); + } + } + } + + @FunctionalInterface + private interface TableIdProvider { + TableId getTableId(E data); + } + + @FunctionalInterface + private interface RowsProvider { + List getRows(E data); + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSource.java deleted file mode 100644 index 195d505e6af..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSource.java +++ /dev/null @@ -1,313 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase; - -import org.apache.flink.cdc.common.annotation.PublicEvolving; -import org.apache.flink.cdc.connectors.base.options.StartupOptions; -import org.apache.flink.cdc.connectors.oceanbase.source.OceanBaseRichSourceFunction; -import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; -import org.apache.flink.streaming.api.functions.source.SourceFunction; - -import com.oceanbase.clogproxy.client.config.ClientConf; -import com.oceanbase.clogproxy.client.config.ObReaderConfig; -import org.apache.commons.lang3.StringUtils; - -import java.time.Duration; -import java.time.Instant; -import java.time.ZoneId; -import java.time.format.DateTimeFormatter; -import java.util.HashMap; -import java.util.Map; -import java.util.Properties; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** A builder to build a SourceFunction which can read snapshot and change events of OceanBase. */ -@PublicEvolving -public class OceanBaseSource { - - public static Builder builder() { - return new Builder<>(); - } - - /** Builder class of {@link OceanBaseSource}. */ - public static class Builder { - - // common config - private StartupOptions startupOptions; - private String username; - private String password; - private String tenantName; - private String databaseName; - private String tableName; - private String tableList; - private String serverTimeZone; - private Duration connectTimeout; - - // snapshot reading config - private String hostname; - private Integer port; - private String compatibleMode; - private String jdbcDriver; - private Properties jdbcProperties; - - // incremental reading config - private String logProxyHost; - private Integer logProxyPort; - private String logProxyClientId; - private Long startupTimestamp; - private String rsList; - private String configUrl; - private String workingMode; - private Properties obcdcProperties; - private Properties debeziumProperties; - - private DebeziumDeserializationSchema deserializer; - - public Builder startupOptions(StartupOptions startupOptions) { - this.startupOptions = startupOptions; - return this; - } - - public Builder username(String username) { - this.username = username; - return this; - } - - public Builder password(String password) { - this.password = password; - return this; - } - - public Builder tenantName(String tenantName) { - this.tenantName = tenantName; - return this; - } - - public Builder databaseName(String databaseName) { - this.databaseName = databaseName; - return this; - } - - public Builder tableName(String tableName) { - this.tableName = tableName; - return this; - } - - public Builder tableList(String tableList) { - this.tableList = tableList; - return this; - } - - public Builder serverTimeZone(String serverTimeZone) { - this.serverTimeZone = serverTimeZone; - return this; - } - - public Builder connectTimeout(Duration connectTimeout) { - this.connectTimeout = connectTimeout; - return this; - } - - public Builder hostname(String hostname) { - this.hostname = hostname; - return this; - } - - public Builder port(int port) { - this.port = port; - return this; - } - - public Builder compatibleMode(String compatibleMode) { - this.compatibleMode = compatibleMode; - return this; - } - - public Builder jdbcDriver(String jdbcDriver) { - this.jdbcDriver = jdbcDriver; - return this; - } - - public Builder jdbcProperties(Properties jdbcProperties) { - this.jdbcProperties = jdbcProperties; - return this; - } - - public Builder logProxyHost(String logProxyHost) { - this.logProxyHost = logProxyHost; - return this; - } - - public Builder logProxyPort(Integer logProxyPort) { - this.logProxyPort = logProxyPort; - return this; - } - - public Builder logProxyClientId(String logProxyClientId) { - this.logProxyClientId = logProxyClientId; - return this; - } - - public Builder startupTimestamp(Long startupTimestamp) { - this.startupTimestamp = startupTimestamp; - return this; - } - - public Builder rsList(String rsList) { - this.rsList = rsList; - return this; - } - - public Builder configUrl(String configUrl) { - this.configUrl = configUrl; - return this; - } - - public Builder workingMode(String workingMode) { - this.workingMode = workingMode; - return this; - } - - public Builder obcdcProperties(Properties obcdcProperties) { - this.obcdcProperties = obcdcProperties; - return this; - } - - public Builder debeziumProperties(Properties debeziumProperties) { - this.debeziumProperties = debeziumProperties; - return this; - } - - public Builder deserializer(DebeziumDeserializationSchema deserializer) { - this.deserializer = deserializer; - return this; - } - - public SourceFunction build() { - checkNotNull(username, "username shouldn't be null"); - checkNotNull(password, "password shouldn't be null"); - checkNotNull(hostname, "hostname shouldn't be null"); - checkNotNull(port, "port shouldn't be null"); - - if (startupOptions == null) { - startupOptions = StartupOptions.initial(); - } - if (compatibleMode == null) { - compatibleMode = "mysql"; - } - if (jdbcDriver == null) { - jdbcDriver = "com.mysql.cj.jdbc.Driver"; - } - - if (connectTimeout == null) { - connectTimeout = Duration.ofSeconds(30); - } - - if (serverTimeZone == null) { - serverTimeZone = ZoneId.systemDefault().getId(); - } - - switch (startupOptions.startupMode) { - case SNAPSHOT: - break; - case INITIAL: - case LATEST_OFFSET: - startupTimestamp = 0L; - break; - case TIMESTAMP: - checkNotNull( - startupTimestamp, - "startupTimestamp shouldn't be null on startup mode 'timestamp'"); - break; - default: - throw new UnsupportedOperationException( - startupOptions.startupMode + " mode is not supported."); - } - - if (StringUtils.isNotEmpty(databaseName) || StringUtils.isNotEmpty(tableName)) { - if (StringUtils.isEmpty(databaseName) || StringUtils.isEmpty(tableName)) { - throw new IllegalArgumentException( - "'database-name' and 'table-name' should be configured at the same time"); - } - } else { - checkNotNull( - tableList, - "'database-name', 'table-name' or 'table-list' should be configured"); - } - - ClientConf clientConf = null; - ObReaderConfig obReaderConfig = null; - - if (!startupOptions.isSnapshotOnly()) { - - checkNotNull(logProxyHost); - checkNotNull(logProxyPort); - checkNotNull(tenantName); - - obReaderConfig = new ObReaderConfig(); - if (StringUtils.isNotEmpty(rsList)) { - obReaderConfig.setRsList(rsList); - } - if (StringUtils.isNotEmpty(configUrl)) { - obReaderConfig.setClusterUrl(configUrl); - } - if (StringUtils.isNotEmpty(workingMode)) { - obReaderConfig.setWorkingMode(workingMode); - } - obReaderConfig.setUsername(username); - obReaderConfig.setPassword(password); - obReaderConfig.setStartTimestamp(startupTimestamp); - obReaderConfig.setTimezone( - DateTimeFormatter.ofPattern("xxx") - .format( - ZoneId.of(serverTimeZone) - .getRules() - .getOffset(Instant.now()))); - - if (obcdcProperties != null && !obcdcProperties.isEmpty()) { - Map extraConfigs = new HashMap<>(); - obcdcProperties.forEach((k, v) -> extraConfigs.put(k.toString(), v.toString())); - obReaderConfig.setExtraConfigs(extraConfigs); - } - } - - return new OceanBaseRichSourceFunction<>( - startupOptions, - username, - password, - tenantName, - databaseName, - tableName, - tableList, - serverTimeZone, - connectTimeout, - hostname, - port, - compatibleMode, - jdbcDriver, - jdbcProperties, - logProxyHost, - logProxyPort, - logProxyClientId, - obReaderConfig, - debeziumProperties, - deserializer); - } - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseRichSourceFunction.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseRichSourceFunction.java deleted file mode 100644 index 84508fba6d6..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseRichSourceFunction.java +++ /dev/null @@ -1,598 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase.source; - -import org.apache.flink.api.common.state.CheckpointListener; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeutils.base.LongSerializer; -import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import org.apache.flink.cdc.connectors.base.options.StartupOptions; -import org.apache.flink.cdc.connectors.oceanbase.source.config.OceanBaseConnectorConfig; -import org.apache.flink.cdc.connectors.oceanbase.source.connection.OceanBaseConnection; -import org.apache.flink.cdc.connectors.oceanbase.source.offset.OceanBaseSourceInfo; -import org.apache.flink.cdc.connectors.oceanbase.source.schema.OceanBaseDatabaseSchema; -import org.apache.flink.cdc.connectors.oceanbase.source.schema.OceanBaseSchema; -import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.state.FunctionInitializationContext; -import org.apache.flink.runtime.state.FunctionSnapshotContext; -import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; -import org.apache.flink.streaming.api.functions.source.RichSourceFunction; -import org.apache.flink.util.Collector; -import org.apache.flink.util.FlinkRuntimeException; - -import com.oceanbase.clogproxy.client.LogProxyClient; -import com.oceanbase.clogproxy.client.config.ClientConf; -import com.oceanbase.clogproxy.client.config.ObReaderConfig; -import com.oceanbase.clogproxy.client.exception.LogProxyClientException; -import com.oceanbase.clogproxy.client.listener.RecordListener; -import com.oceanbase.clogproxy.client.util.ClientUtil; -import com.oceanbase.oms.logmessage.DataMessage; -import com.oceanbase.oms.logmessage.LogMessage; -import io.debezium.connector.SnapshotRecord; -import io.debezium.relational.TableId; -import io.debezium.relational.TableSchema; -import io.debezium.relational.history.TableChanges; -import org.apache.commons.lang3.StringUtils; -import org.apache.kafka.connect.data.Field; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.source.SourceRecord; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.sql.SQLException; -import java.time.Duration; -import java.time.Instant; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.stream.Collectors; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * The source implementation for OceanBase that read snapshot events first and then read the change - * event. - * - * @param The type created by the deserializer. - */ -public class OceanBaseRichSourceFunction extends RichSourceFunction - implements CheckpointListener, CheckpointedFunction, ResultTypeQueryable { - - private static final long serialVersionUID = 2844054619864617340L; - - private static final Logger LOG = LoggerFactory.getLogger(OceanBaseRichSourceFunction.class); - - private final StartupOptions startupOptions; - private final String username; - private final String password; - private final String tenantName; - private final String databaseName; - private final String tableName; - private final String tableList; - private final String serverTimeZone; - private final Duration connectTimeout; - private final String hostname; - private final Integer port; - private final String compatibleMode; - private final String jdbcDriver; - private final Properties jdbcProperties; - private final String logProxyHost; - private final Integer logProxyPort; - private final String logProxyClientId; - private final ObReaderConfig obReaderConfig; - private final Properties debeziumProperties; - private final DebeziumDeserializationSchema deserializer; - - private final List changeRecordBuffer = new LinkedList<>(); - - private transient OceanBaseConnectorConfig connectorConfig; - private transient OceanBaseSourceInfo sourceInfo; - private transient Set tableSet; - private transient OceanBaseSchema obSchema; - private transient OceanBaseDatabaseSchema databaseSchema; - private transient volatile long resolvedTimestamp; - private transient volatile Exception logProxyClientException; - private transient volatile OceanBaseConnection snapshotConnection; - private transient LogProxyClient logProxyClient; - private transient ListState offsetState; - private transient OutputCollector outputCollector; - - public OceanBaseRichSourceFunction( - StartupOptions startupOptions, - String username, - String password, - String tenantName, - String databaseName, - String tableName, - String tableList, - String serverTimeZone, - Duration connectTimeout, - String hostname, - Integer port, - String compatibleMode, - String jdbcDriver, - Properties jdbcProperties, - String logProxyHost, - Integer logProxyPort, - String logProxyClientId, - ObReaderConfig obReaderConfig, - Properties debeziumProperties, - DebeziumDeserializationSchema deserializer) { - this.startupOptions = checkNotNull(startupOptions); - this.username = checkNotNull(username); - this.password = checkNotNull(password); - this.tenantName = tenantName; - this.databaseName = databaseName; - this.tableName = tableName; - this.tableList = tableList; - this.serverTimeZone = checkNotNull(serverTimeZone); - this.connectTimeout = checkNotNull(connectTimeout); - this.hostname = checkNotNull(hostname); - this.port = checkNotNull(port); - this.compatibleMode = checkNotNull(compatibleMode); - this.jdbcDriver = checkNotNull(jdbcDriver); - this.jdbcProperties = jdbcProperties; - this.logProxyHost = logProxyHost; - this.logProxyPort = logProxyPort; - this.logProxyClientId = logProxyClientId; - this.obReaderConfig = obReaderConfig; - this.debeziumProperties = debeziumProperties; - this.deserializer = checkNotNull(deserializer); - } - - @Override - public void open(final Configuration config) throws Exception { - super.open(config); - this.outputCollector = new OutputCollector<>(); - this.connectorConfig = - new OceanBaseConnectorConfig(compatibleMode, serverTimeZone, debeziumProperties); - this.sourceInfo = new OceanBaseSourceInfo(connectorConfig, tenantName); - } - - @Override - public void run(SourceContext ctx) throws Exception { - outputCollector.context = ctx; - try { - LOG.info("Start to initial table whitelist"); - initTableWhiteList(); - - if (resolvedTimestamp <= 0 && !startupOptions.isStreamOnly()) { - sourceInfo.setSnapshot(SnapshotRecord.TRUE); - long startTimestamp = getSnapshotConnection().getCurrentTimestampS(); - LOG.info("Snapshot reading started from timestamp: {}", startTimestamp); - readSnapshotRecords(); - sourceInfo.setSnapshot(SnapshotRecord.FALSE); - LOG.info("Snapshot reading finished"); - resolvedTimestamp = startTimestamp; - } else { - LOG.info("Snapshot reading skipped"); - } - - if (!startupOptions.isSnapshotOnly()) { - sourceInfo.setSnapshot(SnapshotRecord.FALSE); - LOG.info("Change events reading started"); - readChangeRecords(); - } - } finally { - cancel(); - } - } - - private OceanBaseConnection getSnapshotConnection() { - if (snapshotConnection == null) { - snapshotConnection = - new OceanBaseConnection( - hostname, - port, - username, - password, - connectTimeout, - compatibleMode, - jdbcDriver, - jdbcProperties, - getClass().getClassLoader()); - } - return snapshotConnection; - } - - private void closeSnapshotConnection() { - if (snapshotConnection != null) { - try { - snapshotConnection.close(); - } catch (SQLException e) { - LOG.error("Failed to close snapshotConnection", e); - } - snapshotConnection = null; - } - } - - private TableId tableId(String databaseName, String tableName) { - if ("mysql".equalsIgnoreCase(compatibleMode)) { - return new TableId(databaseName, null, tableName); - } - return new TableId(null, databaseName, tableName); - } - - private void initTableWhiteList() { - if (tableSet != null && !tableSet.isEmpty()) { - return; - } - - final Set localTableSet = new HashSet<>(); - - if (StringUtils.isNotBlank(tableList)) { - for (String s : tableList.split(",")) { - if (StringUtils.isNotBlank(s)) { - String[] arr = s.split("\\."); - TableId tableId = tableId(arr[0].trim(), arr[1].trim()); - localTableSet.add(tableId); - } - } - } - - if (StringUtils.isNotBlank(databaseName) && StringUtils.isNotBlank(tableName)) { - try { - List tableIds = getSnapshotConnection().getTables(databaseName, tableName); - LOG.info("Pattern matched tables: {}", tableIds); - localTableSet.addAll(tableIds); - } catch (SQLException e) { - LOG.error( - String.format( - "Query table list by 'databaseName' %s and 'tableName' %s failed.", - databaseName, tableName), - e); - throw new FlinkRuntimeException(e); - } - } - - if (localTableSet.isEmpty()) { - throw new FlinkRuntimeException("No valid table found"); - } - - LOG.info("Table list: {}", localTableSet); - this.tableSet = localTableSet; - // for some 4.x versions, it will be treated as 'tenant.*.*' - if (this.obReaderConfig != null) { - this.obReaderConfig.setTableWhiteList( - localTableSet.stream() - .map(tableId -> String.format("%s.%s", tenantName, tableId.toString())) - .collect(Collectors.joining("|"))); - } - } - - private TableSchema getTableSchema(TableId tableId) { - if (databaseSchema == null) { - databaseSchema = - new OceanBaseDatabaseSchema(connectorConfig, t -> tableSet.contains(t), false); - } - TableSchema tableSchema = databaseSchema.schemaFor(tableId); - if (tableSchema != null) { - return tableSchema; - } - - if (obSchema == null) { - obSchema = new OceanBaseSchema(); - } - TableChanges.TableChange tableChange = - obSchema.getTableSchema(getSnapshotConnection(), tableId); - databaseSchema.refresh(tableChange.getTable()); - return databaseSchema.schemaFor(tableId); - } - - protected void readSnapshotRecords() { - tableSet.forEach(this::readSnapshotRecordsByTable); - } - - private void readSnapshotRecordsByTable(TableId tableId) { - String fullName = getSnapshotConnection().quotedTableIdString(tableId); - sourceInfo.tableEvent(tableId); - try (OceanBaseConnection connection = getSnapshotConnection()) { - LOG.info("Start to read snapshot from {}", connection.quotedTableIdString(tableId)); - connection.query( - "SELECT * FROM " + fullName, - rs -> { - TableSchema tableSchema = getTableSchema(tableId); - List fields = tableSchema.valueSchema().fields(); - - while (rs.next()) { - Object[] fieldValues = new Object[fields.size()]; - for (Field field : fields) { - fieldValues[field.index()] = rs.getObject(field.name()); - } - Struct value = tableSchema.valueFromColumnData(fieldValues); - Instant now = Instant.now(); - Struct struct = - tableSchema - .getEnvelopeSchema() - .read(value, sourceInfo.struct(), now); - try { - deserializer.deserialize( - new SourceRecord( - null, - null, - tableId.identifier(), - null, - null, - null, - struct.schema(), - struct), - outputCollector); - } catch (Exception e) { - LOG.error("Deserialize snapshot record failed ", e); - throw new FlinkRuntimeException(e); - } - } - }); - LOG.info("Read snapshot from {} finished", fullName); - } catch (SQLException e) { - LOG.error("Read snapshot from table " + fullName + " failed", e); - throw new FlinkRuntimeException(e); - } - } - - protected void readChangeRecords() throws InterruptedException, TimeoutException { - if (resolvedTimestamp > 0) { - obReaderConfig.updateCheckpoint(Long.toString(resolvedTimestamp)); - LOG.info("Restore from timestamp: {}", resolvedTimestamp); - } - - ClientConf clientConf = - ClientConf.builder() - .clientId( - logProxyClientId != null - ? logProxyClientId - : String.format( - "%s_%s_%s", - ClientUtil.generateClientId(), - Thread.currentThread().getId(), - tenantName)) - .maxReconnectTimes(0) - .connectTimeoutMs((int) connectTimeout.toMillis()) - .build(); - - logProxyClient = new LogProxyClient(logProxyHost, logProxyPort, obReaderConfig, clientConf); - - final CountDownLatch latch = new CountDownLatch(1); - - logProxyClient.addListener( - new RecordListener() { - - boolean started = false; - - @Override - public void notify(LogMessage message) { - switch (message.getOpt()) { - case HEARTBEAT: - case BEGIN: - if (!started) { - started = true; - latch.countDown(); - } - break; - case INSERT: - case UPDATE: - case DELETE: - if (!started) { - break; - } - SourceRecord record = getChangeRecord(message); - if (record != null) { - changeRecordBuffer.add(record); - } - break; - case COMMIT: - changeRecordBuffer.forEach( - r -> { - try { - deserializer.deserialize(r, outputCollector); - } catch (Exception e) { - throw new FlinkRuntimeException(e); - } - }); - changeRecordBuffer.clear(); - long timestamp = Long.parseLong(message.getSafeTimestamp()); - if (timestamp > resolvedTimestamp) { - resolvedTimestamp = timestamp; - } - break; - case DDL: - // TODO record ddl and remove expired table schema - LOG.trace( - "Ddl: {}", - message.getFieldList().get(0).getValue().toString()); - break; - default: - throw new UnsupportedOperationException( - "Unsupported type: " + message.getOpt()); - } - } - - @Override - public void onException(LogProxyClientException e) { - logProxyClientException = e; - logProxyClient.stop(); - } - }); - - LOG.info( - "Try to start LogProxyClient with client id: {}, config: {}", - clientConf.getClientId(), - obReaderConfig); - logProxyClient.start(); - - if (!latch.await(connectTimeout.getSeconds(), TimeUnit.SECONDS)) { - throw new TimeoutException( - "Timeout to receive log messages in LogProxyClient.RecordListener"); - } - LOG.info("LogProxyClient started successfully"); - - logProxyClient.join(); - - if (logProxyClientException != null) { - throw new RuntimeException("LogProxyClient exception", logProxyClientException); - } - } - - private SourceRecord getChangeRecord(LogMessage message) { - String databaseName = message.getDbName().replace(tenantName + ".", ""); - TableId tableId = tableId(databaseName, message.getTableName()); - if (!tableSet.contains(tableId)) { - return null; - } - - sourceInfo.tableEvent(tableId); - sourceInfo.setSourceTime(Instant.ofEpochSecond(Long.parseLong(message.getTimestamp()))); - Struct source = sourceInfo.struct(); - - TableSchema tableSchema = getTableSchema(tableId); - Struct struct; - Schema valueSchema = tableSchema.valueSchema(); - List fields = valueSchema.fields(); - Struct before, after; - Object[] beforeFieldValues, afterFieldValues; - Map beforeValueMap = new HashMap<>(); - Map afterValueMap = new HashMap<>(); - message.getFieldList() - .forEach( - field -> { - if (field.isPrev()) { - beforeValueMap.put(field.getFieldname(), getFieldValue(field)); - } else { - afterValueMap.put(field.getFieldname(), getFieldValue(field)); - } - }); - switch (message.getOpt()) { - case INSERT: - afterFieldValues = new Object[fields.size()]; - for (Field field : fields) { - afterFieldValues[field.index()] = afterValueMap.get(field.name()); - } - after = tableSchema.valueFromColumnData(afterFieldValues); - struct = tableSchema.getEnvelopeSchema().create(after, source, Instant.now()); - break; - case DELETE: - beforeFieldValues = new Object[fields.size()]; - for (Field field : fields) { - beforeFieldValues[field.index()] = beforeValueMap.get(field.name()); - } - before = tableSchema.valueFromColumnData(beforeFieldValues); - struct = tableSchema.getEnvelopeSchema().delete(before, source, Instant.now()); - break; - case UPDATE: - beforeFieldValues = new Object[fields.size()]; - afterFieldValues = new Object[fields.size()]; - for (Field field : fields) { - beforeFieldValues[field.index()] = beforeValueMap.get(field.name()); - afterFieldValues[field.index()] = afterValueMap.get(field.name()); - } - before = tableSchema.valueFromColumnData(beforeFieldValues); - after = tableSchema.valueFromColumnData(afterFieldValues); - struct = - tableSchema - .getEnvelopeSchema() - .update(before, after, source, Instant.now()); - break; - default: - throw new UnsupportedOperationException(); - } - return new SourceRecord( - null, null, tableId.identifier(), null, null, null, struct.schema(), struct); - } - - private Object getFieldValue(DataMessage.Record.Field field) { - if (field.getValue() == null) { - return null; - } - String encoding = field.getEncoding(); - if ("binary".equalsIgnoreCase(encoding)) { - return field.getValue().getBytes(); - } - return field.getValue().toString(encoding); - } - - @Override - public void notifyCheckpointComplete(long l) { - // do nothing - } - - @Override - public TypeInformation getProducedType() { - return this.deserializer.getProducedType(); - } - - @Override - public void snapshotState(FunctionSnapshotContext context) throws Exception { - LOG.info( - "snapshotState checkpoint: {} at resolvedTimestamp: {}", - context.getCheckpointId(), - resolvedTimestamp); - offsetState.clear(); - offsetState.add(resolvedTimestamp); - } - - @Override - public void initializeState(FunctionInitializationContext context) throws Exception { - LOG.info("initialize checkpoint"); - offsetState = - context.getOperatorStateStore() - .getListState( - new ListStateDescriptor<>( - "resolvedTimestampState", LongSerializer.INSTANCE)); - if (context.isRestored()) { - for (final Long offset : offsetState.get()) { - resolvedTimestamp = offset; - LOG.info("Restore State from resolvedTimestamp: {}", resolvedTimestamp); - return; - } - } - } - - @Override - public void cancel() { - closeSnapshotConnection(); - if (logProxyClient != null) { - logProxyClient.stop(); - } - } - - private static class OutputCollector implements Collector { - - private SourceContext context; - - @Override - public void collect(T record) { - context.collect(record); - } - - @Override - public void close() { - // do nothing - } - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/config/OceanBaseConnectorConfig.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/config/OceanBaseConnectorConfig.java deleted file mode 100644 index a2b55b60c71..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/config/OceanBaseConnectorConfig.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase.source.config; - -import org.apache.flink.cdc.connectors.oceanbase.source.offset.OceanBaseSourceInfoStructMaker; - -import io.debezium.config.Configuration; -import io.debezium.connector.SourceInfoStructMaker; -import io.debezium.relational.ColumnFilterMode; -import io.debezium.relational.RelationalDatabaseConnectorConfig; -import io.debezium.relational.TableId; -import io.debezium.relational.Tables; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Properties; - -/** Debezium connector config. */ -public class OceanBaseConnectorConfig extends RelationalDatabaseConnectorConfig { - - protected static final String LOGICAL_NAME = "oceanbase_cdc_connector"; - protected static final int DEFAULT_SNAPSHOT_FETCH_SIZE = Integer.MIN_VALUE; - protected static final List BUILT_IN_DB_NAMES = - Collections.unmodifiableList( - Arrays.asList( - "information_schema", "mysql", "oceanbase", "LBACSYS", "ORAAUDITOR")); - - private final String compatibleMode; - private final String serverTimeZone; - - public OceanBaseConnectorConfig( - String compatibleMode, String serverTimeZone, Properties properties) { - super( - Configuration.from(properties), - LOGICAL_NAME, - Tables.TableFilter.fromPredicate( - tableId -> - "mysql".equalsIgnoreCase(compatibleMode) - ? !BUILT_IN_DB_NAMES.contains(tableId.catalog()) - : !BUILT_IN_DB_NAMES.contains(tableId.schema())), - TableId::identifier, - DEFAULT_SNAPSHOT_FETCH_SIZE, - "mysql".equalsIgnoreCase(compatibleMode) - ? ColumnFilterMode.CATALOG - : ColumnFilterMode.SCHEMA); - this.compatibleMode = compatibleMode; - this.serverTimeZone = serverTimeZone; - } - - public String getCompatibleMode() { - return compatibleMode; - } - - public String getServerTimeZone() { - return serverTimeZone; - } - - @Override - public String getConnectorName() { - return "oceanbase"; - } - - @Override - public String getContextName() { - return "OceanBase"; - } - - @Override - protected SourceInfoStructMaker getSourceInfoStructMaker(Version version) { - return new OceanBaseSourceInfoStructMaker(); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/connection/OceanBaseConnection.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/connection/OceanBaseConnection.java deleted file mode 100644 index 49ad532a13b..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/connection/OceanBaseConnection.java +++ /dev/null @@ -1,334 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase.source.connection; - -import org.apache.flink.cdc.connectors.oceanbase.utils.OceanBaseUtils; -import org.apache.flink.util.FlinkRuntimeException; - -import io.debezium.jdbc.JdbcConfiguration; -import io.debezium.jdbc.JdbcConnection; -import io.debezium.relational.Column; -import io.debezium.relational.TableId; -import io.debezium.relational.Tables; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.sql.DatabaseMetaData; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Timestamp; -import java.sql.Types; -import java.time.Duration; -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.Optional; -import java.util.Properties; -import java.util.Set; -import java.util.regex.Pattern; -import java.util.stream.Collectors; - -/** {@link JdbcConnection} extension to be used with OceanBase server. */ -public class OceanBaseConnection extends JdbcConnection { - - private static final Logger LOG = LoggerFactory.getLogger(OceanBaseConnection.class); - - private static final Properties DEFAULT_JDBC_PROPERTIES = initializeDefaultJdbcProperties(); - private static final String MYSQL_URL_PATTERN = - "jdbc:mysql://${hostname}:${port}/?connectTimeout=${connectTimeout}"; - private static final String OB_URL_PATTERN = - "jdbc:oceanbase://${hostname}:${port}/?connectTimeout=${connectTimeout}"; - - private static final int TYPE_BINARY_FLOAT = 100; - private static final int TYPE_BINARY_DOUBLE = 101; - private static final int TYPE_TIMESTAMP_WITH_TIME_ZONE = -101; - private static final int TYPE_TIMESTAMP_WITH_LOCAL_TIME_ZONE = -102; - private static final int TYPE_INTERVAL_YEAR_TO_MONTH = -103; - private static final int TYPE_INTERVAL_DAY_TO_SECOND = -104; - - private final String compatibleMode; - - public OceanBaseConnection( - String hostname, - Integer port, - String user, - String password, - Duration timeout, - String compatibleMode, - String jdbcDriver, - Properties jdbcProperties, - ClassLoader classLoader) { - super( - config(hostname, port, user, password, timeout), - JdbcConnection.patternBasedFactory( - formatJdbcUrl(jdbcDriver, jdbcProperties), jdbcDriver, classLoader), - getQuote(compatibleMode) + "", - getQuote(compatibleMode) + ""); - this.compatibleMode = compatibleMode; - } - - private static JdbcConfiguration config( - String hostname, Integer port, String user, String password, Duration timeout) { - return JdbcConfiguration.create() - .with("hostname", hostname) - .with("port", port) - .with("user", user) - .with("password", password) - .with("connectTimeout", timeout == null ? 30000 : timeout.toMillis()) - .build(); - } - - private static String formatJdbcUrl(String jdbcDriver, Properties jdbcProperties) { - Properties combinedProperties = new Properties(); - combinedProperties.putAll(DEFAULT_JDBC_PROPERTIES); - if (jdbcProperties != null) { - combinedProperties.putAll(jdbcProperties); - } - String urlPattern = - OceanBaseUtils.isOceanBaseDriver(jdbcDriver) ? OB_URL_PATTERN : MYSQL_URL_PATTERN; - StringBuilder jdbcUrlStringBuilder = new StringBuilder(urlPattern); - combinedProperties.forEach( - (key, value) -> { - jdbcUrlStringBuilder.append("&").append(key).append("=").append(value); - }); - return jdbcUrlStringBuilder.toString(); - } - - private static Properties initializeDefaultJdbcProperties() { - Properties defaultJdbcProperties = new Properties(); - defaultJdbcProperties.setProperty("useInformationSchema", "true"); - defaultJdbcProperties.setProperty("nullCatalogMeansCurrent", "false"); - defaultJdbcProperties.setProperty("useUnicode", "true"); - defaultJdbcProperties.setProperty("zeroDateTimeBehavior", "convertToNull"); - defaultJdbcProperties.setProperty("characterEncoding", "UTF-8"); - defaultJdbcProperties.setProperty("characterSetResults", "UTF-8"); - return defaultJdbcProperties; - } - - private static char getQuote(String compatibleMode) { - return "mysql".equalsIgnoreCase(compatibleMode) ? '`' : '"'; - } - - /** - * Get current timestamp number in seconds. - * - * @return current timestamp number. - * @throws SQLException If a database access error occurs. - */ - public long getCurrentTimestampS() throws SQLException { - try { - long globalTimestamp = getGlobalTimestamp(); - LOG.info("Global timestamp: {}", globalTimestamp); - return Long.parseLong(String.valueOf(globalTimestamp).substring(0, 10)); - } catch (Exception e) { - LOG.warn("Failed to get global timestamp, use local timestamp instead"); - } - return getCurrentTimestamp() - .orElseThrow(IllegalStateException::new) - .toInstant() - .getEpochSecond(); - } - - private long getGlobalTimestamp() throws SQLException { - String schema = "mysql".equalsIgnoreCase(compatibleMode) ? "oceanbase" : "SYS"; - return querySingleValue( - connection(), - "SELECT TS_VALUE FROM " + schema + ".V$OB_TIMESTAMP_SERVICE", - ps -> {}, - rs -> rs.getLong(1)); - } - - @Override - public Optional getCurrentTimestamp() throws SQLException { - return queryAndMap( - "mysql".equalsIgnoreCase(compatibleMode) - ? "SELECT CURRENT_TIMESTAMP" - : "SELECT CURRENT_TIMESTAMP FROM DUAL", - rs -> rs.next() ? Optional.of(rs.getTimestamp(1)) : Optional.empty()); - } - - /** - * Get table list by database name pattern and table name pattern. - * - * @param dbPattern Database name pattern. - * @param tbPattern Table name pattern. - * @return TableId list. - * @throws SQLException If a database access error occurs. - */ - public List getTables(String dbPattern, String tbPattern) throws SQLException { - List result = new ArrayList<>(); - DatabaseMetaData metaData = connection().getMetaData(); - switch (compatibleMode.toLowerCase()) { - case "mysql": - List dbNames = getResultList(metaData.getCatalogs(), "TABLE_CAT"); - dbNames = - dbNames.stream() - .filter(dbName -> Pattern.matches(dbPattern, dbName)) - .collect(Collectors.toList()); - for (String dbName : dbNames) { - List tableNames = - getResultList( - metaData.getTables(dbName, null, null, supportedTableTypes()), - "TABLE_NAME"); - tableNames.stream() - .filter(tbName -> Pattern.matches(tbPattern, tbName)) - .forEach(tbName -> result.add(new TableId(dbName, null, tbName))); - } - break; - case "oracle": - List schemaNames = getResultList(metaData.getSchemas(), "TABLE_SCHEM"); - schemaNames = - schemaNames.stream() - .filter(schemaName -> Pattern.matches(dbPattern, schemaName)) - .collect(Collectors.toList()); - for (String schemaName : schemaNames) { - List tableNames = - getResultList( - metaData.getTables( - null, schemaName, null, supportedTableTypes()), - "TABLE_NAME"); - tableNames.stream() - .filter(tbName -> Pattern.matches(tbPattern, tbName)) - .forEach(tbName -> result.add(new TableId(null, schemaName, tbName))); - } - break; - default: - throw new FlinkRuntimeException("Unsupported compatible mode: " + compatibleMode); - } - return result; - } - - private List getResultList(ResultSet resultSet, String columnName) throws SQLException { - List result = new ArrayList<>(); - while (resultSet.next()) { - result.add(resultSet.getString(columnName)); - } - return result; - } - - @Override - protected String[] supportedTableTypes() { - return new String[] {"TABLE"}; - } - - @Override - public String quotedTableIdString(TableId tableId) { - return tableId.toQuotedString(getQuote(compatibleMode)); - } - - public void readSchemaForCapturedTables( - Tables tables, - String databaseCatalog, - String schemaNamePattern, - Tables.ColumnNameFilter columnFilter, - boolean removeTablesNotFoundInJdbc, - Set capturedTables) - throws SQLException { - - Set tableIdsBefore = new HashSet<>(tables.tableIds()); - - DatabaseMetaData metadata = connection().getMetaData(); - Map> columnsByTable = new HashMap<>(); - - for (TableId tableId : capturedTables) { - try (ResultSet columnMetadata = - metadata.getColumns( - databaseCatalog, schemaNamePattern, tableId.table(), null)) { - while (columnMetadata.next()) { - // add all whitelisted columns - readTableColumn(columnMetadata, tableId, columnFilter) - .ifPresent( - column -> { - columnsByTable - .computeIfAbsent(tableId, t -> new ArrayList<>()) - .add(column.create()); - }); - } - } - } - - // Read the metadata for the primary keys ... - for (Map.Entry> tableEntry : columnsByTable.entrySet()) { - // First get the primary key information, which must be done for *each* table ... - List pkColumnNames = readPrimaryKeyNames(metadata, tableEntry.getKey()); - - // Then define the table ... - List columns = tableEntry.getValue(); - Collections.sort(columns); - tables.overwriteTable(tableEntry.getKey(), columns, pkColumnNames, null); - } - - if (removeTablesNotFoundInJdbc) { - // Remove any definitions for tables that were not found in the database metadata ... - tableIdsBefore.removeAll(columnsByTable.keySet()); - tableIdsBefore.forEach(tables::removeTable); - } - } - - @Override - protected int resolveNativeType(String typeName) { - String upperCaseTypeName = typeName.toUpperCase(); - if (upperCaseTypeName.startsWith("JSON")) { - return Types.VARCHAR; - } - if (upperCaseTypeName.startsWith("NCHAR")) { - return Types.NCHAR; - } - if (upperCaseTypeName.startsWith("NVARCHAR2")) { - return Types.NVARCHAR; - } - if (upperCaseTypeName.startsWith("TIMESTAMP")) { - if (upperCaseTypeName.contains("WITH TIME ZONE")) { - return TYPE_TIMESTAMP_WITH_TIME_ZONE; - } - if (upperCaseTypeName.contains("WITH LOCAL TIME ZONE")) { - return TYPE_TIMESTAMP_WITH_LOCAL_TIME_ZONE; - } - return Types.TIMESTAMP; - } - if (upperCaseTypeName.startsWith("INTERVAL")) { - if (upperCaseTypeName.contains("TO MONTH")) { - return TYPE_INTERVAL_YEAR_TO_MONTH; - } - if (upperCaseTypeName.contains("TO SECOND")) { - return TYPE_INTERVAL_DAY_TO_SECOND; - } - } - return Column.UNSET_INT_VALUE; - } - - @Override - protected int resolveJdbcType(int metadataJdbcType, int nativeType) { - switch (metadataJdbcType) { - case TYPE_BINARY_FLOAT: - return Types.REAL; - case TYPE_BINARY_DOUBLE: - return Types.DOUBLE; - case TYPE_TIMESTAMP_WITH_TIME_ZONE: - case TYPE_TIMESTAMP_WITH_LOCAL_TIME_ZONE: - case TYPE_INTERVAL_YEAR_TO_MONTH: - case TYPE_INTERVAL_DAY_TO_SECOND: - return Types.OTHER; - default: - return nativeType == Column.UNSET_INT_VALUE ? metadataJdbcType : nativeType; - } - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/converter/OceanBaseDeserializationConverterFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/converter/OceanBaseDeserializationConverterFactory.java deleted file mode 100644 index 3256b8a777f..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/converter/OceanBaseDeserializationConverterFactory.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase.source.converter; - -import org.apache.flink.cdc.debezium.table.DeserializationRuntimeConverter; -import org.apache.flink.cdc.debezium.table.DeserializationRuntimeConverterFactory; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.logical.LogicalType; - -import org.apache.kafka.connect.data.Schema; - -import java.time.ZoneId; -import java.util.Optional; - -/** Used to create {@link DeserializationRuntimeConverterFactory} specified to OceanBase. */ -public class OceanBaseDeserializationConverterFactory { - - public static DeserializationRuntimeConverterFactory instance() { - return new DeserializationRuntimeConverterFactory() { - - private static final long serialVersionUID = 1L; - - @Override - public Optional createUserDefinedConverter( - LogicalType logicalType, ZoneId serverTimeZone) { - switch (logicalType.getTypeRoot()) { - case ARRAY: - return createArrayConverter(); - default: - return Optional.empty(); - } - } - }; - } - - private static Optional createArrayConverter() { - return Optional.of( - new DeserializationRuntimeConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(Object dbzObj, Schema schema) throws Exception { - if (dbzObj instanceof String) { - String[] enums = ((String) dbzObj).split(","); - StringData[] elements = new StringData[enums.length]; - for (int i = 0; i < enums.length; i++) { - elements[i] = StringData.fromString(enums[i]); - } - return new GenericArrayData(elements); - } - throw new IllegalArgumentException( - String.format( - "Unable convert to Flink ARRAY type from unexpected value '%s'", - dbzObj)); - } - }); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/converter/OceanBaseValueConverters.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/converter/OceanBaseValueConverters.java deleted file mode 100644 index bf2a125db78..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/converter/OceanBaseValueConverters.java +++ /dev/null @@ -1,509 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase.source.converter; - -import org.apache.flink.cdc.connectors.oceanbase.source.config.OceanBaseConnectorConfig; - -import io.debezium.config.CommonConnectorConfig; -import io.debezium.data.Bits; -import io.debezium.data.SpecialValueDecimal; -import io.debezium.jdbc.JdbcValueConverters; -import io.debezium.relational.Column; -import io.debezium.relational.ValueConverter; -import io.debezium.time.MicroTimestamp; -import io.debezium.time.NanoTimestamp; -import org.apache.kafka.connect.data.Decimal; -import org.apache.kafka.connect.data.Field; -import org.apache.kafka.connect.data.SchemaBuilder; - -import java.math.BigDecimal; -import java.math.BigInteger; -import java.nio.ByteBuffer; -import java.sql.Blob; -import java.sql.Clob; -import java.sql.Date; -import java.sql.SQLException; -import java.sql.Time; -import java.sql.Timestamp; -import java.sql.Types; -import java.time.Instant; -import java.time.LocalDateTime; -import java.time.ZoneId; -import java.time.ZoneOffset; -import java.time.format.DateTimeFormatter; -import java.time.format.DateTimeFormatterBuilder; -import java.time.temporal.ChronoField; -import java.util.Locale; - -/** JdbcValueConverters for OceanBase. */ -public class OceanBaseValueConverters extends JdbcValueConverters { - - public static final String EMPTY_BLOB_FUNCTION = "EMPTY_BLOB()"; - public static final String EMPTY_CLOB_FUNCTION = "EMPTY_CLOB()"; - - private static final DateTimeFormatter TIMESTAMP_FORMATTER = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .appendPattern("yyyy-MM-dd HH:mm:ss") - .optionalStart() - .appendPattern(".") - .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, false) - .optionalEnd() - .toFormatter(); - - private static final DateTimeFormatter TIMESTAMP_AM_PM_SHORT_FORMATTER = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .appendPattern("dd-MMM-yy hh.mm.ss") - .optionalStart() - .appendPattern(".") - .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, false) - .optionalEnd() - .appendPattern(" a") - .toFormatter(Locale.ENGLISH); - - private final String compatibleMode; - private final String serverTimezone; - - public OceanBaseValueConverters(OceanBaseConnectorConfig connectorConfig) { - super( - connectorConfig.getDecimalMode(), - connectorConfig.getTemporalPrecisionMode(), - ZoneOffset.UTC, - x -> x, - BigIntUnsignedMode.PRECISE, - connectorConfig.binaryHandlingMode()); - this.compatibleMode = connectorConfig.getCompatibleMode(); - this.serverTimezone = connectorConfig.getServerTimeZone(); - } - - @Override - protected int getTimePrecision(Column column) { - if ("mysql".equalsIgnoreCase(compatibleMode)) { - return super.getTimePrecision(column); - } - return column.scale().orElse(0); - } - - protected boolean isUnsignedColumn(Column column) { - return column.typeName().toUpperCase().contains("UNSIGNED"); - } - - @Override - public SchemaBuilder schemaBuilder(Column column) { - logger.debug( - "Building schema for column {} of type {} named {} with constraints ({},{})", - column.name(), - column.jdbcType(), - column.typeName(), - column.length(), - column.scale()); - - switch (column.jdbcType()) { - case Types.BIT: - if (column.length() > 1) { - return Bits.builder(column.length()); - } - return SchemaBuilder.bool(); - case Types.TINYINT: - if (column.length() == 1) { - return SchemaBuilder.bool(); - } - if (isUnsignedColumn(column)) { - return SchemaBuilder.int16(); - } - return SchemaBuilder.int8(); - case Types.SMALLINT: - if (isUnsignedColumn(column)) { - return SchemaBuilder.int32(); - } - return SchemaBuilder.int16(); - case Types.INTEGER: - if (!column.typeName().toUpperCase().startsWith("MEDIUMINT") - && isUnsignedColumn(column)) { - return SchemaBuilder.int64(); - } - return SchemaBuilder.int32(); - case Types.BIGINT: - if (isUnsignedColumn(column)) { - return Decimal.builder(0); - } - return SchemaBuilder.int64(); - case Types.FLOAT: - return getDecimalSchema(column); - case Types.NUMERIC: - case Types.DECIMAL: - if ("mysql".equalsIgnoreCase(compatibleMode)) { - return getDecimalSchema(column); - } - return getNumericSchema(column); - case Types.REAL: - return SchemaBuilder.float32(); - case Types.DOUBLE: - return SchemaBuilder.float64(); - case Types.DATE: - if ("mysql".equalsIgnoreCase(compatibleMode)) { - if (column.typeName().equalsIgnoreCase("YEAR")) { - return io.debezium.time.Year.builder(); - } - if (adaptiveTimePrecisionMode || adaptiveTimeMicrosecondsPrecisionMode) { - return io.debezium.time.Date.builder(); - } - return org.apache.kafka.connect.data.Date.builder(); - } - return getTimestampSchema(column); - case Types.TIME: - if (adaptiveTimeMicrosecondsPrecisionMode) { - return io.debezium.time.MicroTime.builder(); - } - if (adaptiveTimePrecisionMode) { - if (getTimePrecision(column) <= 3) { - return io.debezium.time.Time.builder(); - } - if (getTimePrecision(column) <= 6) { - return io.debezium.time.MicroTime.builder(); - } - return io.debezium.time.NanoTime.builder(); - } - return org.apache.kafka.connect.data.Time.builder(); - case Types.TIMESTAMP: - return getTimestampSchema(column); - case Types.CHAR: - case Types.VARCHAR: - case Types.LONGVARCHAR: - case Types.NCHAR: - case Types.NVARCHAR: - case Types.CLOB: - return SchemaBuilder.string(); - case Types.BINARY: - case Types.VARBINARY: - case Types.LONGVARBINARY: - case Types.BLOB: - return binaryMode.getSchema(); - default: - return super.schemaBuilder(column); - } - } - - protected SchemaBuilder getNumericSchema(Column column) { - if (column.scale().isPresent()) { - int scale = column.scale().get(); - if (scale <= 0) { - int width = column.length() - scale; - if (width < 3) { - return SchemaBuilder.int8(); - } else if (width < 5) { - return SchemaBuilder.int16(); - } else if (width < 10) { - return SchemaBuilder.int32(); - } else if (width < 19) { - return SchemaBuilder.int64(); - } - } - } - return getDecimalSchema(column); - } - - protected SchemaBuilder getDecimalSchema(Column column) { - return SpecialValueDecimal.builder(decimalMode, column.length(), column.scale().orElse(0)); - } - - protected SchemaBuilder getTimestampSchema(Column column) { - if (adaptiveTimePrecisionMode || adaptiveTimeMicrosecondsPrecisionMode) { - if (getTimePrecision(column) <= 3) { - return io.debezium.time.Timestamp.builder(); - } - if (getTimePrecision(column) <= 6) { - return MicroTimestamp.builder(); - } - return NanoTimestamp.builder(); - } - return org.apache.kafka.connect.data.Timestamp.builder(); - } - - @Override - public ValueConverter converter(Column column, Field fieldDefn) { - switch (column.jdbcType()) { - case Types.BIT: - return convertBits(column, fieldDefn); - case Types.TINYINT: - if (column.length() == 1) { - return data -> convertBit(column, fieldDefn, data); - } - if (isUnsignedColumn(column)) { - return data -> convertSmallInt(column, fieldDefn, data); - } - return data -> convertTinyInt(column, fieldDefn, data); - case Types.SMALLINT: - if (isUnsignedColumn(column)) { - return data -> convertInteger(column, fieldDefn, data); - } - return data -> convertSmallInt(column, fieldDefn, data); - case Types.INTEGER: - if (column.typeName().toUpperCase().startsWith("MEDIUMINT")) { - return data -> convertInteger(column, fieldDefn, data); - } - if (isUnsignedColumn(column)) { - return data -> convertBigInt(column, fieldDefn, data); - } - return data -> convertInteger(column, fieldDefn, data); - case Types.BIGINT: - if (isUnsignedColumn(column)) { - switch (bigIntUnsignedMode) { - case LONG: - return (data) -> convertBigInt(column, fieldDefn, data); - case PRECISE: - return (data) -> convertUnsignedBigint(column, fieldDefn, data); - } - } - return (data) -> convertBigInt(column, fieldDefn, data); - case Types.FLOAT: - return data -> convertDecimal(column, fieldDefn, data); - case Types.NUMERIC: - case Types.DECIMAL: - if ("mysql".equalsIgnoreCase(compatibleMode)) { - return data -> convertDecimal(column, fieldDefn, data); - } - return data -> convertNumeric(column, fieldDefn, data); - case Types.REAL: - return data -> convertReal(column, fieldDefn, data); - case Types.DOUBLE: - return data -> convertDouble(column, fieldDefn, data); - case Types.DATE: - if ("mysql".equalsIgnoreCase(compatibleMode)) { - if (column.typeName().equalsIgnoreCase("YEAR")) { - return (data) -> convertYearToInt(column, fieldDefn, data); - } - if (adaptiveTimePrecisionMode || adaptiveTimeMicrosecondsPrecisionMode) { - return (data) -> convertDateToEpochDays(column, fieldDefn, data); - } - return (data) -> convertDateToEpochDaysAsDate(column, fieldDefn, data); - } - return (data) -> convertTimestamp(column, fieldDefn, data); - case Types.TIME: - return (data) -> convertTime(column, fieldDefn, data); - case Types.TIMESTAMP: - return data -> convertTimestamp(column, fieldDefn, data); - case Types.CHAR: - case Types.VARCHAR: - case Types.LONGVARCHAR: - case Types.NCHAR: - case Types.NVARCHAR: - case Types.CLOB: - return data -> convertString(column, fieldDefn, data); - case Types.BINARY: - case Types.VARBINARY: - case Types.LONGVARBINARY: - case Types.BLOB: - return (data) -> convertBinary(column, fieldDefn, data, binaryMode); - default: - return super.converter(column, fieldDefn); - } - } - - @Override - protected Object convertBits(Column column, Field fieldDefn, Object data, int numBytes) { - if (data instanceof String) { - return ByteBuffer.allocate(numBytes).putLong(Long.parseLong((String) data)).array(); - } - return super.convertBits(column, fieldDefn, data, numBytes); - } - - @Override - protected Object convertBit(Column column, Field fieldDefn, Object data) { - if (data instanceof String) { - return Boolean.parseBoolean((String) data) || "1".equals(data); - } - return super.convertBit(column, fieldDefn, data); - } - - @Override - protected Object convertTinyInt(Column column, Field fieldDefn, Object data) { - if (data instanceof String) { - return Byte.parseByte((String) data); - } - if (data instanceof Number) { - return ((Number) data).byteValue(); - } - throw new IllegalArgumentException( - "Unexpected value for JDBC type " - + column.jdbcType() - + " and column " - + column - + ": class=" - + data.getClass()); - } - - @Override - protected Object convertBigInt(Column column, Field fieldDefn, Object data) { - if (data instanceof String) { - return new BigInteger((String) data).longValue(); - } - return super.convertBigInt(column, fieldDefn, data); - } - - protected Object convertUnsignedBigint(Column column, Field fieldDefn, Object data) { - if (data instanceof String) { - return new BigDecimal((String) data); - } - if (data instanceof BigInteger) { - return new BigDecimal((BigInteger) data); - } - return convertDecimal(column, fieldDefn, data); - } - - @Override - protected Object convertReal(Column column, Field fieldDefn, Object data) { - if (data instanceof String) { - return Float.parseFloat((String) data); - } - return super.convertReal(column, fieldDefn, data); - } - - @Override - protected Object convertDouble(Column column, Field fieldDefn, Object data) { - if (data instanceof String) { - return Double.parseDouble((String) data); - } - return super.convertDouble(column, fieldDefn, data); - } - - @Override - protected Object convertNumeric(Column column, Field fieldDefn, Object data) { - if (column.scale().isPresent()) { - int scale = column.scale().get(); - - if (scale <= 0) { - int width = column.length() - scale; - if (width < 3) { - return convertTinyInt(column, fieldDefn, data); - } else if (width < 5) { - return convertSmallInt(column, fieldDefn, data); - } else if (width < 10) { - return convertInteger(column, fieldDefn, data); - } else if (width < 19) { - return convertBigInt(column, fieldDefn, data); - } - } - } - return convertDecimal(column, fieldDefn, data); - } - - protected Object convertYearToInt(Column column, Field fieldDefn, Object data) { - if (data instanceof Date) { - return ((Date) data).toLocalDate().getYear(); - } - return convertInteger(column, fieldDefn, data); - } - - @Override - protected Object convertDateToEpochDays(Column column, Field fieldDefn, Object data) { - if (data instanceof String) { - data = Date.valueOf((String) data); - } - return super.convertDateToEpochDays(column, fieldDefn, data); - } - - @Override - protected Object convertDateToEpochDaysAsDate(Column column, Field fieldDefn, Object data) { - if (data instanceof String) { - data = Date.valueOf((String) data); - } - return super.convertDateToEpochDaysAsDate(column, fieldDefn, data); - } - - @Override - protected Object convertTime(Column column, Field fieldDefn, Object data) { - if (data instanceof String) { - data = Time.valueOf((String) data); - } - return super.convertTime(column, fieldDefn, data); - } - - protected Object convertTimestamp(Column column, Field fieldDefn, Object data) { - if (data instanceof String) { - if ("mysql".equalsIgnoreCase(compatibleMode)) { - data = Timestamp.valueOf(((String) data).trim()); - } else { - data = resolveTimestampStringAsInstant((String) data); - } - } - if (adaptiveTimePrecisionMode || adaptiveTimeMicrosecondsPrecisionMode) { - if (getTimePrecision(column) <= 3) { - return convertTimestampToEpochMillis(column, fieldDefn, data); - } - if (getTimePrecision(column) <= 6) { - return convertTimestampToEpochMicros(column, fieldDefn, data); - } - return convertTimestampToEpochNanos(column, fieldDefn, data); - } - return convertTimestampToEpochMillisAsDate(column, fieldDefn, data); - } - - protected Instant resolveTimestampStringAsInstant(String dateText) { - LocalDateTime dateTime; - if (dateText.indexOf(" AM") > 0 || dateText.indexOf(" PM") > 0) { - dateTime = LocalDateTime.from(TIMESTAMP_AM_PM_SHORT_FORMATTER.parse(dateText.trim())); - } else { - dateTime = LocalDateTime.from(TIMESTAMP_FORMATTER.parse(dateText.trim())); - } - return dateTime.atZone(ZoneId.of(serverTimezone)).toInstant(); - } - - @Override - protected Object convertString(Column column, Field fieldDefn, Object data) { - if (data instanceof Clob) { - try { - Clob clob = (Clob) data; - return clob.getSubString(1, (int) clob.length()); - } catch (SQLException e) { - throw new RuntimeException("Couldn't convert value for column " + column.name(), e); - } - } - if (data instanceof String) { - String s = (String) data; - if (EMPTY_CLOB_FUNCTION.equals(s)) { - return column.isOptional() ? null : ""; - } - } - return super.convertString(column, fieldDefn, data); - } - - @Override - protected Object convertBinary( - Column column, - Field fieldDefn, - Object data, - CommonConnectorConfig.BinaryHandlingMode mode) { - try { - if (data instanceof Blob) { - Blob blob = (Blob) data; - data = blob.getBytes(1, Long.valueOf(blob.length()).intValue()); - } - if (data instanceof String) { - String str = (String) data; - if (EMPTY_BLOB_FUNCTION.equals(str)) { - data = column.isOptional() ? null : ""; - } - } - return super.convertBinary(column, fieldDefn, data, mode); - } catch (SQLException e) { - throw new RuntimeException("Couldn't convert value for column " + column.name(), e); - } - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/offset/OceanBaseSourceInfo.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/offset/OceanBaseSourceInfo.java deleted file mode 100644 index 1319b1a22c8..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/offset/OceanBaseSourceInfo.java +++ /dev/null @@ -1,101 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase.source.offset; - -import org.apache.flink.cdc.connectors.oceanbase.source.config.OceanBaseConnectorConfig; - -import io.debezium.connector.common.BaseSourceInfo; -import io.debezium.relational.TableId; - -import java.time.Instant; -import java.util.Collections; -import java.util.Objects; -import java.util.Set; -import java.util.stream.Collectors; - -/** OceanBase source info. */ -public class OceanBaseSourceInfo extends BaseSourceInfo { - - public static final String TENANT_KEY = "tenant"; - public static final String TRANSACTION_ID_KEY = "transaction_id"; - - private final String tenant; - - private Instant sourceTime; - private Set tableIds; - private String transactionId; - - public OceanBaseSourceInfo(OceanBaseConnectorConfig config, String tenant) { - super(config); - this.tenant = tenant; - } - - public String tenant() { - return tenant; - } - - @Override - protected Instant timestamp() { - return sourceTime; - } - - public void setSourceTime(Instant sourceTime) { - this.sourceTime = sourceTime; - } - - public void beginTransaction(String transactionId) { - this.transactionId = transactionId; - } - - public void commitTransaction() { - this.transactionId = null; - } - - public String transactionId() { - return transactionId; - } - - public void tableEvent(TableId tableId) { - this.tableIds = Collections.singleton(tableId); - } - - @Override - protected String database() { - return (tableIds != null) ? tableIds.iterator().next().catalog() : null; - } - - public String tableSchema() { - return (tableIds == null || tableIds.isEmpty()) - ? null - : tableIds.stream() - .filter(Objects::nonNull) - .map(TableId::schema) - .filter(Objects::nonNull) - .distinct() - .collect(Collectors.joining(",")); - } - - public String table() { - return (tableIds == null || tableIds.isEmpty()) - ? null - : tableIds.stream() - .filter(Objects::nonNull) - .map(TableId::table) - .collect(Collectors.joining(",")); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/offset/OceanBaseSourceInfoStructMaker.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/offset/OceanBaseSourceInfoStructMaker.java deleted file mode 100644 index 604ef1e77d5..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/offset/OceanBaseSourceInfoStructMaker.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase.source.offset; - -import io.debezium.connector.SourceInfoStructMaker; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.data.Struct; - -import java.time.Instant; - -/** The {@link SourceInfoStructMaker} implementation for OceanBase. */ -public class OceanBaseSourceInfoStructMaker implements SourceInfoStructMaker { - private final Schema schema; - - public OceanBaseSourceInfoStructMaker() { - this.schema = - SchemaBuilder.struct() - .field(OceanBaseSourceInfo.TABLE_NAME_KEY, Schema.STRING_SCHEMA) - .field(OceanBaseSourceInfo.TIMESTAMP_KEY, Schema.INT64_SCHEMA) - .field(OceanBaseSourceInfo.TENANT_KEY, Schema.OPTIONAL_STRING_SCHEMA) - .field(OceanBaseSourceInfo.DATABASE_NAME_KEY, Schema.OPTIONAL_STRING_SCHEMA) - .field(OceanBaseSourceInfo.SCHEMA_NAME_KEY, Schema.OPTIONAL_STRING_SCHEMA) - .field( - OceanBaseSourceInfo.TRANSACTION_ID_KEY, - Schema.OPTIONAL_STRING_SCHEMA) - .build(); - } - - @Override - public Schema schema() { - return schema; - } - - @Override - public Struct struct(OceanBaseSourceInfo sourceInfo) { - Struct source = new Struct(schema); - source.put(OceanBaseSourceInfo.TABLE_NAME_KEY, sourceInfo.table()); - - Instant timestamp = sourceInfo.timestamp(); - source.put( - OceanBaseSourceInfo.TIMESTAMP_KEY, - timestamp != null ? timestamp.toEpochMilli() : 0); - - if (sourceInfo.tenant() != null) { - source.put(OceanBaseSourceInfo.TENANT_KEY, sourceInfo.tenant()); - } - if (sourceInfo.database() != null) { - source.put(OceanBaseSourceInfo.DATABASE_NAME_KEY, sourceInfo.database()); - } - if (sourceInfo.tableSchema() != null) { - source.put(OceanBaseSourceInfo.SCHEMA_NAME_KEY, sourceInfo.tableSchema()); - } - if (sourceInfo.transactionId() != null) { - source.put(OceanBaseSourceInfo.TRANSACTION_ID_KEY, sourceInfo.transactionId()); - } - return source; - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseDatabaseSchema.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseDatabaseSchema.java deleted file mode 100644 index 094937bad85..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseDatabaseSchema.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase.source.schema; - -import org.apache.flink.cdc.connectors.oceanbase.source.config.OceanBaseConnectorConfig; -import org.apache.flink.cdc.connectors.oceanbase.source.converter.OceanBaseValueConverters; - -import io.debezium.relational.RelationalDatabaseSchema; -import io.debezium.relational.TableSchemaBuilder; -import io.debezium.relational.Tables; -import io.debezium.schema.TopicSelector; - -/** OceanBase database schema. */ -public class OceanBaseDatabaseSchema extends RelationalDatabaseSchema { - - public OceanBaseDatabaseSchema( - OceanBaseConnectorConfig connectorConfig, - Tables.TableFilter tableFilter, - boolean tableIdCaseInsensitive) { - super( - connectorConfig, - TopicSelector.defaultSelector( - connectorConfig, - (tableId, prefix, delimiter) -> - String.join(delimiter, prefix, tableId.identifier())), - tableFilter, - connectorConfig.getColumnFilter(), - new TableSchemaBuilder( - new OceanBaseValueConverters(connectorConfig), - connectorConfig.schemaNameAdjustmentMode().createAdjuster(), - connectorConfig.customConverterRegistry(), - connectorConfig.getSourceInfoStructMaker().schema(), - connectorConfig.getSanitizeFieldNames(), - false), - tableIdCaseInsensitive, - connectorConfig.getKeyMapper()); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseSchema.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseSchema.java deleted file mode 100644 index b77cc64abd6..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseSchema.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase.source.schema; - -import org.apache.flink.cdc.connectors.oceanbase.source.connection.OceanBaseConnection; -import org.apache.flink.util.FlinkRuntimeException; - -import io.debezium.jdbc.JdbcConnection; -import io.debezium.relational.Table; -import io.debezium.relational.TableId; -import io.debezium.relational.Tables; -import io.debezium.relational.history.TableChanges; - -import java.sql.SQLException; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - -/** A component used to get schema by table path. */ -public class OceanBaseSchema { - - private final Map schemasByTableId; - - public OceanBaseSchema() { - this.schemasByTableId = new HashMap<>(); - } - - public TableChanges.TableChange getTableSchema(JdbcConnection connection, TableId tableId) { - TableChanges.TableChange schema = schemasByTableId.get(tableId); - if (schema == null) { - schema = readTableSchema(connection, tableId); - schemasByTableId.put(tableId, schema); - } - return schema; - } - - private TableChanges.TableChange readTableSchema(JdbcConnection jdbc, TableId tableId) { - OceanBaseConnection connection = (OceanBaseConnection) jdbc; - Set tableIdSet = new HashSet<>(); - tableIdSet.add(tableId); - - final Map tableChangeMap = new HashMap<>(); - Tables tables = new Tables(); - tables.overwriteTable(tables.editOrCreateTable(tableId).create()); - - try { - connection.readSchemaForCapturedTables( - tables, tableId.catalog(), tableId.schema(), null, false, tableIdSet); - Table table = tables.forTable(tableId); - TableChanges.TableChange tableChange = - new TableChanges.TableChange(TableChanges.TableChangeType.CREATE, table); - tableChangeMap.put(tableId, tableChange); - } catch (SQLException e) { - throw new FlinkRuntimeException( - String.format("Failed to read schema for table %s ", tableId), e); - } - - if (!tableChangeMap.containsKey(tableId)) { - throw new FlinkRuntimeException( - String.format("Can't obtain schema for table %s ", tableId)); - } - - return tableChangeMap.get(tableId); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMySqlTableSourceFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMySqlTableSourceFactory.java new file mode 100644 index 00000000000..279eb014bbc --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMySqlTableSourceFactory.java @@ -0,0 +1,45 @@ +/* + * 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.flink.cdc.connectors.oceanbase.table; + +import org.apache.flink.cdc.connectors.mysql.table.MySqlTableSourceFactory; + +/** + * Factory class for creating table sources that capture data changes from OceanBase databases. + * + *

    This factory extends {@link MySqlTableSourceFactory} to reuse MySQL CDC connector. + * + *

    Key characteristics: + * + *

      + *
    • Identified by factory identifier "oceanbase-cdc" + *
    • Compatible with OceanBase Binlog Service + *
    + * + * @see org.apache.flink.cdc.connectors.mysql.table.MySqlTableSourceFactory Base MySQL + * implementation + */ +public class OceanBaseMySqlTableSourceFactory extends MySqlTableSourceFactory { + + private static final String IDENTIFIER = "oceanbase-cdc"; + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseReadableMetadata.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseReadableMetadata.java deleted file mode 100644 index 12a6282f4fe..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseReadableMetadata.java +++ /dev/null @@ -1,140 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase.table; - -import org.apache.flink.cdc.connectors.oceanbase.source.offset.OceanBaseSourceInfo; -import org.apache.flink.cdc.debezium.table.MetadataConverter; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.DataType; - -import io.debezium.data.Envelope; -import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.source.SourceRecord; - -/** Defines the supported metadata columns for {@link OceanBaseTableSource}. */ -public enum OceanBaseReadableMetadata { - - /** Name of the tenant that contains the row. */ - TENANT( - "tenant_name", - DataTypes.STRING().nullable(), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object read(SourceRecord record) { - Struct messageStruct = (Struct) record.value(); - Struct sourceStruct = messageStruct.getStruct(Envelope.FieldName.SOURCE); - return StringData.fromString( - sourceStruct.getString(OceanBaseSourceInfo.TENANT_KEY)); - } - }), - - /** Name of the database that contains the row. */ - DATABASE( - "database_name", - DataTypes.STRING().nullable(), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object read(SourceRecord record) { - Struct messageStruct = (Struct) record.value(); - Struct sourceStruct = messageStruct.getStruct(Envelope.FieldName.SOURCE); - return StringData.fromString( - sourceStruct.getString(OceanBaseSourceInfo.DATABASE_NAME_KEY)); - } - }), - - /** Name of the schema that contains the row. */ - SCHEMA( - "schema_name", - DataTypes.STRING().nullable(), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object read(SourceRecord record) { - Struct messageStruct = (Struct) record.value(); - Struct sourceStruct = messageStruct.getStruct(Envelope.FieldName.SOURCE); - return StringData.fromString( - sourceStruct.getString(OceanBaseSourceInfo.SCHEMA_NAME_KEY)); - } - }), - - /** Name of the table that contains the row. */ - TABLE( - "table_name", - DataTypes.STRING().notNull(), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object read(SourceRecord record) { - Struct messageStruct = (Struct) record.value(); - Struct sourceStruct = messageStruct.getStruct(Envelope.FieldName.SOURCE); - return StringData.fromString( - sourceStruct.getString(OceanBaseSourceInfo.TABLE_NAME_KEY)); - } - }), - - /** - * It indicates the time that the change was made in the database. If the record is read from - * snapshot of the table instead of the change stream, the value is always 0. - */ - OP_TS( - "op_ts", - DataTypes.TIMESTAMP_LTZ(3).notNull(), - new MetadataConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object read(SourceRecord record) { - Struct messageStruct = (Struct) record.value(); - Struct sourceStruct = messageStruct.getStruct(Envelope.FieldName.SOURCE); - return TimestampData.fromEpochMillis( - sourceStruct.getInt64(OceanBaseSourceInfo.TIMESTAMP_KEY)); - } - }); - - private final String key; - - private final DataType dataType; - - private final MetadataConverter converter; - - OceanBaseReadableMetadata(String key, DataType dataType, MetadataConverter converter) { - this.key = key; - this.dataType = dataType; - this.converter = converter; - } - - public String getKey() { - return key; - } - - public DataType getDataType() { - return dataType; - } - - public MetadataConverter getConverter() { - return converter; - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSource.java deleted file mode 100644 index e3084b5fb83..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSource.java +++ /dev/null @@ -1,333 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase.table; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.cdc.connectors.base.options.StartupOptions; -import org.apache.flink.cdc.connectors.oceanbase.OceanBaseSource; -import org.apache.flink.cdc.connectors.oceanbase.source.converter.OceanBaseDeserializationConverterFactory; -import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; -import org.apache.flink.cdc.debezium.table.MetadataConverter; -import org.apache.flink.cdc.debezium.table.RowDataDebeziumDeserializeSchema; -import org.apache.flink.table.catalog.ResolvedSchema; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.connector.source.ScanTableSource; -import org.apache.flink.table.connector.source.SourceFunctionProvider; -import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; - -import java.time.Duration; -import java.time.ZoneId; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Properties; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** A {@link DynamicTableSource} implementation for OceanBase. */ -public class OceanBaseTableSource implements ScanTableSource, SupportsReadingMetadata { - - private final ResolvedSchema physicalSchema; - - private final StartupOptions startupOptions; - private final String username; - private final String password; - private final String tenantName; - private final String databaseName; - private final String tableName; - private final String tableList; - private final Duration connectTimeout; - private final String serverTimeZone; - - private final String hostname; - private final int port; - private final String compatibleMode; - private final String jdbcDriver; - private final Properties jdbcProperties; - - private final String logProxyHost; - private final Integer logProxyPort; - private final String logProxyClientId; - private final Long startupTimestamp; - private final String rsList; - private final String configUrl; - private final String workingMode; - private final Properties obcdcProperties; - private final Properties debeziumProperties; - - // -------------------------------------------------------------------------------------------- - // Mutable attributes - // -------------------------------------------------------------------------------------------- - - /** Data type that describes the final output of the source. */ - protected DataType producedDataType; - - /** Metadata that is appended at the end of a physical source row. */ - protected List metadataKeys; - - public OceanBaseTableSource( - ResolvedSchema physicalSchema, - StartupOptions startupOptions, - String username, - String password, - String tenantName, - String databaseName, - String tableName, - String tableList, - String serverTimeZone, - Duration connectTimeout, - String hostname, - int port, - String compatibleMode, - String jdbcDriver, - Properties jdbcProperties, - String logProxyHost, - Integer logProxyPort, - String logProxyClientId, - Long startupTimestamp, - String rsList, - String configUrl, - String workingMode, - Properties obcdcProperties, - Properties debeziumProperties) { - this.physicalSchema = physicalSchema; - this.startupOptions = checkNotNull(startupOptions); - this.username = checkNotNull(username); - this.password = checkNotNull(password); - this.tenantName = tenantName; - this.databaseName = databaseName; - this.tableName = tableName; - this.tableList = tableList; - this.serverTimeZone = serverTimeZone; - this.connectTimeout = connectTimeout; - this.hostname = checkNotNull(hostname); - this.port = port; - this.compatibleMode = compatibleMode; - this.jdbcDriver = jdbcDriver; - this.jdbcProperties = jdbcProperties; - this.logProxyHost = logProxyHost; - this.logProxyPort = logProxyPort; - this.logProxyClientId = logProxyClientId; - this.startupTimestamp = startupTimestamp; - this.rsList = rsList; - this.configUrl = configUrl; - this.workingMode = workingMode; - this.obcdcProperties = obcdcProperties; - this.debeziumProperties = debeziumProperties; - - this.producedDataType = physicalSchema.toPhysicalRowDataType(); - this.metadataKeys = Collections.emptyList(); - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.all(); - } - - @Override - public ScanRuntimeProvider getScanRuntimeProvider(ScanContext context) { - RowType physicalDataType = - (RowType) physicalSchema.toPhysicalRowDataType().getLogicalType(); - MetadataConverter[] metadataConverters = getMetadataConverters(); - TypeInformation resultTypeInfo = context.createTypeInformation(producedDataType); - - DebeziumDeserializationSchema deserializer = - RowDataDebeziumDeserializeSchema.newBuilder() - .setPhysicalRowType(physicalDataType) - .setMetadataConverters(metadataConverters) - .setResultTypeInfo(resultTypeInfo) - .setServerTimeZone( - serverTimeZone == null - ? ZoneId.systemDefault() - : ZoneId.of(serverTimeZone)) - .setUserDefinedConverterFactory( - OceanBaseDeserializationConverterFactory.instance()) - .build(); - - OceanBaseSource.Builder builder = - OceanBaseSource.builder() - .startupOptions(startupOptions) - .username(username) - .password(password) - .tenantName(tenantName) - .databaseName(databaseName) - .tableName(tableName) - .tableList(tableList) - .serverTimeZone(serverTimeZone) - .connectTimeout(connectTimeout) - .hostname(hostname) - .port(port) - .compatibleMode(compatibleMode) - .jdbcDriver(jdbcDriver) - .jdbcProperties(jdbcProperties) - .logProxyHost(logProxyHost) - .logProxyPort(logProxyPort) - .logProxyClientId(logProxyClientId) - .startupTimestamp(startupTimestamp) - .rsList(rsList) - .configUrl(configUrl) - .workingMode(workingMode) - .obcdcProperties(obcdcProperties) - .debeziumProperties(debeziumProperties) - .deserializer(deserializer); - return SourceFunctionProvider.of(builder.build(), false); - } - - protected MetadataConverter[] getMetadataConverters() { - if (metadataKeys.isEmpty()) { - return new MetadataConverter[0]; - } - return metadataKeys.stream() - .map( - key -> - Stream.of(OceanBaseReadableMetadata.values()) - .filter(m -> m.getKey().equals(key)) - .findFirst() - .orElseThrow(IllegalStateException::new)) - .map(OceanBaseReadableMetadata::getConverter) - .toArray(MetadataConverter[]::new); - } - - @Override - public Map listReadableMetadata() { - return Stream.of(OceanBaseReadableMetadata.values()) - .collect( - Collectors.toMap( - OceanBaseReadableMetadata::getKey, - OceanBaseReadableMetadata::getDataType)); - } - - @Override - public void applyReadableMetadata(List metadataKeys, DataType producedDataType) { - this.metadataKeys = metadataKeys; - this.producedDataType = producedDataType; - } - - @Override - public DynamicTableSource copy() { - OceanBaseTableSource source = - new OceanBaseTableSource( - physicalSchema, - startupOptions, - username, - password, - tenantName, - databaseName, - tableName, - tableList, - serverTimeZone, - connectTimeout, - hostname, - port, - compatibleMode, - jdbcDriver, - jdbcProperties, - logProxyHost, - logProxyPort, - logProxyClientId, - startupTimestamp, - rsList, - configUrl, - workingMode, - obcdcProperties, - debeziumProperties); - source.metadataKeys = metadataKeys; - source.producedDataType = producedDataType; - return source; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - OceanBaseTableSource that = (OceanBaseTableSource) o; - return Objects.equals(this.physicalSchema, that.physicalSchema) - && Objects.equals(this.startupOptions, that.startupOptions) - && Objects.equals(this.username, that.username) - && Objects.equals(this.password, that.password) - && Objects.equals(this.tenantName, that.tenantName) - && Objects.equals(this.databaseName, that.databaseName) - && Objects.equals(this.tableName, that.tableName) - && Objects.equals(this.tableList, that.tableList) - && Objects.equals(this.serverTimeZone, that.serverTimeZone) - && Objects.equals(this.connectTimeout, that.connectTimeout) - && Objects.equals(this.hostname, that.hostname) - && Objects.equals(this.port, that.port) - && Objects.equals(this.compatibleMode, that.compatibleMode) - && Objects.equals(this.jdbcDriver, that.jdbcDriver) - && Objects.equals(this.jdbcProperties, that.jdbcProperties) - && Objects.equals(this.logProxyHost, that.logProxyHost) - && Objects.equals(this.logProxyPort, that.logProxyPort) - && Objects.equals(this.logProxyClientId, that.logProxyClientId) - && Objects.equals(this.startupTimestamp, that.startupTimestamp) - && Objects.equals(this.rsList, that.rsList) - && Objects.equals(this.configUrl, that.configUrl) - && Objects.equals(this.workingMode, that.workingMode) - && Objects.equals(this.obcdcProperties, that.obcdcProperties) - && Objects.equals(this.debeziumProperties, that.debeziumProperties) - && Objects.equals(this.producedDataType, that.producedDataType) - && Objects.equals(this.metadataKeys, that.metadataKeys); - } - - @Override - public int hashCode() { - return Objects.hash( - physicalSchema, - startupOptions, - username, - password, - tenantName, - databaseName, - tableName, - tableList, - serverTimeZone, - connectTimeout, - hostname, - port, - compatibleMode, - jdbcDriver, - jdbcProperties, - logProxyHost, - logProxyPort, - logProxyClientId, - startupTimestamp, - rsList, - configUrl, - workingMode, - obcdcProperties, - debeziumProperties, - producedDataType, - metadataKeys); - } - - @Override - public String asSummaryString() { - return "OceanBase-CDC"; - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSourceFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSourceFactory.java deleted file mode 100644 index a236c0eb667..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSourceFactory.java +++ /dev/null @@ -1,347 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase.table; - -import org.apache.flink.cdc.connectors.base.options.StartupOptions; -import org.apache.flink.cdc.connectors.oceanbase.utils.OceanBaseUtils; -import org.apache.flink.cdc.connectors.oceanbase.utils.OptionUtils; -import org.apache.flink.cdc.debezium.table.DebeziumOptions; -import org.apache.flink.cdc.debezium.utils.JdbcUrlUtils; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.catalog.ResolvedSchema; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.factories.DynamicTableSourceFactory; -import org.apache.flink.table.factories.FactoryUtil; - -import java.time.Duration; -import java.util.HashSet; -import java.util.Map; -import java.util.Objects; -import java.util.Properties; -import java.util.Set; - -import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_STARTUP_MODE; - -/** Factory for creating configured instance of {@link OceanBaseTableSource}. */ -public class OceanBaseTableSourceFactory implements DynamicTableSourceFactory { - - private static final String IDENTIFIER = "oceanbase-cdc"; - - public static final ConfigOption USERNAME = - ConfigOptions.key("username") - .stringType() - .noDefaultValue() - .withDescription("Username to be used when connecting to OceanBase."); - - public static final ConfigOption PASSWORD = - ConfigOptions.key("password") - .stringType() - .noDefaultValue() - .withDescription("Password to be used when connecting to OceanBase."); - - public static final ConfigOption TENANT_NAME = - ConfigOptions.key("tenant-name") - .stringType() - .noDefaultValue() - .withDescription("Tenant name of OceanBase to monitor."); - - public static final ConfigOption DATABASE_NAME = - ConfigOptions.key("database-name") - .stringType() - .noDefaultValue() - .withDescription( - "Database name of OceanBase to monitor, should be regular expression. Only can be used with 'initial' mode."); - - public static final ConfigOption TABLE_NAME = - ConfigOptions.key("table-name") - .stringType() - .noDefaultValue() - .withDescription( - "Table name of OceanBase to monitor, should be regular expression. Only can be used with 'initial' mode."); - - public static final ConfigOption TABLE_LIST = - ConfigOptions.key("table-list") - .stringType() - .noDefaultValue() - .withDescription( - "List of full names of tables, separated by commas, e.g. \"db1.table1, db2.table2\"."); - - public static final ConfigOption SERVER_TIME_ZONE = - ConfigOptions.key("server-time-zone") - .stringType() - .defaultValue("+00:00") - .withDescription("The session time zone in database server."); - - public static final ConfigOption CONNECT_TIMEOUT = - ConfigOptions.key("connect.timeout") - .durationType() - .defaultValue(Duration.ofSeconds(30)) - .withDescription( - "The maximum time that the connector should wait after trying to connect to the database server or log proxy server before timing out."); - - public static final ConfigOption HOSTNAME = - ConfigOptions.key("hostname") - .stringType() - .noDefaultValue() - .withDescription( - "IP address or hostname of the OceanBase database server or OceanBase proxy server."); - - public static final ConfigOption PORT = - ConfigOptions.key("port") - .intType() - .noDefaultValue() - .withDescription( - "Integer port number of OceanBase database server or OceanBase proxy server."); - - public static final ConfigOption COMPATIBLE_MODE = - ConfigOptions.key("compatible-mode") - .stringType() - .defaultValue("mysql") - .withDescription( - "The compatible mode of OceanBase, can be 'mysql' or 'oracle'."); - - public static final ConfigOption JDBC_DRIVER = - ConfigOptions.key("jdbc.driver") - .stringType() - .defaultValue("com.mysql.cj.jdbc.Driver") - .withDescription( - "JDBC driver class name, use 'com.mysql.cj.jdbc.Driver' by default."); - - public static final ConfigOption LOG_PROXY_HOST = - ConfigOptions.key("logproxy.host") - .stringType() - .noDefaultValue() - .withDescription("Hostname or IP address of OceanBase log proxy service."); - - public static final ConfigOption LOG_PROXY_PORT = - ConfigOptions.key("logproxy.port") - .intType() - .noDefaultValue() - .withDescription("Port number of OceanBase log proxy service."); - - public static final ConfigOption LOG_PROXY_CLIENT_ID = - ConfigOptions.key("logproxy.client.id") - .stringType() - .noDefaultValue() - .withDescription( - "Id of log proxy client, used to distinguish different connections."); - - public static final ConfigOption SCAN_STARTUP_TIMESTAMP = - ConfigOptions.key("scan.startup.timestamp") - .longType() - .noDefaultValue() - .withDescription( - "Optional timestamp in seconds used in case of \"timestamp\" startup mode."); - - public static final ConfigOption RS_LIST = - ConfigOptions.key("rootserver-list") - .stringType() - .noDefaultValue() - .withDescription( - "The semicolon-separated list of root servers in format `ip:rpc_port:sql_port`, corresponding to the parameter 'rootservice_list' in the database."); - - public static final ConfigOption CONFIG_URL = - ConfigOptions.key("config-url") - .stringType() - .noDefaultValue() - .withDescription( - "The url used to get root servers list, corresponding to the parameter 'obconfig_url' in the database."); - - public static final ConfigOption WORKING_MODE = - ConfigOptions.key("working-mode") - .stringType() - .defaultValue("storage") - .withDescription( - "The working mode of 'obcdc', can be `storage` (default value, supported from `obcdc` 3.1.3) or `memory`."); - - public static final String OBCDC_PROPERTIES_PREFIX = "obcdc.properties."; - - @Override - public DynamicTableSource createDynamicTableSource(Context context) { - final FactoryUtil.TableFactoryHelper helper = - FactoryUtil.createTableFactoryHelper(this, context); - helper.validateExcept( - JdbcUrlUtils.PROPERTIES_PREFIX, - OBCDC_PROPERTIES_PREFIX, - DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX); - - ResolvedSchema physicalSchema = context.getCatalogTable().getResolvedSchema(); - - ReadableConfig config = helper.getOptions(); - - StartupOptions startupOptions = getStartupOptions(config); - - String username = config.get(USERNAME); - String password = config.get(PASSWORD); - String tenantName = config.get(TENANT_NAME); - String databaseName = config.get(DATABASE_NAME); - String tableName = config.get(TABLE_NAME); - String tableList = config.get(TABLE_LIST); - - String serverTimeZone = config.get(SERVER_TIME_ZONE); - Duration connectTimeout = config.get(CONNECT_TIMEOUT); - - String hostname = config.get(HOSTNAME); - Integer port = config.get(PORT); - String compatibleMode = config.get(COMPATIBLE_MODE); - String jdbcDriver = config.get(JDBC_DRIVER); - - validateJdbcDriver(compatibleMode, jdbcDriver); - - String logProxyHost = config.get(LOG_PROXY_HOST); - Integer logProxyPort = config.get(LOG_PROXY_PORT); - String logProxyClientId = config.get(LOG_PROXY_CLIENT_ID); - Long startupTimestamp = config.get(SCAN_STARTUP_TIMESTAMP); - String rsList = config.get(RS_LIST); - String configUrl = config.get(CONFIG_URL); - String workingMode = config.get(WORKING_MODE); - - OptionUtils.printOptions(IDENTIFIER, ((Configuration) config).toMap()); - - return new OceanBaseTableSource( - physicalSchema, - startupOptions, - username, - password, - tenantName, - databaseName, - tableName, - tableList, - serverTimeZone, - connectTimeout, - hostname, - port, - compatibleMode, - jdbcDriver, - JdbcUrlUtils.getJdbcProperties(context.getCatalogTable().getOptions()), - logProxyHost, - logProxyPort, - logProxyClientId, - startupTimestamp, - rsList, - configUrl, - workingMode, - getProperties(context.getCatalogTable().getOptions(), OBCDC_PROPERTIES_PREFIX), - DebeziumOptions.getDebeziumProperties(context.getCatalogTable().getOptions())); - } - - @Override - public String factoryIdentifier() { - return IDENTIFIER; - } - - @Override - public Set> requiredOptions() { - Set> options = new HashSet<>(); - options.add(USERNAME); - options.add(PASSWORD); - options.add(HOSTNAME); - options.add(PORT); - return options; - } - - @Override - public Set> optionalOptions() { - Set> options = new HashSet<>(); - options.add(SCAN_STARTUP_MODE); - options.add(SCAN_STARTUP_TIMESTAMP); - options.add(DATABASE_NAME); - options.add(TABLE_NAME); - options.add(TABLE_LIST); - options.add(COMPATIBLE_MODE); - options.add(JDBC_DRIVER); - options.add(CONNECT_TIMEOUT); - options.add(SERVER_TIME_ZONE); - options.add(TENANT_NAME); - options.add(LOG_PROXY_HOST); - options.add(LOG_PROXY_PORT); - options.add(LOG_PROXY_CLIENT_ID); - options.add(RS_LIST); - options.add(CONFIG_URL); - options.add(WORKING_MODE); - return options; - } - - private static final String SCAN_STARTUP_MODE_VALUE_INITIAL = "initial"; - private static final String SCAN_STARTUP_MODE_VALUE_SNAPSHOT = "snapshot"; - private static final String SCAN_STARTUP_MODE_VALUE_LATEST = "latest-offset"; - private static final String SCAN_STARTUP_MODE_VALUE_TIMESTAMP = "timestamp"; - - private static StartupOptions getStartupOptions(ReadableConfig config) { - String modeString = config.get(SCAN_STARTUP_MODE); - - switch (modeString.toLowerCase()) { - case SCAN_STARTUP_MODE_VALUE_INITIAL: - return StartupOptions.initial(); - case SCAN_STARTUP_MODE_VALUE_SNAPSHOT: - return StartupOptions.snapshot(); - case SCAN_STARTUP_MODE_VALUE_LATEST: - return StartupOptions.latest(); - case SCAN_STARTUP_MODE_VALUE_TIMESTAMP: - if (config.get(SCAN_STARTUP_TIMESTAMP) != null) { - return StartupOptions.timestamp(config.get(SCAN_STARTUP_TIMESTAMP) * 1000); - } - throw new ValidationException( - String.format( - "Option '%s' should not be empty", SCAN_STARTUP_TIMESTAMP.key())); - - default: - throw new ValidationException( - String.format( - "Invalid value for option '%s'. Supported values are [%s, %s, %s, %s], but was: %s", - SCAN_STARTUP_MODE.key(), - SCAN_STARTUP_MODE_VALUE_INITIAL, - SCAN_STARTUP_MODE_VALUE_SNAPSHOT, - SCAN_STARTUP_MODE_VALUE_LATEST, - SCAN_STARTUP_MODE_VALUE_TIMESTAMP, - modeString)); - } - } - - private void validateJdbcDriver(String compatibleMode, String jdbcDriver) { - Objects.requireNonNull(compatibleMode, "'compatible-mode' is required."); - Objects.requireNonNull(jdbcDriver, "'jdbc.driver' is required."); - if ("oracle".equalsIgnoreCase(compatibleMode) - && !OceanBaseUtils.isOceanBaseDriver(jdbcDriver)) { - throw new IllegalArgumentException( - "OceanBase JDBC driver is required for OceanBase Oracle mode."); - } - try { - Class.forName(jdbcDriver); - } catch (ClassNotFoundException e) { - throw new RuntimeException("Jdbc driver class not found", e); - } - } - - private Properties getProperties(Map tableOptions, String prefix) { - Properties properties = new Properties(); - tableOptions.keySet().stream() - .filter(key -> key.startsWith(prefix)) - .forEach( - key -> { - final String value = tableOptions.get(key); - final String subKey = key.substring(prefix.length()); - properties.put(subKey, value); - }); - return properties; - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/OceanBaseUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/OceanBaseUtils.java deleted file mode 100644 index 7cb738b68b7..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/OceanBaseUtils.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase.utils; - -/** Utils for OceanBase. */ -public class OceanBaseUtils { - - public static boolean isOceanBaseDriver(String driverClass) { - return "com.oceanbase.jdbc.Driver".equals(driverClass) - || "com.alipay.oceanbase.jdbc.Driver".equals(driverClass); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/OptionUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/OptionUtils.java deleted file mode 100644 index a71dd26a627..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/OptionUtils.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase.utils; - -import org.apache.flink.configuration.ConfigurationUtils; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Map; - -/** A utility class to print configuration of connectors. */ -public class OptionUtils { - - private static final Logger LOG = LoggerFactory.getLogger(OptionUtils.class); - - /** Utility class can not be instantiated. */ - private OptionUtils() {} - - public static void printOptions(String identifier, Map config) { - Map hideMap = ConfigurationUtils.hideSensitiveValues(config); - LOG.info("Print {} connector configuration:", identifier); - for (String key : hideMap.keySet()) { - LOG.info("{} = {}", key, hideMap.get(key)); - } - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index 9eadc06c76a..abb9b5eaa86 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.cdc.connectors.oceanbase.table.OceanBaseTableSourceFactory +org.apache.flink.cdc.connectors.oceanbase.table.OceanBaseMySqlTableSourceFactory diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseCharsetITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseCharsetITCase.java new file mode 100644 index 00000000000..5e1e979891e --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseCharsetITCase.java @@ -0,0 +1,227 @@ +/* + * 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.flink.cdc.connectors.oceanbase; + +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.planner.factories.TestValuesTableFactory; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.StringUtils; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.sql.Connection; +import java.sql.Statement; +import java.util.Arrays; +import java.util.stream.Stream; + +/** Test supporting different column charsets for OceanBase. */ +@Disabled( + "Temporarily disabled for GitHub CI due to unavailability of OceanBase Binlog Service docker image. These tests are currently only supported for local execution.") +public class OceanBaseCharsetITCase extends OceanBaseSourceTestBase { + + private static final String DDL_FILE = "charset_test"; + private static final String DATABASE_NAME = "cdc_c_" + getRandomSuffix(); + + private final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(); + + private final StreamTableEnvironment tEnv = + StreamTableEnvironment.create( + env, EnvironmentSettings.newInstance().inStreamingMode().build()); + + @BeforeAll + public static void beforeClass() throws InterruptedException { + initializeOceanBaseTables( + DDL_FILE, + DATABASE_NAME, + s -> // see: + // https://www.oceanbase.com/docs/common-oceanbase-database-cn-1000000002017544 + !StringUtils.isNullOrWhitespaceOnly(s) + && (s.contains("utf8_test") + || s.contains("latin1_test") + || s.contains("gbk_test") + || s.contains("big5_test") + || s.contains("ascii_test") + || s.contains("sjis_test"))); + } + + @AfterAll + public static void after() { + dropDatabase(DATABASE_NAME); + } + + @BeforeEach + public void before() { + TestValuesTableFactory.clearAllData(); + env.setParallelism(4); + env.enableCheckpointing(200); + } + + public static Stream parameters() { + return Stream.of( + Arguments.of( + "utf8_test", + new String[] {"+I[1, 测试数据]", "+I[2, Craig Marshall]", "+I[3, 另一个测试数据]"}, + new String[] { + "-D[1, 测试数据]", + "-D[2, Craig Marshall]", + "-D[3, 另一个测试数据]", + "+I[11, 测试数据]", + "+I[12, Craig Marshall]", + "+I[13, 另一个测试数据]" + }), + Arguments.of( + "ascii_test", + new String[] { + "+I[1, ascii test!?]", "+I[2, Craig Marshall]", "+I[3, {test}]" + }, + new String[] { + "-D[1, ascii test!?]", + "-D[2, Craig Marshall]", + "-D[3, {test}]", + "+I[11, ascii test!?]", + "+I[12, Craig Marshall]", + "+I[13, {test}]" + }), + Arguments.of( + "gbk_test", + new String[] {"+I[1, 测试数据]", "+I[2, Craig Marshall]", "+I[3, 另一个测试数据]"}, + new String[] { + "-D[1, 测试数据]", + "-D[2, Craig Marshall]", + "-D[3, 另一个测试数据]", + "+I[11, 测试数据]", + "+I[12, Craig Marshall]", + "+I[13, 另一个测试数据]" + }), + Arguments.of( + "latin1_test", + new String[] {"+I[1, ÀÆÉ]", "+I[2, Craig Marshall]", "+I[3, Üæû]"}, + new String[] { + "-D[1, ÀÆÉ]", + "-D[2, Craig Marshall]", + "-D[3, Üæû]", + "+I[11, ÀÆÉ]", + "+I[12, Craig Marshall]", + "+I[13, Üæû]" + }), + Arguments.of( + "big5_test", + new String[] {"+I[1, 大五]", "+I[2, Craig Marshall]", "+I[3, 丹店]"}, + new String[] { + "-D[1, 大五]", + "-D[2, Craig Marshall]", + "-D[3, 丹店]", + "+I[11, 大五]", + "+I[12, Craig Marshall]", + "+I[13, 丹店]" + }), + Arguments.of( + "sjis_test", + new String[] {"+I[1, ひびぴ]", "+I[2, Craig Marshall]", "+I[3, フブプ]"}, + new String[] { + "-D[1, ひびぴ]", + "-D[2, Craig Marshall]", + "-D[3, フブプ]", + "+I[11, ひびぴ]", + "+I[12, Craig Marshall]", + "+I[13, フブプ]" + })); + } + + @ParameterizedTest + @MethodSource("parameters") + public void testCharset(String testName, String[] snapshotExpected, String[] binlogExpected) + throws Exception { + String sourceDDL = + String.format( + "CREATE TABLE %s (\n" + + " table_id BIGINT,\n" + + " table_name STRING,\n" + + " primary key(table_id) not enforced" + + ") WITH (" + + " 'connector' = 'oceanbase-cdc'," + + " 'hostname' = '%s'," + + " 'port' = '%s'," + + " 'username' = '%s'," + + " 'password' = '%s'," + + " 'database-name' = '%s'," + + " 'table-name' = '%s'," + + " 'scan.incremental.snapshot.enabled' = '%s'," + + " 'server-id' = '%s'," + + " 'server-time-zone' = 'Asia/Shanghai'," + + " 'jdbc.properties.connectTimeout' = '6000000000'," + + " 'jdbc.properties.socketTimeout' = '6000000000'," + + " 'jdbc.properties.autoReconnect' = 'true'," + + " 'jdbc.properties.failOverReadOnly' = 'false'," + + " 'scan.incremental.snapshot.chunk.size' = '%s'" + + ")", + testName, + getHost(), + PORT, + USER_NAME, + PASSWORD, + DATABASE_NAME, + testName, + true, + getServerId(), + 4); + tEnv.executeSql(sourceDDL); + // async submit job + TableResult result = + tEnv.executeSql(String.format("SELECT table_id,table_name FROM %s", testName)); + + // test snapshot phase + CloseableIterator iterator = result.collect(); + waitForSnapshotStarted(iterator); + assertEqualsInAnyOrder( + Arrays.asList(snapshotExpected), fetchRows(iterator, snapshotExpected.length)); + + // test binlog phase + try (Connection connection = getJdbcConnection(); + Statement statement = connection.createStatement()) { + statement.execute( + String.format( + "/*TDDL:FORBID_EXECUTE_DML_ALL=FALSE*/UPDATE %s.%s SET table_id = table_id + 10;", + DATABASE_NAME, testName)); + } + assertEqualsInAnyOrder( + Arrays.asList(binlogExpected), fetchRows(iterator, binlogExpected.length)); + result.getJobClient().get().cancel().get(); + + // Sleep to avoid the issue: The last packet successfully received from the server was 35 + // milliseconds ago. + Thread.sleep(1_000); + } + + private static void waitForSnapshotStarted(CloseableIterator iterator) throws Exception { + while (!iterator.hasNext()) { + Thread.sleep(100); + } + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseFailoverITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseFailoverITCase.java new file mode 100644 index 00000000000..aaad93e1ca9 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseFailoverITCase.java @@ -0,0 +1,521 @@ +/* + * 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.flink.cdc.connectors.oceanbase; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.cdc.connectors.utils.ExternalResourceProxy; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.StringUtils; + +import io.debezium.jdbc.JdbcConnection; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.FutureTask; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Stream; + +import static java.lang.String.format; +import static org.apache.flink.api.common.JobStatus.RUNNING; + +/** failover IT tests for oceanbase. */ +@Disabled( + "Temporarily disabled for GitHub CI due to unavailability of OceanBase Binlog Service docker image. These tests are currently only supported for local execution.") +@Timeout(value = 180, unit = TimeUnit.SECONDS) +public class OceanBaseFailoverITCase extends OceanBaseSourceTestBase { + + private static final String DEFAULT_SCAN_STARTUP_MODE = "initial"; + private static final String DDL_FILE = "oceanbase_ddl_test"; + private static final String DEFAULT_TEST_DATABASE = "customer_" + getRandomSuffix(); + protected static final int DEFAULT_PARALLELISM = 4; + + private final List firstPartBinlogEvents = + Arrays.asList( + "-U[103, user_3, Shanghai, 123567891234]", + "+U[103, user_3, Hangzhou, 123567891234]", + "-D[102, user_2, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "-U[103, user_3, Hangzhou, 123567891234]", + "+U[103, user_3, Shanghai, 123567891234]"); + + private final List secondPartBinlogEvents = + Arrays.asList( + "-U[1010, user_11, Shanghai, 123567891234]", + "+I[2001, user_22, Shanghai, 123567891234]", + "+I[2002, user_23, Shanghai, 123567891234]", + "+I[2003, user_24, Shanghai, 123567891234]", + "+U[1010, user_11, Hangzhou, 123567891234]"); + + public static Stream parameters() { + return Stream.of( + Arguments.of("customers", null), + Arguments.of("customers", "id"), + Arguments.of("customers_no_pk", "id")); + } + + @RegisterExtension + public final ExternalResourceProxy miniClusterResource = + new ExternalResourceProxy<>( + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .withHaLeadershipControl() + .build())); + + @BeforeEach + public void setup() throws InterruptedException { + initializeOceanBaseTables( + DDL_FILE, + DEFAULT_TEST_DATABASE, + s -> !StringUtils.isNullOrWhitespaceOnly(s) && (s.contains("customers"))); + } + + @AfterEach + public void clean() { + dropDatabase(DEFAULT_TEST_DATABASE); + } + + // Failover tests + @ParameterizedTest + @MethodSource("parameters") + @Timeout(value = 120, unit = TimeUnit.SECONDS) + public void testTaskManagerFailoverInSnapshotPhase(String tableName, String chunkColumnName) + throws Exception { + testMySqlParallelSource( + FailoverType.TM, + FailoverPhase.SNAPSHOT, + new String[] {tableName, "customers_1"}, + tableName, + chunkColumnName); + } + + @ParameterizedTest + @MethodSource("parameters") + public void testTaskManagerFailoverInBinlogPhase(String tableName, String chunkColumnName) + throws Exception { + testMySqlParallelSource( + FailoverType.TM, + FailoverPhase.BINLOG, + new String[] {tableName, "customers_1"}, + tableName, + chunkColumnName); + } + + @ParameterizedTest + @MethodSource("parameters") + public void testTaskManagerFailoverFromLatestOffset(String tableName, String chunkColumnName) + throws Exception { + testMySqlParallelSource( + DEFAULT_PARALLELISM, + "latest-offset", + FailoverType.TM, + FailoverPhase.BINLOG, + new String[] {tableName, "customers_1"}, + RestartStrategies.fixedDelayRestart(1, 0), + tableName, + chunkColumnName); + } + + @ParameterizedTest + @MethodSource("parameters") + public void testJobManagerFailoverInSnapshotPhase(String tableName, String chunkColumnName) + throws Exception { + testMySqlParallelSource( + FailoverType.JM, + FailoverPhase.SNAPSHOT, + new String[] {tableName, "customers_1"}, + tableName, + chunkColumnName); + } + + @ParameterizedTest + @MethodSource("parameters") + public void testJobManagerFailoverInBinlogPhase(String tableName, String chunkColumnName) + throws Exception { + testMySqlParallelSource( + FailoverType.JM, + FailoverPhase.BINLOG, + new String[] {tableName, "customers_1"}, + tableName, + chunkColumnName); + } + + @ParameterizedTest + @MethodSource("parameters") + public void testJobManagerFailoverFromLatestOffset(String tableName, String chunkColumnName) + throws Exception { + testMySqlParallelSource( + DEFAULT_PARALLELISM, + "latest-offset", + FailoverType.JM, + FailoverPhase.BINLOG, + new String[] {tableName, "customers_1"}, + RestartStrategies.fixedDelayRestart(1, 0), + tableName, + chunkColumnName); + } + + @ParameterizedTest + @MethodSource("parameters") + public void testTaskManagerFailoverSingleParallelism(String tableName, String chunkColumnName) + throws Exception { + testMySqlParallelSource( + 1, + FailoverType.TM, + FailoverPhase.SNAPSHOT, + new String[] {tableName}, + tableName, + chunkColumnName); + } + + @ParameterizedTest + @MethodSource("parameters") + public void testJobManagerFailoverSingleParallelism(String tableName, String chunkColumnName) + throws Exception { + testMySqlParallelSource( + 1, + FailoverType.JM, + FailoverPhase.SNAPSHOT, + new String[] {tableName}, + tableName, + chunkColumnName); + } + + private void testMySqlParallelSource( + FailoverType failoverType, + FailoverPhase failoverPhase, + String[] captureCustomerTables, + String tableName, + String chunkColumnName) + throws Exception { + testMySqlParallelSource( + DEFAULT_PARALLELISM, + failoverType, + failoverPhase, + captureCustomerTables, + tableName, + chunkColumnName); + } + + private void testMySqlParallelSource( + int parallelism, + FailoverType failoverType, + FailoverPhase failoverPhase, + String[] captureCustomerTables, + String tableName, + String chunkColumnName) + throws Exception { + testMySqlParallelSource( + parallelism, + DEFAULT_SCAN_STARTUP_MODE, + failoverType, + failoverPhase, + captureCustomerTables, + RestartStrategies.fixedDelayRestart(1, 0), + tableName, + chunkColumnName); + } + + private void testMySqlParallelSource( + int parallelism, + String scanStartupMode, + FailoverType failoverType, + FailoverPhase failoverPhase, + String[] captureCustomerTables, + RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration, + String tableName, + String chunkColumnName) + throws Exception { + testMySqlParallelSource( + parallelism, + scanStartupMode, + failoverType, + failoverPhase, + captureCustomerTables, + restartStrategyConfiguration, + false, + tableName, + chunkColumnName); + } + + private void testMySqlParallelSource( + int parallelism, + String scanStartupMode, + FailoverType failoverType, + FailoverPhase failoverPhase, + String[] captureCustomerTables, + RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration, + boolean skipSnapshotBackfill, + String tableName, + String chunkColumnName) + throws Exception { + captureCustomerTables = new String[] {tableName}; + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.enableCheckpointing(3000L, CheckpointingMode.EXACTLY_ONCE); + StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); + + env.setParallelism(parallelism); + env.enableCheckpointing(200L); + env.setRestartStrategy(restartStrategyConfiguration); + String sourceDDL = + format( + "CREATE TABLE customers (" + + " id BIGINT NOT NULL," + + " name STRING," + + " address STRING," + + " phone_number STRING" + + ("customers_no_pk".equals(tableName) + ? "" + : ", primary key (id) not enforced") + + ") WITH (" + + " 'connector' = 'oceanbase-cdc'," + + " 'scan.incremental.snapshot.enabled' = 'true'," + + " 'hostname' = '%s'," + + " 'port' = '%s'," + + " 'username' = '%s'," + + " 'password' = '%s'," + + " 'database-name' = '%s'," + + " 'table-name' = '%s'," + + " 'scan.startup.mode' = '%s'," + + " 'scan.incremental.snapshot.chunk.size' = '100'," + + " 'scan.incremental.snapshot.backfill.skip' = '%s'," + + " 'server-time-zone' = 'Asia/Shanghai'," + + " 'server-id' = '%s'" + + " %s" + + ")", + getHost(), + getPort(), + getUserName(), + getPassword(), + DEFAULT_TEST_DATABASE, + getTableNameRegex(captureCustomerTables), + scanStartupMode, + skipSnapshotBackfill, + getServerId(), + chunkColumnName == null + ? "" + : String.format( + ", 'scan.incremental.snapshot.chunk.key-column' = '%s'", + chunkColumnName)); + tEnv.executeSql(sourceDDL); + TableResult tableResult = tEnv.executeSql("select * from customers"); + + // first step: check the snapshot data + if (DEFAULT_SCAN_STARTUP_MODE.equals(scanStartupMode)) { + checkSnapshotData(tableResult, failoverType, failoverPhase, captureCustomerTables); + } + + // second step: check the binlog data + checkBinlogData(tableResult, failoverType, failoverPhase, captureCustomerTables); + + // sleepMs(3000); + tableResult.getJobClient().get().cancel().get(); + } + + private void checkSnapshotData( + TableResult tableResult, + FailoverType failoverType, + FailoverPhase failoverPhase, + String[] captureCustomerTables) + throws Exception { + String[] snapshotForSingleTable = + new String[] { + "+I[101, user_1, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "+I[103, user_3, Shanghai, 123567891234]", + "+I[109, user_4, Shanghai, 123567891234]", + "+I[110, user_5, Shanghai, 123567891234]", + "+I[111, user_6, Shanghai, 123567891234]", + "+I[118, user_7, Shanghai, 123567891234]", + "+I[121, user_8, Shanghai, 123567891234]", + "+I[123, user_9, Shanghai, 123567891234]", + "+I[1009, user_10, Shanghai, 123567891234]", + "+I[1010, user_11, Shanghai, 123567891234]", + "+I[1011, user_12, Shanghai, 123567891234]", + "+I[1012, user_13, Shanghai, 123567891234]", + "+I[1013, user_14, Shanghai, 123567891234]", + "+I[1014, user_15, Shanghai, 123567891234]", + "+I[1015, user_16, Shanghai, 123567891234]", + "+I[1016, user_17, Shanghai, 123567891234]", + "+I[1017, user_18, Shanghai, 123567891234]", + "+I[1018, user_19, Shanghai, 123567891234]", + "+I[1019, user_20, Shanghai, 123567891234]", + "+I[2000, user_21, Shanghai, 123567891234]" + }; + + List expectedSnapshotData = new ArrayList<>(); + for (int i = 0; i < captureCustomerTables.length; i++) { + expectedSnapshotData.addAll(Arrays.asList(snapshotForSingleTable)); + } + + CloseableIterator iterator = tableResult.collect(); + JobID jobId = tableResult.getJobClient().get().getJobID(); + + // trigger failover after some snapshot splits read finished + if (failoverPhase == FailoverPhase.SNAPSHOT && iterator.hasNext()) { + triggerFailover( + failoverType, + jobId, + miniClusterResource.get().getMiniCluster(), + () -> sleepMs(100)); + } + + assertEqualsInAnyOrder( + expectedSnapshotData, fetchRows(iterator, expectedSnapshotData.size())); + } + + private void checkBinlogData( + TableResult tableResult, + FailoverType failoverType, + FailoverPhase failoverPhase, + String[] captureCustomerTables) + throws Exception { + waitUntilJobRunning(tableResult); + CloseableIterator iterator = tableResult.collect(); + JobID jobId = tableResult.getJobClient().get().getJobID(); + + for (String tableId : captureCustomerTables) { + makeFirstPartBinlogEvents(getConnection(), DEFAULT_TEST_DATABASE + '.' + tableId); + } + + // wait for the binlog reading + Thread.sleep(3_000L); + + if (failoverPhase == FailoverPhase.BINLOG) { + triggerFailover( + failoverType, + jobId, + miniClusterResource.get().getMiniCluster(), + () -> sleepMs(200)); + waitUntilJobRunning(tableResult); + } + for (String tableId : captureCustomerTables) { + makeSecondPartBinlogEvents(getConnection(), DEFAULT_TEST_DATABASE + '.' + tableId); + } + + List expectedBinlogData = new ArrayList<>(); + for (int i = 0; i < captureCustomerTables.length; i++) { + expectedBinlogData.addAll(firstPartBinlogEvents); + expectedBinlogData.addAll(secondPartBinlogEvents); + } + sleepMs(3_000); + assertEqualsInAnyOrder(expectedBinlogData, fetchRows(iterator, expectedBinlogData.size())); + Assertions.assertThat(hasNextData(iterator)).isFalse(); + } + + private void waitUntilJobRunning(TableResult tableResult) + throws InterruptedException, ExecutionException { + do { + Thread.sleep(5000L); + } while (tableResult.getJobClient().get().getJobStatus().get() != RUNNING); + } + + private boolean hasNextData(final CloseableIterator iterator) + throws InterruptedException, ExecutionException { + ExecutorService executor = Executors.newSingleThreadExecutor(); + try { + FutureTask future = new FutureTask(iterator::hasNext); + executor.execute(future); + return future.get(3, TimeUnit.SECONDS); + } catch (TimeoutException e) { + return false; + } finally { + executor.shutdown(); + } + } + + /** + * Make some changes on the specified customer table. Changelog in string could be accessed by + * {@link #firstPartBinlogEvents}. + */ + private void makeFirstPartBinlogEvents(JdbcConnection connection, String tableId) + throws SQLException { + try { + connection.setAutoCommit(false); + + // make binlog events for the first split + connection.execute( + "UPDATE " + tableId + " SET address = 'Hangzhou' where id = 103", + "DELETE FROM " + tableId + " where id = 102", + "INSERT INTO " + tableId + " VALUES(102, 'user_2','Shanghai','123567891234')", + "UPDATE " + tableId + " SET address = 'Shanghai' where id = 103"); + connection.commit(); + } finally { + connection.close(); + } + } + + /** + * Make some other changes on the specified customer table. Changelog in string could be + * accessed by {@link #secondPartBinlogEvents}. + */ + private void makeSecondPartBinlogEvents(JdbcConnection connection, String tableId) + throws SQLException { + try { + connection.setAutoCommit(false); + + // make binlog events for split-1 + connection.execute("UPDATE " + tableId + " SET address = 'Hangzhou' where id = 1010"); + connection.commit(); + + // make binlog events for the last split + connection.execute( + "INSERT INTO " + + tableId + + " VALUES(2001, 'user_22','Shanghai','123567891234')," + + " (2002, 'user_23','Shanghai','123567891234')," + + "(2003, 'user_24','Shanghai','123567891234')"); + connection.commit(); + } finally { + connection.close(); + } + } + + private void sleepMs(long millis) { + try { + Thread.sleep(millis); + } catch (InterruptedException ignored) { + } + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSourceITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSourceITCase.java new file mode 100644 index 00000000000..75b6726ff09 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSourceITCase.java @@ -0,0 +1,376 @@ +/* + * 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.flink.cdc.connectors.oceanbase; + +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.planner.factories.TestValuesTableFactory; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; + +import java.sql.Connection; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static java.lang.String.format; + +/** OceanBase CDC source connector integration test. */ +@Disabled( + "Temporarily disabled for GitHub CI due to unavailability of OceanBase Binlog Service docker image. These tests are currently only supported for local execution.") +public class OceanBaseSourceITCase extends OceanBaseSourceTestBase { + private static final String DDL_FILE = "oceanbase_ddl_test"; + private static final String DATABASE_NAME = "cdc_s_" + getRandomSuffix(); + + @BeforeAll + public static void beforeClass() throws InterruptedException { + initializeOceanBaseTables(DDL_FILE, DATABASE_NAME, null); + } + + @AfterAll + public static void afterClass() { + dropDatabase(DATABASE_NAME); + } + + @Test + public void testSingleKey() throws Exception { + int parallelism = 1; + String[] captureCustomerTables = new String[] {"orders"}; + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); + + env.setParallelism(parallelism); + env.enableCheckpointing(200L); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); + String sourceDDL = + format( + "CREATE TABLE orders_source (" + + " id BIGINT NOT NULL," + + " seller_id STRING," + + " order_id STRING," + + " buyer_id STRING," + + " create_time TIMESTAMP," + + " primary key (id) not enforced" + + ") WITH (" + + " 'connector' = 'oceanbase-cdc'," + + " 'scan.incremental.snapshot.enabled' = 'true'," + + " 'hostname' = '%s'," + + " 'port' = '%s'," + + " 'username' = '%s'," + + " 'password' = '%s'," + + " 'database-name' = '%s'," + + " 'table-name' = '%s'," + + " 'scan.incremental.snapshot.chunk.size' = '100'," + + " 'server-time-zone' = 'Asia/Shanghai'," + + " 'server-id' = '%s'" + + ")", + getHost(), + PORT, + USER_NAME, + PASSWORD, + DATABASE_NAME, + getTableNameRegex(captureCustomerTables), + getServerId()); + + // first step: check the snapshot data + String[] snapshotForSingleTable = + new String[] { + "+I[1, 1001, 1, 102, 2022-01-16T00:00]", + "+I[2, 1002, 2, 105, 2022-01-16T00:00]", + "+I[3, 1004, 3, 109, 2022-01-16T00:00]", + "+I[4, 1002, 2, 106, 2022-01-16T00:00]", + "+I[5, 1003, 1, 107, 2022-01-16T00:00]", + }; + tEnv.executeSql(sourceDDL); + TableResult tableResult = tEnv.executeSql("select * from orders_source"); + CloseableIterator iterator = tableResult.collect(); + List expectedSnapshotData = new ArrayList<>(); + for (int i = 0; i < captureCustomerTables.length; i++) { + expectedSnapshotData.addAll(Arrays.asList(snapshotForSingleTable)); + } + + List realSnapshotData = fetchRows(iterator, expectedSnapshotData.size()); + assertEqualsInAnyOrder(expectedSnapshotData, realSnapshotData); + + // second step: check the sink data + tEnv.executeSql( + "CREATE TABLE sink (" + + " id BIGINT NOT NULL," + + " seller_id STRING," + + " order_id STRING," + + " buyer_id STRING," + + " create_time TIMESTAMP," + + " primary key (id) not enforced" + + ") WITH (" + + " 'connector' = 'values'," + + " 'sink-insert-only' = 'false'" + + ")"); + tableResult = tEnv.executeSql("insert into sink select * from orders_source"); + + waitForSinkSize("sink", realSnapshotData.size()); + assertEqualsInAnyOrder( + expectedSnapshotData, TestValuesTableFactory.getRawResultsAsStrings("sink")); + + // third step: check dml events + try (Connection connection = getJdbcConnection(); + Statement statement = connection.createStatement()) { + statement.execute("use " + DATABASE_NAME); + statement.execute("INSERT INTO orders VALUES (6, 1006,1006, 1006,'2022-01-17');"); + statement.execute("INSERT INTO orders VALUES (7,1007, 1007,1007, '2022-01-17');"); + statement.execute("UPDATE orders SET seller_id= 9999, order_id=9999 WHERE id=6;"); + statement.execute("UPDATE orders SET seller_id= 9999, order_id=9999 WHERE id=7;"); + statement.execute("DELETE FROM orders WHERE id=7;"); + } + + String[] expectedBinlog = + new String[] { + "+I[6, 1006, 1006, 1006, 2022-01-17T00:00]", + "+I[7, 1007, 1007, 1007, 2022-01-17T00:00]", + "-U[6, 1006, 1006, 1006, 2022-01-17T00:00]", + "+U[6, 9999, 9999, 1006, 2022-01-17T00:00]", + "-U[7, 1007, 1007, 1007, 2022-01-17T00:00]", + "+U[7, 9999, 9999, 1007, 2022-01-17T00:00]", + "-D[7, 9999, 9999, 1007, 2022-01-17T00:00]" + }; + List expectedBinlogData = new ArrayList<>(); + for (int i = 0; i < captureCustomerTables.length; i++) { + expectedBinlogData.addAll(Arrays.asList(expectedBinlog)); + } + List realBinlog = fetchRows(iterator, expectedBinlog.length); + assertEqualsInOrder(expectedBinlogData, realBinlog); + Thread.sleep(3_000); + tableResult.getJobClient().get().cancel().get(); + } + + @Test + public void testFullTypesDdl() throws Exception { + int parallelism = 1; + String[] captureCustomerTables = new String[] {"oceanbase_full_types"}; + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); + + env.setParallelism(parallelism); + env.enableCheckpointing(200L); + String sourceDDL = + format( + "CREATE TABLE oceanbase_full_types (\n" + + " `id` INT NOT NULL,\n" + + " tiny_c TINYINT,\n" + + " tiny_un_c SMALLINT ,\n" + + " small_c SMALLINT,\n" + + " small_un_c INT,\n" + + " medium_c INT,\n" + + " medium_un_c INT,\n" + + " int_c INT ,\n" + + " int_un_c BIGINT,\n" + + " int11_c BIGINT,\n" + + " big_c BIGINT,\n" + + " big_un_c DECIMAL(20, 0),\n" + + " varchar_c VARCHAR(255),\n" + + " char_c CHAR(3),\n" + + " real_c FLOAT,\n" + + " float_c FLOAT,\n" + + " double_c DOUBLE,\n" + + " decimal_c DECIMAL(8, 4),\n" + + " numeric_c DECIMAL(6, 0),\n" + + " big_decimal_c STRING,\n" + + " bit1_c BOOLEAN,\n" + + " tiny1_c BOOLEAN,\n" + + " boolean_c BOOLEAN,\n" + + " date_c DATE,\n" + + " time_c TIME(0),\n" + + " datetime3_c TIMESTAMP(3),\n" + + " datetime6_c TIMESTAMP(6),\n" + + " timestamp_c TIMESTAMP(0),\n" + + " file_uuid BYTES,\n" + + " bit_c BINARY(8),\n" + + " text_c STRING,\n" + + " tiny_blob_c BYTES,\n" + + " blob_c BYTES,\n" + + " medium_blob_c BYTES,\n" + + " long_blob_c BYTES,\n" + + " year_c INT,\n" + + " enum_c STRING,\n" + + " set_c ARRAY,\n" + + " json_c STRING,\n" + + " point_c STRING,\n" + + " geometry_c STRING,\n" + + " linestring_c STRING,\n" + + " polygon_c STRING,\n" + + " multipoint_c STRING,\n" + + " multiline_c STRING,\n" + + " multipolygon_c STRING,\n" + + " geometrycollection_c STRING,\n" + + " primary key (`id`) not enforced" + + ") WITH (" + + " 'connector' = 'oceanbase-cdc'," + + " 'scan.incremental.snapshot.enabled' = 'true'," + + " 'hostname' = '%s'," + + " 'port' = '%s'," + + " 'username' = '%s'," + + " 'password' = '%s'," + + " 'database-name' = '%s'," + + " 'table-name' = '%s'," + + " 'scan.incremental.snapshot.chunk.size' = '100'," + + " 'server-time-zone' = 'Asia/Shanghai'," + + " 'server-id' = '%s'" + + ")", + getHost(), + PORT, + USER_NAME, + PASSWORD, + DATABASE_NAME, + getTableNameRegex(captureCustomerTables), + getServerId()); + tEnv.executeSql(sourceDDL); + + TableResult tableResult = tEnv.executeSql("select * from oceanbase_full_types"); + CloseableIterator iterator = tableResult.collect(); + List realSnapshotData = fetchRows(iterator, 1); + String[] expectedSnapshotData = + new String[] { + "+I[1, 127, 255, 32767, 65535, 8388607, 16777215, 2147483647, 4294967295, 2147483647, 9223372036854775807, 18446744073709551615, Hello World, abc, 123.102, 123.102, 404.4443, 123.4567, 346, 34567892.1, false, true, true, 2020-07-17, 18:00:22, 2020-07-17T18:00:22.123, 2020-07-17T18:00:22.123456, 2020-07-17T18:00:22, [101, 26, -19, 8, 57, 15, 72, -109, -78, -15, 54, -110, 62, 123, 116, 0], [4, 4, 4, 4, 4, 4, 4, 4], text, [16], [16], [16], [16], 2021, red, [a, b], {\"key1\": \"value1\", \"key2\": \"value2\", \"num1\": 16708304.0, \"num2\": 16708305}, {\"coordinates\":[1,1],\"type\":\"Point\",\"srid\":0}, {\"coordinates\":[[[1,1],[2,1],[2,2],[1,2],[1,1]]],\"type\":\"Polygon\",\"srid\":0}, {\"coordinates\":[[3,0],[3,3],[3,5]],\"type\":\"LineString\",\"srid\":0}, {\"coordinates\":[[[1,1],[2,1],[2,2],[1,2],[1,1]]],\"type\":\"Polygon\",\"srid\":0}, {\"coordinates\":[[1,1],[2,2]],\"type\":\"MultiPoint\",\"srid\":0}, {\"coordinates\":[[[1,1],[2,2],[3,3]],[[4,4],[5,5]]],\"type\":\"MultiLineString\",\"srid\":0}, {\"coordinates\":[[[[0,0],[10,0],[10,10],[0,10],[0,0]]],[[[5,5],[7,5],[7,7],[5,7],[5,5]]]],\"type\":\"MultiPolygon\",\"srid\":0}, {\"geometries\":[{\"type\":\"Point\",\"coordinates\":[10,10]},{\"type\":\"Point\",\"coordinates\":[30,30]},{\"type\":\"LineString\",\"coordinates\":[[15,15],[20,20]]}],\"type\":\"GeometryCollection\",\"srid\":0}]" + }; + assertEqualsInAnyOrder(Arrays.asList(expectedSnapshotData), realSnapshotData); + Thread.sleep(3_000); + tableResult.getJobClient().get().cancel().get(); + } + + @Test + public void testMultiKeys() throws Exception { + int parallelism = 1; + String[] captureCustomerTables = new String[] {"orders_with_multi_pks"}; + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); + + env.setParallelism(parallelism); + env.enableCheckpointing(200L); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); + String sourceDDL = + format( + "CREATE TABLE orders_with_multi_pks (" + + " id BIGINT NOT NULL," + + " seller_id STRING," + + " order_id STRING," + + " buyer_id STRING," + + " create_time TIMESTAMP," + + " primary key (id,order_id) not enforced" + + ") WITH (" + + " 'connector' = 'oceanbase-cdc'," + + " 'scan.incremental.snapshot.enabled' = 'true'," + + " 'hostname' = '%s'," + + " 'port' = '%s'," + + " 'username' = '%s'," + + " 'password' = '%s'," + + " 'database-name' = '%s'," + + " 'table-name' = '%s'," + + " 'scan.incremental.snapshot.chunk.size' = '100'," + + " 'server-time-zone' = 'Asia/Shanghai'," + + " 'server-id' = '%s'" + + ")", + getHost(), + PORT, + USER_NAME, + PASSWORD, + DATABASE_NAME, + getTableNameRegex(captureCustomerTables), + getServerId()); + + // first step: check the snapshot data + String[] snapshotForSingleTable = + new String[] { + "+I[1, 1001, 1, 102, 2022-01-16T00:00]", + "+I[2, 1002, 2, 105, 2022-01-16T00:00]", + "+I[3, 1004, 3, 109, 2022-01-16T00:00]", + "+I[4, 1002, 2, 106, 2022-01-16T00:00]", + "+I[5, 1003, 1, 107, 2022-01-16T00:00]", + }; + tEnv.executeSql(sourceDDL); + TableResult tableResult = tEnv.executeSql("select * from orders_with_multi_pks"); + CloseableIterator iterator = tableResult.collect(); + List expectedSnapshotData = new ArrayList<>(); + for (int i = 0; i < captureCustomerTables.length; i++) { + expectedSnapshotData.addAll(Arrays.asList(snapshotForSingleTable)); + } + + List realSnapshotData = fetchRows(iterator, expectedSnapshotData.size()); + assertEqualsInAnyOrder(expectedSnapshotData, realSnapshotData); + + // second step: check the sink data + tEnv.executeSql( + "CREATE TABLE multi_key_sink (" + + " id BIGINT NOT NULL," + + " seller_id STRING," + + " order_id STRING," + + " buyer_id STRING," + + " create_time TIMESTAMP," + + " primary key (id,order_id) not enforced" + + ") WITH (" + + " 'connector' = 'values'," + + " 'sink-insert-only' = 'false'" + + ")"); + + tEnv.executeSql("insert into multi_key_sink select * from orders_with_multi_pks"); + + waitForSinkSize("multi_key_sink", realSnapshotData.size()); + assertEqualsInAnyOrder( + expectedSnapshotData, + TestValuesTableFactory.getRawResultsAsStrings("multi_key_sink")); + + // third step: check dml events + try (Connection connection = getJdbcConnection(); + Statement statement = connection.createStatement()) { + statement.execute("use " + DATABASE_NAME); + statement.execute( + "INSERT INTO orders_with_multi_pks VALUES (6, 1006,1006, 1006,'2022-01-17');"); + statement.execute( + "INSERT INTO orders_with_multi_pks VALUES (7,1007, 1007,1007, '2022-01-17');"); + statement.execute( + "UPDATE orders_with_multi_pks SET seller_id= 9999, order_id=9999 WHERE id=6;"); + statement.execute( + "UPDATE orders_with_multi_pks SET seller_id= 9999, order_id=9999 WHERE id=7;"); + statement.execute("DELETE FROM orders_with_multi_pks WHERE id=7;"); + } + + String[] expectedBinlog = + new String[] { + "+I[6, 1006, 1006, 1006, 2022-01-17T00:00]", + "+I[7, 1007, 1007, 1007, 2022-01-17T00:00]", + "-D[6, 1006, 1006, 1006, 2022-01-17T00:00]", + "+I[6, 9999, 9999, 1006, 2022-01-17T00:00]", + "-D[7, 1007, 1007, 1007, 2022-01-17T00:00]", + "+I[7, 9999, 9999, 1007, 2022-01-17T00:00]", + "-D[7, 9999, 9999, 1007, 2022-01-17T00:00]" + }; + List realBinlog = fetchRows(iterator, expectedBinlog.length); + assertEqualsInAnyOrder(Arrays.asList(expectedBinlog), realBinlog); + Thread.sleep(3_000); + tableResult.getJobClient().get().cancel().get(); + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSourceTestBase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSourceTestBase.java new file mode 100644 index 00000000000..0ea3cb9662d --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSourceTestBase.java @@ -0,0 +1,261 @@ +/* + * 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.flink.cdc.connectors.oceanbase; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils; +import org.apache.flink.runtime.highavailability.nonha.embedded.HaLeadershipControl; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.table.planner.factories.TestValuesTableFactory; +import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.types.Row; + +import io.debezium.connector.mysql.MySqlConnection; +import org.apache.commons.lang3.StringUtils; +import org.assertj.core.api.Assertions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.UUID; +import java.util.function.Function; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import static java.lang.String.format; +import static org.apache.flink.util.Preconditions.checkState; + +/** Basic class for testing Database OceanBase which supported the mysql protocol. */ +public abstract class OceanBaseSourceTestBase extends AbstractTestBase { + + private static final Logger LOG = LoggerFactory.getLogger(OceanBaseSourceTestBase.class); + private static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)--.*$"); + protected static final Integer PORT = 3306; + protected static final String USER_NAME = System.getenv("OCEANBASE_USERNAME"); + protected static final String PASSWORD = System.getenv("OCEANBASE_PASSWORD"); + protected static final String HOSTNAME = System.getenv("OCEANBASE_HOSTNAME"); + + protected static String getHost() { + return HOSTNAME; + } + + protected static Integer getPort() { + return PORT; + } + + protected static String getUserName() { + return USER_NAME; + } + + protected static String getPassword() { + return PASSWORD; + } + + protected static String getJdbcUrl() { + return String.format("jdbc:mysql://%s:%s", HOSTNAME, PORT); + } + + protected static Connection getJdbcConnection() throws SQLException { + String jdbcUrl = getJdbcUrl(); + LOG.info("jdbcUrl is :" + jdbcUrl); + return DriverManager.getConnection(jdbcUrl, USER_NAME, PASSWORD); + } + + /** initialize database and tables with ${databaseName}.sql for testing. */ + protected static void initializeOceanBaseTables( + String ddlName, String dbName, Function filter) + throws InterruptedException { + final String ddlFile = String.format("ddl/mysql/%s.sql", ddlName); + final URL ddlTestFile = OceanBaseSourceTestBase.class.getClassLoader().getResource(ddlFile); + Assertions.assertThat(ddlTestFile).withFailMessage("Cannot locate " + ddlFile).isNotNull(); + // need to sleep 1s, make sure the jdbc connection can be created + Thread.sleep(1000); + try (Connection connection = getJdbcConnection(); + Statement statement = connection.createStatement()) { + statement.execute("drop database if exists " + dbName); + statement.execute("create database if not exists " + dbName); + statement.execute("use " + dbName + ";"); + final List statements = + Arrays.stream( + Files.readAllLines(Paths.get(ddlTestFile.toURI())).stream() + .map(String::trim) + .filter(x -> !x.startsWith("--") && !x.isEmpty()) + .map( + x -> { + final Matcher m = + COMMENT_PATTERN.matcher(x); + return m.matches() ? m.group(1) : x; + }) + .collect(Collectors.joining("\n")) + .split(";")) + .filter(sql -> filter == null || filter.apply(sql)) + .collect(Collectors.toList()); + for (String stmt : statements) { + statement.execute(stmt); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + protected static void dropDatabase(String dbName) { + try (Connection connection = getJdbcConnection(); + Statement statement = connection.createStatement()) { + statement.execute("drop database if exists " + dbName); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + protected static List fetchRows(Iterator iter, int size) { + List rows = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + Row row = iter.next(); + rows.add(row.toString()); + } + return rows; + } + + protected String getTableNameRegex(String[] captureCustomerTables) { + checkState(captureCustomerTables.length > 0); + if (captureCustomerTables.length == 1) { + return captureCustomerTables[0]; + } else { + // pattern that matches multiple tables + return format("(%s)", StringUtils.join(captureCustomerTables, "|")); + } + } + + protected String getServerId() { + final Random random = new Random(); + int serverId = random.nextInt(100) + 5400; + return serverId + "-" + (serverId + 4); + } + + // ------------------------------------------------------------------------ + // test utilities + // ------------------------------------------------------------------------ + + protected static void waitForSinkSize(String sinkName, int expectedSize) + throws InterruptedException { + while (sinkSize(sinkName) < expectedSize) { + Thread.sleep(100); + } + } + + protected static int sinkSize(String sinkName) { + synchronized (TestValuesTableFactory.class) { + try { + return TestValuesTableFactory.getRawResults(sinkName).size(); + } catch (IllegalArgumentException e) { + // job is not started yet + return 0; + } + } + } + + protected static void assertEqualsInAnyOrder(List expected, List actual) { + Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(expected); + } + + protected static void assertEqualsInOrder(List expected, List actual) { + Assertions.assertThat(actual).containsExactlyElementsOf(expected); + } + + protected static String getRandomSuffix() { + String base = UUID.randomUUID().toString().replaceAll("-", ""); + if (base.length() > 10) { + return base.substring(0, 11); + } + return base; + } + + /** The type of failover. */ + protected enum FailoverType { + TM, + JM, + NONE + } + + /** The phase of failover. */ + protected enum FailoverPhase { + SNAPSHOT, + BINLOG, + NEVER + } + + protected static void triggerFailover( + FailoverType type, JobID jobId, MiniCluster miniCluster, Runnable afterFailAction) + throws Exception { + switch (type) { + case TM: + restartTaskManager(miniCluster, afterFailAction); + break; + case JM: + triggerJobManagerFailover(jobId, miniCluster, afterFailAction); + break; + case NONE: + break; + default: + throw new IllegalStateException("Unexpected value: " + type); + } + } + + protected static void triggerJobManagerFailover( + JobID jobId, MiniCluster miniCluster, Runnable afterFailAction) throws Exception { + final HaLeadershipControl haLeadershipControl = miniCluster.getHaLeadershipControl().get(); + haLeadershipControl.revokeJobMasterLeadership(jobId).get(); + afterFailAction.run(); + haLeadershipControl.grantJobMasterLeadership(jobId).get(); + } + + protected static void restartTaskManager(MiniCluster miniCluster, Runnable afterFailAction) + throws Exception { + miniCluster.terminateTaskManager(0).get(); + afterFailAction.run(); + miniCluster.startTaskManager(); + } + + protected MySqlConnection getConnection() { + Map properties = new HashMap<>(); + properties.put("database.hostname", getHost()); + properties.put("database.port", String.valueOf(getPort())); + properties.put("database.user", getUserName()); + properties.put("database.password", getPassword()); + properties.put("database.serverTimezone", ZoneId.of("UTC").toString()); + io.debezium.config.Configuration configuration = + io.debezium.config.Configuration.from(properties); + return DebeziumUtils.createMySqlConnection(configuration, new Properties()); + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java deleted file mode 100644 index d9f12f9e762..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java +++ /dev/null @@ -1,167 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase; - -import org.apache.flink.cdc.connectors.oceanbase.testutils.OceanBaseCdcMetadata; -import org.apache.flink.cdc.connectors.utils.StaticExternalResourceProxy; -import org.apache.flink.table.planner.factories.TestValuesTableFactory; -import org.apache.flink.table.utils.LegacyRowResource; -import org.apache.flink.test.util.AbstractTestBase; - -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.extension.RegisterExtension; - -import java.net.URL; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; -import java.sql.Statement; -import java.time.Duration; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.TimeoutException; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import java.util.stream.Collectors; - -/** Basic class for testing OceanBase source. */ -public abstract class OceanBaseTestBase extends AbstractTestBase { - - private static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)--.*$"); - - @RegisterExtension - public static StaticExternalResourceProxy usesLegacyRows = - new StaticExternalResourceProxy<>(LegacyRowResource.INSTANCE); - - public static final Duration FETCH_TIMEOUT = Duration.ofSeconds(60); - - protected abstract OceanBaseCdcMetadata metadata(); - - protected String commonOptionsString() { - return String.format( - " 'connector' = 'oceanbase-cdc', " - + " 'username' = '%s', " - + " 'password' = '%s', " - + " 'hostname' = '%s', " - + " 'port' = '%s', " - + " 'compatible-mode' = '%s', " - + " 'jdbc.driver' = '%s'", - metadata().getUsername(), - metadata().getPassword(), - metadata().getHostname(), - metadata().getPort(), - metadata().getCompatibleMode(), - metadata().getDriverClass()); - } - - protected String logProxyOptionsString() { - return String.format( - " 'working-mode' = 'memory'," - + " 'tenant-name' = '%s'," - + " 'logproxy.host' = '%s'," - + " 'logproxy.port' = '%s'", - metadata().getTenantName(), - metadata().getLogProxyHost(), - metadata().getLogProxyPort()); - } - - protected String initialOptionsString() { - return " 'scan.startup.mode' = 'initial', " - + commonOptionsString() - + ", " - + logProxyOptionsString(); - } - - protected String snapshotOptionsString() { - return " 'scan.startup.mode' = 'snapshot', " + commonOptionsString(); - } - - protected Connection getJdbcConnection() throws SQLException { - return DriverManager.getConnection( - metadata().getJdbcUrl(), metadata().getUsername(), metadata().getPassword()); - } - - protected void setGlobalTimeZone(String serverTimeZone) throws SQLException { - try (Connection connection = getJdbcConnection(); - Statement statement = connection.createStatement()) { - statement.execute(String.format("SET GLOBAL time_zone = '%s';", serverTimeZone)); - } - } - - protected void initializeTable(String sqlFile) { - final String ddlFile = - String.format("ddl/%s/%s.sql", metadata().getCompatibleMode(), sqlFile); - final URL ddlTestFile = getClass().getClassLoader().getResource(ddlFile); - Assertions.assertThat(ddlTestFile).withFailMessage("Cannot locate " + ddlFile).isNotNull(); - try (Connection connection = getJdbcConnection(); - Statement statement = connection.createStatement()) { - final List statements = - Arrays.stream( - Files.readAllLines(Paths.get(ddlTestFile.toURI())).stream() - .map(String::trim) - .filter(x -> !x.startsWith("--") && !x.isEmpty()) - .map( - x -> { - final Matcher m = - COMMENT_PATTERN.matcher(x); - return m.matches() ? m.group(1) : x; - }) - .collect(Collectors.joining("\n")) - .split(";")) - .collect(Collectors.toList()); - for (String stmt : statements) { - statement.execute(stmt); - } - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - public static void waitForSinkSize(String sinkName, int expectedSize) - throws InterruptedException, TimeoutException { - long deadlineTimestamp = System.currentTimeMillis() + FETCH_TIMEOUT.toMillis(); - while (System.currentTimeMillis() < deadlineTimestamp) { - if (sinkSize(sinkName) < expectedSize) { - Thread.sleep(100); - } else { - return; - } - } - throw new TimeoutException( - String.format( - "Failed to fetch enough records in sink.\nExpected size: %d\nActual values: %s", - expectedSize, TestValuesTableFactory.getRawResults(sinkName))); - } - - public static int sinkSize(String sinkName) { - synchronized (TestValuesTableFactory.class) { - try { - return TestValuesTableFactory.getRawResultsAsStrings(sinkName).size(); - } catch (IllegalArgumentException e) { - // job is not started yet - return 0; - } - } - } - - public static void assertContainsInAnyOrder(List expected, List actual) { - Assertions.assertThat(actual).containsAll(expected); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestUtils.java index 47400f2e59d..fd75c282236 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestUtils.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestUtils.java @@ -17,7 +17,6 @@ package org.apache.flink.cdc.connectors.oceanbase; -import org.apache.flink.cdc.connectors.oceanbase.testutils.LogProxyContainer; import org.apache.flink.cdc.connectors.oceanbase.testutils.OceanBaseContainer; import org.slf4j.Logger; @@ -33,18 +32,10 @@ public class OceanBaseTestUtils { private static final Logger LOG = LoggerFactory.getLogger(OceanBaseTestUtils.class); private static final String OB_4_3_3_VERSION = "4.3.3.0-100000142024101215"; - private static final String OB_LOG_PROXY_2_0_2_VERSION = "2.0.2-101000142024080121"; - private static final String CDC_TEST_OB_VERSION = "4.2.1.6-106000012024042515"; private static final String SYS_PASSWORD = "123456"; private static final String TEST_PASSWORD = "654321"; - public static OceanBaseContainer createOceanBaseContainerForCDC() { - return createOceanBaseContainer(CDC_TEST_OB_VERSION, "mini") - .withSysPassword(SYS_PASSWORD) - .withStartupTimeout(Duration.ofMinutes(4)); - } - public static OceanBaseContainer createOceanBaseContainerForJdbc() { return createOceanBaseContainer(OB_4_3_3_VERSION, "mini") .withStartupTimeout(Duration.ofMinutes(4)); @@ -58,11 +49,4 @@ public static OceanBaseContainer createOceanBaseContainer(String version, String .withEnv("OB_LOG_DISK_SIZE", "4G") .withLogConsumer(new Slf4jLogConsumer(LOG)); } - - public static LogProxyContainer createLogProxyContainer() { - return new LogProxyContainer(OB_LOG_PROXY_2_0_2_VERSION) - .withSysPassword(SYS_PASSWORD) - .withStartupTimeout(Duration.ofMinutes(1)) - .withLogConsumer(new Slf4jLogConsumer(LOG)); - } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMySQLModeITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMySQLModeITCase.java deleted file mode 100644 index 644eb997cf6..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMySQLModeITCase.java +++ /dev/null @@ -1,648 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase.table; - -import org.apache.flink.api.common.JobStatus; -import org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestBase; -import org.apache.flink.cdc.connectors.oceanbase.testutils.LogProxyContainer; -import org.apache.flink.cdc.connectors.oceanbase.testutils.OceanBaseCdcMetadata; -import org.apache.flink.cdc.connectors.oceanbase.testutils.OceanBaseContainer; -import org.apache.flink.cdc.connectors.oceanbase.testutils.OceanBaseMySQLCdcMetadata; -import org.apache.flink.cdc.connectors.oceanbase.testutils.UniqueDatabase; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableResult; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.table.planner.factories.TestValuesTableFactory; - -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.testcontainers.containers.Network; -import org.testcontainers.junit.jupiter.Container; -import org.testcontainers.junit.jupiter.Testcontainers; - -import java.sql.Connection; -import java.sql.Statement; -import java.time.ZoneId; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestUtils.createLogProxyContainer; -import static org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestUtils.createOceanBaseContainerForCDC; - -/** Integration tests for OceanBase MySQL mode table source. */ -@Testcontainers -class OceanBaseMySQLModeITCase extends OceanBaseTestBase { - - private final StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment(); - private final StreamTableEnvironment tEnv = - StreamTableEnvironment.create( - env, EnvironmentSettings.newInstance().inStreamingMode().build()); - - private static final Network NETWORK = Network.newNetwork(); - - @Container - private static final OceanBaseContainer OB_SERVER = - createOceanBaseContainerForCDC().withNetwork(NETWORK); - - @Container - private static final LogProxyContainer LOG_PROXY = - createLogProxyContainer().withNetwork(NETWORK); - - private static final OceanBaseCdcMetadata METADATA = - new OceanBaseMySQLCdcMetadata(OB_SERVER, LOG_PROXY); - - private UniqueDatabase inventoryDatabase; - private UniqueDatabase columnTypesDatabase; - - @Override - protected OceanBaseCdcMetadata metadata() { - return METADATA; - } - - @BeforeEach - public void before() { - TestValuesTableFactory.clearAllData(); - env.enableCheckpointing(1000); - env.getCheckpointConfig().setMinPauseBetweenCheckpoints(500); - } - - @AfterEach - public void after() { - if (inventoryDatabase != null) { - inventoryDatabase.dropDatabase(); - inventoryDatabase = null; - } - - if (columnTypesDatabase != null) { - columnTypesDatabase.dropDatabase(); - columnTypesDatabase = null; - } - } - - @Override - protected String logProxyOptionsString() { - return super.logProxyOptionsString() - + " , " - + String.format(" 'rootserver-list' = '%s'", METADATA.getRsList()); - } - - /** - * Current OceanBase connector uses timestamp (in seconds) to mark the offset during the - * transition from {@code SNAPSHOT} to {@code STREAMING} mode. Thus, if some snapshot inserting - * events are too close to the transitioning offset, snapshot inserting events might be emitted - * multiple times.
    - * This could be safely removed after switching to incremental snapshot framework which provides - * Exactly-once guarantee. - */ - private void waitForTableInitialization() throws InterruptedException { - Thread.sleep(5000L); - } - - @Test - void testTableList() throws Exception { - inventoryDatabase = new UniqueDatabase(OB_SERVER, "inventory"); - inventoryDatabase.createAndInitialize("mysql"); - waitForTableInitialization(); - - String sourceDDL = - String.format( - "CREATE TABLE ob_source (" - + " `id` INT NOT NULL," - + " name STRING," - + " description STRING," - + " weight DECIMAL(20, 10)," - + " PRIMARY KEY (`id`) NOT ENFORCED" - + ") WITH (" - + initialOptionsString() - + ", " - + " 'table-list' = '%s'" - + ")", - inventoryDatabase.getDatabaseName() + ".products"); - - String sinkDDL = - "CREATE TABLE sink (" - + " `id` INT NOT NULL," - + " name STRING," - + " description STRING," - + " weight DECIMAL(20, 10)," - + " PRIMARY KEY (`id`) NOT ENFORCED" - + ") WITH (" - + " 'connector' = 'values'," - + " 'sink-insert-only' = 'false'," - + " 'sink-expected-messages-num' = '30'" - + ")"; - - tEnv.executeSql(sourceDDL); - tEnv.executeSql(sinkDDL); - - TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM ob_source"); - - waitForSinkSize("sink", 9); - int snapshotSize = sinkSize("sink"); - - try (Connection connection = getJdbcConnection(); - Statement statement = connection.createStatement()) { - statement.execute(String.format("USE %s;", inventoryDatabase.getDatabaseName())); - statement.execute( - "UPDATE products SET description='18oz carpenter hammer' WHERE id=106;"); - statement.execute("UPDATE products SET weight='5.1' WHERE id=107;"); - statement.execute( - "INSERT INTO products VALUES (default,'jacket','water resistent white wind breaker',0.2);"); // 110 - statement.execute( - "INSERT INTO products VALUES (default,'scooter','Big 2-wheel scooter ',5.18);"); - statement.execute( - "UPDATE products SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110;"); - statement.execute("UPDATE products SET weight='5.17' WHERE id=111;"); - statement.execute("DELETE FROM products WHERE id=111;"); - } - - waitForSinkSize("sink", snapshotSize + 7); - - /* - *
    -         * The final database table looks like this:
    -         *
    -         * > SELECT * FROM products;
    -         * +-----+--------------------+---------------------------------------------------------+--------+
    -         * | id  | name               | description                                             | weight |
    -         * +-----+--------------------+---------------------------------------------------------+--------+
    -         * | 101 | scooter            | Small 2-wheel scooter                                   |   3.14 |
    -         * | 102 | car battery        | 12V car battery                                         |    8.1 |
    -         * | 103 | 12-pack drill bits | 12-pack of drill bits with sizes ranging from #40 to #3 |    0.8 |
    -         * | 104 | hammer             | 12oz carpenter's hammer                                 |   0.75 |
    -         * | 105 | hammer             | 14oz carpenter's hammer                                 |  0.875 |
    -         * | 106 | hammer             | 18oz carpenter hammer                                   |      1 |
    -         * | 107 | rocks              | box of assorted rocks                                   |    5.1 |
    -         * | 108 | jacket             | water resistent black wind breaker                      |    0.1 |
    -         * | 109 | spare tire         | 24 inch spare tire                                      |   22.2 |
    -         * | 110 | jacket             | new water resistent white wind breaker                  |    0.5 |
    -         * +-----+--------------------+---------------------------------------------------------+--------+
    -         * 
    - */ - - List expected = - Arrays.asList( - "+I(101,scooter,Small 2-wheel scooter,3.1400000000)", - "+I(102,car battery,12V car battery,8.1000000000)", - "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8000000000)", - "+I(104,hammer,12oz carpenter's hammer,0.7500000000)", - "+I(105,hammer,14oz carpenter's hammer,0.8750000000)", - "+I(106,hammer,16oz carpenter's hammer,1.0000000000)", - "+I(107,rocks,box of assorted rocks,5.3000000000)", - "+I(108,jacket,water resistent black wind breaker,0.1000000000)", - "+I(109,spare tire,24 inch spare tire,22.2000000000)", - "+U(106,hammer,18oz carpenter hammer,1.0000000000)", - "+U(107,rocks,box of assorted rocks,5.1000000000)", - "+I(110,jacket,water resistent white wind breaker,0.2000000000)", - "+I(111,scooter,Big 2-wheel scooter ,5.1800000000)", - "+U(110,jacket,new water resistent white wind breaker,0.5000000000)", - "+U(111,scooter,Big 2-wheel scooter ,5.1700000000)", - "-D(111,scooter,Big 2-wheel scooter ,5.1700000000)"); - List actual = TestValuesTableFactory.getRawResultsAsStrings("sink"); - assertContainsInAnyOrder(expected, actual); - - result.getJobClient().get().cancel().get(); - } - - @Test - void testMetadataColumns() throws Exception { - inventoryDatabase = new UniqueDatabase(OB_SERVER, "inventory"); - inventoryDatabase.createAndInitialize("mysql"); - waitForTableInitialization(); - - String sourceDDL = - String.format( - "CREATE TABLE ob_source (" - + " tenant STRING METADATA FROM 'tenant_name' VIRTUAL," - + " database STRING METADATA FROM 'database_name' VIRTUAL," - + " `table` STRING METADATA FROM 'table_name' VIRTUAL," - + " `id` INT NOT NULL," - + " name STRING," - + " description STRING," - + " weight DECIMAL(20, 10)," - + " PRIMARY KEY (`id`) NOT ENFORCED" - + ") WITH (" - + initialOptionsString() - + "," - + " 'database-name' = '%s'," - + " 'table-name' = '%s'" - + ")", - String.format("^%s$", inventoryDatabase.getDatabaseName()), - "^products$"); - - String sinkDDL = - "CREATE TABLE sink (" - + " tenant STRING," - + " database STRING," - + " `table` STRING," - + " `id` DECIMAL(20, 0) NOT NULL," - + " name STRING," - + " description STRING," - + " weight DECIMAL(20, 10)," - + " primary key (tenant, database, `table`, `id`) not enforced" - + ") WITH (" - + " 'connector' = 'values'," - + " 'sink-insert-only' = 'false'," - + " 'sink-expected-messages-num' = '20'" - + ")"; - tEnv.executeSql(sourceDDL); - tEnv.executeSql(sinkDDL); - - // async submit job - TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM ob_source"); - - waitForSinkSize("sink", 9); - int snapshotSize = sinkSize("sink"); - - try (Connection connection = getJdbcConnection(); - Statement statement = connection.createStatement()) { - statement.execute(String.format("USE %s;", inventoryDatabase.getDatabaseName())); - statement.execute( - "UPDATE products SET description='18oz carpenter hammer' WHERE id=106;"); - } - - waitForSinkSize("sink", snapshotSize + 1); - - String tenant = metadata().getTenantName(); - - List expected = - Stream.of( - "+I(%s,%s,products,101,scooter,Small 2-wheel scooter,3.1400000000)", - "+I(%s,%s,products,102,car battery,12V car battery,8.1000000000)", - "+I(%s,%s,products,103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8000000000)", - "+I(%s,%s,products,104,hammer,12oz carpenter's hammer,0.7500000000)", - "+I(%s,%s,products,105,hammer,14oz carpenter's hammer,0.8750000000)", - "+I(%s,%s,products,106,hammer,16oz carpenter's hammer,1.0000000000)", - "+I(%s,%s,products,107,rocks,box of assorted rocks,5.3000000000)", - "+I(%s,%s,products,108,jacket,water resistent black wind breaker,0.1000000000)", - "+I(%s,%s,products,109,spare tire,24 inch spare tire,22.2000000000)", - "+U(%s,%s,products,106,hammer,18oz carpenter hammer,1.0000000000)") - .map( - line -> - String.format( - line, tenant, inventoryDatabase.getDatabaseName())) - .collect(Collectors.toList()); - List actual = TestValuesTableFactory.getRawResultsAsStrings("sink"); - assertContainsInAnyOrder(expected, actual); - result.getJobClient().get().cancel().get(); - } - - @Test - void testAllDataTypes() throws Exception { - String serverTimeZone = "+00:00"; - setGlobalTimeZone(serverTimeZone); - tEnv.getConfig().setLocalTimeZone(ZoneId.of(serverTimeZone)); - - columnTypesDatabase = new UniqueDatabase(OB_SERVER, "column_type_test"); - columnTypesDatabase.createAndInitialize("mysql"); - waitForTableInitialization(); - - String sourceDDL = - String.format( - "CREATE TABLE ob_source (\n" - + " `id` INT NOT NULL,\n" - + " bit1_c BOOLEAN,\n" - + " tiny1_c BOOLEAN,\n" - + " boolean_c BOOLEAN,\n" - + " tiny_c TINYINT,\n" - + " tiny_un_c SMALLINT,\n" - + " small_c SMALLINT ,\n" - + " small_un_c INT ,\n" - + " medium_c INT,\n" - + " medium_un_c INT,\n" - + " int11_c INT,\n" - + " int_c INT,\n" - + " int_un_c BIGINT,\n" - + " big_c BIGINT,\n" - + " big_un_c DECIMAL(20, 0),\n" - + " real_c FLOAT,\n" - + " float_c FLOAT,\n" - + " double_c DOUBLE,\n" - + " decimal_c DECIMAL(8, 4),\n" - + " numeric_c DECIMAL(6, 0),\n" - + " big_decimal_c STRING,\n" - + " date_c DATE,\n" - + " time_c TIME(0),\n" - + " datetime3_c TIMESTAMP(3),\n" - + " datetime6_c TIMESTAMP(6),\n" - + " timestamp_c TIMESTAMP,\n" - + " timestamp3_c TIMESTAMP(3),\n" - + " timestamp6_c TIMESTAMP(6),\n" - + " char_c CHAR(3),\n" - + " varchar_c VARCHAR(255),\n" - + " file_uuid BINARY(16),\n" - + " bit_c BINARY(8),\n" - + " text_c STRING,\n" - + " tiny_blob_c BYTES,\n" - + " medium_blob_c BYTES,\n" - + " blob_c BYTES,\n" - + " long_blob_c BYTES,\n" - + " year_c INT,\n" - + " set_c ARRAY,\n" - + " enum_c STRING,\n" - + " json_c STRING,\n" - + " primary key (`id`) not enforced" - + ") WITH (" - + initialOptionsString() - + "," - + " 'database-name' = '%s'," - + " 'table-name' = '%s'," - + " 'server-time-zone' = '%s'" - + ")", - String.format("^%s$", columnTypesDatabase.getDatabaseName()), - "^full_types$", - serverTimeZone); - String sinkDDL = - "CREATE TABLE sink (" - + " `id` INT NOT NULL,\n" - + " bit1_c BOOLEAN,\n" - + " tiny1_c BOOLEAN,\n" - + " boolean_c BOOLEAN,\n" - + " tiny_c TINYINT,\n" - + " tiny_un_c SMALLINT,\n" - + " small_c SMALLINT ,\n" - + " small_un_c INT ,\n" - + " medium_c INT,\n" - + " medium_un_c INT,\n" - + " int11_c INT,\n" - + " int_c INT,\n" - + " int_un_c BIGINT,\n" - + " big_c BIGINT,\n" - + " big_un_c DECIMAL(20, 0),\n" - + " real_c FLOAT,\n" - + " float_c FLOAT,\n" - + " double_c DOUBLE,\n" - + " decimal_c DECIMAL(8, 4),\n" - + " numeric_c DECIMAL(6, 0),\n" - + " big_decimal_c STRING,\n" - + " date_c DATE,\n" - + " time_c TIME(0),\n" - + " datetime3_c TIMESTAMP(3),\n" - + " datetime6_c TIMESTAMP(6),\n" - + " timestamp_c TIMESTAMP,\n" - + " timestamp3_c TIMESTAMP(3),\n" - + " timestamp6_c TIMESTAMP(6),\n" - + " char_c CHAR(3),\n" - + " varchar_c VARCHAR(255),\n" - + " file_uuid STRING,\n" - + " bit_c BINARY(8),\n" - + " text_c STRING,\n" - + " tiny_blob_c BYTES,\n" - + " medium_blob_c BYTES,\n" - + " blob_c BYTES,\n" - + " long_blob_c BYTES,\n" - + " year_c INT,\n" - + " set_c ARRAY,\n" - + " enum_c STRING,\n" - + " json_c STRING,\n" - + " primary key (`id`) not enforced" - + ") WITH (" - + " 'connector' = 'values'," - + " 'sink-insert-only' = 'false'," - + " 'sink-expected-messages-num' = '3'" - + ")"; - tEnv.executeSql(sourceDDL); - tEnv.executeSql(sinkDDL); - - TableResult result = - tEnv.executeSql( - "INSERT INTO sink SELECT id,\n" - + "bit1_c,\n" - + "tiny1_c,\n" - + "boolean_c,\n" - + "tiny_c,\n" - + "tiny_un_c,\n" - + "small_c ,\n" - + "small_un_c,\n" - + "medium_c,\n" - + "medium_un_c,\n" - + "int11_c,\n" - + "int_c,\n" - + "int_un_c,\n" - + "big_c,\n" - + "big_un_c,\n" - + "real_c,\n" - + "float_c,\n" - + "double_c,\n" - + "decimal_c,\n" - + "numeric_c,\n" - + "big_decimal_c,\n" - + "date_c,\n" - + "time_c,\n" - + "datetime3_c,\n" - + "datetime6_c,\n" - + "timestamp_c,\n" - + "timestamp3_c,\n" - + "timestamp6_c,\n" - + "char_c,\n" - + "varchar_c,\n" - + "TO_BASE64(DECODE(file_uuid, 'UTF-8')),\n" - + "bit_c,\n" - + "text_c,\n" - + "tiny_blob_c,\n" - + "medium_blob_c,\n" - + "blob_c,\n" - + "long_blob_c,\n" - + "year_c,\n" - + "set_c,\n" - + "enum_c,\n" - + "json_c\n" - + " FROM ob_source"); - - waitForSinkSize("sink", 1); - int snapshotSize = sinkSize("sink"); - - try (Connection connection = getJdbcConnection(); - Statement statement = connection.createStatement()) { - statement.execute(String.format("USE %s;", columnTypesDatabase.getDatabaseName())); - statement.execute( - "UPDATE full_types SET timestamp_c = '2020-07-17 18:33:22' WHERE id=1;"); - } - - waitForSinkSize("sink", snapshotSize + 1); - - List expected = - Arrays.asList( - "+I(1,false,true,true,127,255,32767,65535,8388607,16777215,2147483647,2147483647,4294967295,9223372036854775807,18446744073709551615,123.102,123.102,404.4443,123.4567,346,34567892.1,2020-07-17,18:00:22,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,2020-07-17T18:00:22,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,abc,Hello World,ZRrvv70IOQ9I77+977+977+9Nu+/vT57dAA=,[4, 4, 4, 4, 4, 4, 4, 4],text,[16],[16],[16],[16],2022,[a, b],red,{\"key1\": \"value1\"})", - "+U(1,false,true,true,127,255,32767,65535,8388607,16777215,2147483647,2147483647,4294967295,9223372036854775807,18446744073709551615,123.102,123.102,404.4443,123.4567,346,34567892.1,2020-07-17,18:00:22,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,2020-07-17T18:33:22,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,abc,Hello World,ZRrvv70IOQ9I77+977+977+9Nu+/vT57dAA=,[4, 4, 4, 4, 4, 4, 4, 4],text,[16],[16],[16],[16],2022,[a, b],red,{\"key1\": \"value1\"})"); - - List actual = TestValuesTableFactory.getRawResultsAsStrings("sink"); - assertContainsInAnyOrder(expected, actual); - result.getJobClient().get().cancel().get(); - } - - @Test - void testTimezoneBerlin() throws Exception { - testTimeDataTypes("+02:00"); - } - - @Test - void testTimezoneShanghai() throws Exception { - testTimeDataTypes("+08:00"); - } - - void testTimeDataTypes(String serverTimeZone) throws Exception { - setGlobalTimeZone(serverTimeZone); - tEnv.getConfig().setLocalTimeZone(ZoneId.of(serverTimeZone)); - - columnTypesDatabase = new UniqueDatabase(OB_SERVER, "column_type_test"); - columnTypesDatabase.createAndInitialize("mysql"); - waitForTableInitialization(); - - String sourceDDL = - String.format( - "CREATE TABLE ob_source (\n" - + " `id` INT NOT NULL,\n" - + " date_c DATE,\n" - + " time_c TIME(0),\n" - + " datetime3_c TIMESTAMP(3),\n" - + " datetime6_c TIMESTAMP(6),\n" - + " timestamp_c TIMESTAMP,\n" - + " primary key (`id`) not enforced" - + ") WITH (" - + initialOptionsString() - + "," - + " 'database-name' = '%s'," - + " 'table-name' = '%s'," - + " 'server-time-zone' = '%s'" - + ")", - columnTypesDatabase.getDatabaseName(), - "full_types", - serverTimeZone); - - String sinkDDL = - "CREATE TABLE sink (" - + " `id` INT NOT NULL,\n" - + " date_c DATE,\n" - + " time_c TIME(0),\n" - + " datetime3_c TIMESTAMP(3),\n" - + " datetime6_c TIMESTAMP(6),\n" - + " timestamp_c TIMESTAMP,\n" - + " primary key (`id`) not enforced" - + ") WITH (" - + " 'connector' = 'values'," - + " 'sink-insert-only' = 'false'," - + " 'sink-expected-messages-num' = '20'" - + ")"; - - tEnv.executeSql(sourceDDL); - tEnv.executeSql(sinkDDL); - - TableResult result = - tEnv.executeSql( - "INSERT INTO sink SELECT `id`, date_c, time_c, datetime3_c, datetime6_c, cast(timestamp_c as timestamp) FROM ob_source"); - - // wait for snapshot finished and begin binlog - waitForSinkSize("sink", 1); - int snapshotSize = sinkSize("sink"); - - try (Connection connection = getJdbcConnection(); - Statement statement = connection.createStatement()) { - statement.execute(String.format("USE %s;", columnTypesDatabase.getDatabaseName())); - statement.execute( - "UPDATE full_types SET timestamp_c = '2020-07-17 18:33:22' WHERE id=1;"); - } - - waitForSinkSize("sink", snapshotSize + 1); - - List expected = - Arrays.asList( - "+I(1,2020-07-17,18:00:22,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,2020-07-17T18:00:22)", - "+U(1,2020-07-17,18:00:22,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,2020-07-17T18:33:22)"); - - List actual = TestValuesTableFactory.getRawResultsAsStrings("sink"); - assertContainsInAnyOrder(expected, actual); - result.getJobClient().get().cancel().get(); - } - - @Test - void testSnapshotOnly() throws Exception { - inventoryDatabase = new UniqueDatabase(OB_SERVER, "inventory"); - inventoryDatabase.createAndInitialize("mysql"); - waitForTableInitialization(); - - String sourceDDL = - String.format( - "CREATE TABLE ob_source (" - + " `id` INT NOT NULL," - + " name STRING," - + " description STRING," - + " weight DECIMAL(20, 10)," - + " PRIMARY KEY (`id`) NOT ENFORCED" - + ") WITH (" - + snapshotOptionsString() - + ", " - + " 'table-list' = '%s'" - + ")", - inventoryDatabase.getDatabaseName() + ".products"); - - String sinkDDL = - "CREATE TABLE sink (" - + " `id` INT NOT NULL," - + " name STRING," - + " description STRING," - + " weight DECIMAL(20, 10)," - + " PRIMARY KEY (`id`) NOT ENFORCED" - + ") WITH (" - + " 'connector' = 'values'," - + " 'sink-insert-only' = 'false'," - + " 'sink-expected-messages-num' = '30'" - + ")"; - - tEnv.executeSql(sourceDDL); - tEnv.executeSql(sinkDDL); - - TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM ob_source"); - - waitForSinkSize("sink", 9); - - List expected = - Arrays.asList( - "+I(101,scooter,Small 2-wheel scooter,3.1400000000)", - "+I(102,car battery,12V car battery,8.1000000000)", - "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8000000000)", - "+I(104,hammer,12oz carpenter's hammer,0.7500000000)", - "+I(105,hammer,14oz carpenter's hammer,0.8750000000)", - "+I(106,hammer,16oz carpenter's hammer,1.0000000000)", - "+I(107,rocks,box of assorted rocks,5.3000000000)", - "+I(108,jacket,water resistent black wind breaker,0.1000000000)", - "+I(109,spare tire,24 inch spare tire,22.2000000000)"); - List actual = TestValuesTableFactory.getRawResultsAsStrings("sink"); - assertContainsInAnyOrder(expected, actual); - - try { - while (result.getJobClient() - .map(JobClient::getJobStatus) - .map(CompletableFuture::join) - .orElse(JobStatus.FINISHED) - .equals(JobStatus.RUNNING)) { - Thread.sleep(100); - // Waiting for job to finish (SNAPSHOT job will end spontaneously) - } - } catch (IllegalStateException e) { - // It's fine if miniCluster has been shut down - } - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseOracleModeITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseOracleModeITCase.java deleted file mode 100644 index aad8041fecc..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseOracleModeITCase.java +++ /dev/null @@ -1,227 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase.table; - -import org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestBase; -import org.apache.flink.cdc.connectors.oceanbase.testutils.OceanBaseCdcMetadata; -import org.apache.flink.cdc.connectors.oceanbase.testutils.OceanBaseOracleCdcMetadata; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableResult; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.table.planner.factories.TestValuesTableFactory; - -import org.junit.jupiter.api.Disabled; -import org.junit.jupiter.api.Test; - -import java.sql.Connection; -import java.sql.Statement; -import java.util.Arrays; -import java.util.List; - -/** Integration tests for OceanBase Oracle mode table source. */ -@Disabled("Test ignored before oceanbase-xe docker image is available") -class OceanBaseOracleModeITCase extends OceanBaseTestBase { - - private final StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment(); - private final StreamTableEnvironment tEnv = - StreamTableEnvironment.create( - env, EnvironmentSettings.newInstance().inStreamingMode().build()); - - private static final OceanBaseCdcMetadata METADATA = new OceanBaseOracleCdcMetadata(); - - @Override - protected OceanBaseCdcMetadata metadata() { - return METADATA; - } - - @Override - protected String logProxyOptionsString() { - return super.logProxyOptionsString() - + " , " - + String.format(" 'config-url' = '%s'", METADATA.getConfigUrl()); - } - - @Test - void testAllDataTypes() throws Exception { - initializeTable("column_type_test"); - - String schema = metadata().getDatabase(); - String sourceDDL = - String.format( - "CREATE TABLE full_types (" - + " ID INT NOT NULL," - + " VAL_VARCHAR STRING," - + " VAL_VARCHAR2 STRING," - + " VAL_NVARCHAR2 STRING," - + " VAL_CHAR STRING," - + " VAL_NCHAR STRING," - + " VAL_BF FLOAT," - + " VAL_BD DOUBLE," - + " VAL_F FLOAT," - + " VAL_F_10 FLOAT," - + " VAL_NUM DECIMAL(10, 6)," - + " VAL_DP DOUBLE," - + " VAL_R DECIMAL(38,2)," - + " VAL_DECIMAL DECIMAL(10, 6)," - + " VAL_NUMERIC DECIMAL(10, 6)," - + " VAL_NUM_VS DECIMAL(10, 3)," - + " VAL_INT DECIMAL(38,0)," - + " VAL_INTEGER DECIMAL(38,0)," - + " VAL_SMALLINT DECIMAL(38,0)," - + " VAL_NUMBER_38_NO_SCALE DECIMAL(38,0)," - + " VAL_NUMBER_38_SCALE_0 DECIMAL(38,0)," - + " VAL_NUMBER_1 BOOLEAN," - + " VAL_NUMBER_2 TINYINT," - + " VAL_NUMBER_4 SMALLINT," - + " VAL_NUMBER_9 INT," - + " VAL_NUMBER_18 BIGINT," - + " VAL_NUMBER_2_NEGATIVE_SCALE TINYINT," - + " VAL_NUMBER_4_NEGATIVE_SCALE SMALLINT," - + " VAL_NUMBER_9_NEGATIVE_SCALE INT," - + " VAL_NUMBER_18_NEGATIVE_SCALE BIGINT," - + " VAL_NUMBER_36_NEGATIVE_SCALE DECIMAL(38,0)," - + " VAL_DATE TIMESTAMP," - + " VAL_TS TIMESTAMP," - + " VAL_TS_PRECISION2 TIMESTAMP(2)," - + " VAL_TS_PRECISION4 TIMESTAMP(4)," - + " VAL_TS_PRECISION9 TIMESTAMP(6)," - + " VAL_CLOB_INLINE STRING," - + " VAL_BLOB_INLINE BYTES," - + " PRIMARY KEY (ID) NOT ENFORCED" - + ") WITH (" - + initialOptionsString() - + ", " - + " 'table-list' = '%s'" - + ")", - schema + ".FULL_TYPES"); - - String sinkDDL = - "CREATE TABLE sink (" - + " ID INT," - + " VAL_VARCHAR STRING," - + " VAL_VARCHAR2 STRING," - + " VAL_NVARCHAR2 STRING," - + " VAL_CHAR STRING," - + " VAL_NCHAR STRING," - + " VAL_BF FLOAT," - + " VAL_BD DOUBLE," - + " VAL_F FLOAT," - + " VAL_F_10 FLOAT," - + " VAL_NUM DECIMAL(10, 6)," - + " VAL_DP DOUBLE," - + " VAL_R DECIMAL(38,2)," - + " VAL_DECIMAL DECIMAL(10, 6)," - + " VAL_NUMERIC DECIMAL(10, 6)," - + " VAL_NUM_VS DECIMAL(10, 3)," - + " VAL_INT DECIMAL(38,0)," - + " VAL_INTEGER DECIMAL(38,0)," - + " VAL_SMALLINT DECIMAL(38,0)," - + " VAL_NUMBER_38_NO_SCALE DECIMAL(38,0)," - + " VAL_NUMBER_38_SCALE_0 DECIMAL(38,0)," - + " VAL_NUMBER_1 BOOLEAN," - + " VAL_NUMBER_2 TINYINT," - + " VAL_NUMBER_4 SMALLINT," - + " VAL_NUMBER_9 INT," - + " VAL_NUMBER_18 BIGINT," - + " VAL_NUMBER_2_NEGATIVE_SCALE TINYINT," - + " VAL_NUMBER_4_NEGATIVE_SCALE SMALLINT," - + " VAL_NUMBER_9_NEGATIVE_SCALE INT," - + " VAL_NUMBER_18_NEGATIVE_SCALE BIGINT," - + " VAL_NUMBER_36_NEGATIVE_SCALE DECIMAL(38,0)," - + " VAL_DATE TIMESTAMP," - + " VAL_TS TIMESTAMP," - + " VAL_TS_PRECISION2 TIMESTAMP(2)," - + " VAL_TS_PRECISION4 TIMESTAMP(4)," - + " VAL_TS_PRECISION9 TIMESTAMP(6)," - + " VAL_CLOB_INLINE STRING," - + " VAL_BLOB_INLINE STRING," - + " PRIMARY KEY (ID) NOT ENFORCED" - + ") WITH (" - + " 'connector' = 'values'," - + " 'sink-insert-only' = 'false'," - + " 'sink-expected-messages-num' = '2'" - + ")"; - - tEnv.executeSql(sourceDDL); - tEnv.executeSql(sinkDDL); - - TableResult result = - tEnv.executeSql( - "INSERT INTO sink SELECT " - + " ID," - + " VAL_VARCHAR," - + " VAL_VARCHAR2," - + " VAL_NVARCHAR2," - + " VAL_CHAR," - + " VAL_NCHAR," - + " VAL_BF," - + " VAL_BD," - + " VAL_F," - + " VAL_F_10," - + " VAL_NUM," - + " VAL_DP," - + " VAL_R," - + " VAL_DECIMAL," - + " VAL_NUMERIC," - + " VAL_NUM_VS," - + " VAL_INT," - + " VAL_INTEGER," - + " VAL_SMALLINT," - + " VAL_NUMBER_38_NO_SCALE," - + " VAL_NUMBER_38_SCALE_0," - + " VAL_NUMBER_1," - + " VAL_NUMBER_2," - + " VAL_NUMBER_4," - + " VAL_NUMBER_9," - + " VAL_NUMBER_18," - + " VAL_NUMBER_2_NEGATIVE_SCALE," - + " VAL_NUMBER_4_NEGATIVE_SCALE," - + " VAL_NUMBER_9_NEGATIVE_SCALE," - + " VAL_NUMBER_18_NEGATIVE_SCALE," - + " VAL_NUMBER_36_NEGATIVE_SCALE," - + " VAL_DATE," - + " VAL_TS," - + " VAL_TS_PRECISION2," - + " VAL_TS_PRECISION4," - + " VAL_TS_PRECISION9," - + " VAL_CLOB_INLINE," - + " DECODE(VAL_BLOB_INLINE, 'UTF-8')" - + " FROM full_types"); - - waitForSinkSize("sink", 1); - - try (Connection connection = getJdbcConnection(); - Statement statement = connection.createStatement()) { - statement.execute( - "UPDATE FULL_TYPES SET VAL_TS = '2022-10-30 12:34:56.12545' WHERE id=1;"); - } - - waitForSinkSize("sink", 2); - - List expected = - Arrays.asList( - "+I(1,vc2,vc2,nvc2,c ,nc ,1.1,2.22,3.33,8.888,4.444400,5.555,6.66,1234.567891,1234.567891,77.323,1,22,333,4444,5555,true,99,9999,999999999,999999999999999999,90,9900,999999990,999999999999999900,99999999999999999999999999999999999900,2022-10-30T00:00,2022-10-30T12:34:56.007890,2022-10-30T12:34:56.130,2022-10-30T12:34:56.125500,2022-10-30T12:34:56.125457,col_clob,col_blob)", - "+U(1,vc2,vc2,nvc2,c ,nc ,1.1,2.22,3.33,8.888,4.444400,5.555,6.66,1234.567891,1234.567891,77.323,1,22,333,4444,5555,true,99,9999,999999999,999999999999999999,90,9900,999999990,999999999999999900,99999999999999999999999999999999999900,2022-10-30T00:00,2022-10-30T12:34:56.125450,2022-10-30T12:34:56.130,2022-10-30T12:34:56.125500,2022-10-30T12:34:56.125457,col_clob,col_blob)"); - - List actual = TestValuesTableFactory.getRawResultsAsStrings("sink"); - assertContainsInAnyOrder(expected, actual); - result.getJobClient().get().cancel().get(); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableFactoryTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableFactoryTest.java deleted file mode 100644 index 9a4dbfcddf3..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableFactoryTest.java +++ /dev/null @@ -1,261 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase.table; - -import org.apache.flink.cdc.connectors.base.options.StartupOptions; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.Schema; -import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.catalog.Column; -import org.apache.flink.table.catalog.ObjectIdentifier; -import org.apache.flink.table.catalog.ResolvedCatalogTable; -import org.apache.flink.table.catalog.ResolvedSchema; -import org.apache.flink.table.catalog.UniqueConstraint; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.factories.FactoryUtil; - -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.Test; - -import java.time.Duration; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.Properties; - -/** Test for {@link OceanBaseTableSource} created by {@link OceanBaseTableSourceFactory}. */ -class OceanBaseTableFactoryTest { - - private static final ResolvedSchema SCHEMA = - new ResolvedSchema( - Arrays.asList( - Column.physical("aaa", DataTypes.INT().notNull()), - Column.physical("bbb", DataTypes.STRING().notNull()), - Column.physical("ccc", DataTypes.DOUBLE()), - Column.physical("ddd", DataTypes.DECIMAL(31, 18)), - Column.physical("eee", DataTypes.TIMESTAMP(3))), - Collections.emptyList(), - UniqueConstraint.primaryKey("pk", Collections.singletonList("aaa"))); - - private static final ResolvedSchema SCHEMA_WITH_METADATA = - new ResolvedSchema( - Arrays.asList( - Column.physical("aaa", DataTypes.INT().notNull()), - Column.physical("bbb", DataTypes.STRING().notNull()), - Column.physical("ccc", DataTypes.DOUBLE()), - Column.physical("ddd", DataTypes.DECIMAL(31, 18)), - Column.physical("eee", DataTypes.TIMESTAMP(3)), - Column.metadata("time", DataTypes.TIMESTAMP_LTZ(3), "op_ts", true), - Column.metadata("tenant", DataTypes.STRING(), "tenant_name", true), - Column.metadata("database", DataTypes.STRING(), "database_name", true), - Column.metadata("table", DataTypes.STRING(), "table_name", true)), - Collections.emptyList(), - UniqueConstraint.primaryKey("pk", Collections.singletonList("aaa"))); - - private static final String STARTUP_MODE = "latest-offset"; - private static final String USERNAME = "user@sys"; - private static final String PASSWORD = "pswd"; - private static final String TENANT_NAME = "sys"; - private static final String DATABASE_NAME = "db[0-9]"; - private static final String TABLE_NAME = "table[0-9]"; - private static final String TABLE_LIST = "db.table"; - private static final String SERVER_TIME_ZONE = "+00:00"; - private static final String CONNECT_TIMEOUT = "30s"; - private static final String HOSTNAME = "127.0.0.1"; - private static final Integer PORT = 2881; - private static final String COMPATIBLE_MODE = "mysql"; - private static final String DRIVER_CLASS = "com.mysql.cj.jdbc.Driver"; - private static final String LOG_PROXY_HOST = "127.0.0.1"; - private static final Integer LOG_PROXY_PORT = 2983; - private static final String LOG_PROXY_CLIENT_ID = "clientId"; - private static final String RS_LIST = "127.0.0.1:2882:2881"; - private static final String WORKING_MODE = "storage"; - - @Test - void testCommonProperties() { - Map options = getRequiredOptions(); - options.put("database-name", DATABASE_NAME); - options.put("table-name", TABLE_NAME); - options.put("table-list", TABLE_LIST); - options.put("rootserver-list", RS_LIST); - - DynamicTableSource actualSource = createTableSource(SCHEMA, options); - OceanBaseTableSource expectedSource = - new OceanBaseTableSource( - SCHEMA, - StartupOptions.latest(), - USERNAME, - PASSWORD, - TENANT_NAME, - DATABASE_NAME, - TABLE_NAME, - TABLE_LIST, - SERVER_TIME_ZONE, - Duration.parse("PT" + CONNECT_TIMEOUT), - HOSTNAME, - PORT, - COMPATIBLE_MODE, - DRIVER_CLASS, - new Properties(), - LOG_PROXY_HOST, - LOG_PROXY_PORT, - null, - null, - RS_LIST, - null, - WORKING_MODE, - new Properties(), - new Properties()); - Assertions.assertThat(actualSource).isEqualTo(expectedSource); - } - - @Test - void testOptionalProperties() { - Map options = getRequiredOptions(); - options.put("scan.startup.mode", "initial"); - options.put("database-name", DATABASE_NAME); - options.put("table-name", TABLE_NAME); - options.put("table-list", TABLE_LIST); - options.put("compatible-mode", COMPATIBLE_MODE); - options.put("jdbc.driver", DRIVER_CLASS); - options.put("logproxy.client.id", LOG_PROXY_CLIENT_ID); - options.put("rootserver-list", RS_LIST); - DynamicTableSource actualSource = createTableSource(SCHEMA, options); - - OceanBaseTableSource expectedSource = - new OceanBaseTableSource( - SCHEMA, - StartupOptions.initial(), - USERNAME, - PASSWORD, - TENANT_NAME, - DATABASE_NAME, - TABLE_NAME, - TABLE_LIST, - SERVER_TIME_ZONE, - Duration.parse("PT" + CONNECT_TIMEOUT), - "127.0.0.1", - 2881, - COMPATIBLE_MODE, - DRIVER_CLASS, - new Properties(), - LOG_PROXY_HOST, - LOG_PROXY_PORT, - LOG_PROXY_CLIENT_ID, - null, - RS_LIST, - null, - WORKING_MODE, - new Properties(), - new Properties()); - Assertions.assertThat(actualSource).isEqualTo(expectedSource); - } - - @Test - void testMetadataColumns() { - Map options = getRequiredOptions(); - options.put("database-name", DATABASE_NAME); - options.put("table-name", TABLE_NAME); - options.put("table-list", TABLE_LIST); - options.put("rootserver-list", RS_LIST); - - DynamicTableSource actualSource = createTableSource(SCHEMA_WITH_METADATA, options); - OceanBaseTableSource oceanBaseTableSource = (OceanBaseTableSource) actualSource; - oceanBaseTableSource.applyReadableMetadata( - Arrays.asList("op_ts", "tenant_name", "database_name", "table_name"), - SCHEMA_WITH_METADATA.toSourceRowDataType()); - actualSource = oceanBaseTableSource.copy(); - - OceanBaseTableSource expectedSource = - new OceanBaseTableSource( - SCHEMA_WITH_METADATA, - StartupOptions.latest(), - USERNAME, - PASSWORD, - TENANT_NAME, - DATABASE_NAME, - TABLE_NAME, - TABLE_LIST, - SERVER_TIME_ZONE, - Duration.parse("PT" + CONNECT_TIMEOUT), - HOSTNAME, - PORT, - COMPATIBLE_MODE, - DRIVER_CLASS, - new Properties(), - LOG_PROXY_HOST, - LOG_PROXY_PORT, - null, - null, - RS_LIST, - null, - WORKING_MODE, - new Properties(), - new Properties()); - expectedSource.producedDataType = SCHEMA_WITH_METADATA.toSourceRowDataType(); - expectedSource.metadataKeys = - Arrays.asList("op_ts", "tenant_name", "database_name", "table_name"); - - Assertions.assertThat(actualSource).isEqualTo(expectedSource); - } - - @Test - void testValidation() { - Assertions.assertThatThrownBy( - () -> { - Map properties = getRequiredOptions(); - properties.put("unknown", "abc"); - createTableSource(SCHEMA, properties); - }) - .hasStackTraceContaining("Unsupported options:\n\nunknown"); - } - - private Map getRequiredOptions() { - Map options = new HashMap<>(); - options.put("connector", "oceanbase-cdc"); - options.put("scan.startup.mode", STARTUP_MODE); - options.put("username", USERNAME); - options.put("password", PASSWORD); - options.put("hostname", HOSTNAME); - options.put("port", String.valueOf(PORT)); - options.put("tenant-name", TENANT_NAME); - options.put("logproxy.host", LOG_PROXY_HOST); - options.put("logproxy.port", String.valueOf(LOG_PROXY_PORT)); - return options; - } - - private static DynamicTableSource createTableSource( - ResolvedSchema schema, Map options) { - return FactoryUtil.createTableSource( - null, - ObjectIdentifier.of("default", "default", "t1"), - new ResolvedCatalogTable( - CatalogTable.of( - Schema.newBuilder().fromResolvedSchema(schema).build(), - "mock source", - new ArrayList<>(), - options), - schema), - new Configuration(), - OceanBaseTableFactoryTest.class.getClassLoader(), - false); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/LogProxyContainer.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/LogProxyContainer.java deleted file mode 100644 index c33eccbb428..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/LogProxyContainer.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase.testutils; - -import org.jetbrains.annotations.NotNull; -import org.testcontainers.containers.GenericContainer; -import org.testcontainers.containers.wait.strategy.Wait; -import org.testcontainers.utility.DockerImageName; - -import java.util.Collections; -import java.util.Set; - -/** OceanBase Log Proxy container. */ -public class LogProxyContainer extends GenericContainer { - - private static final String IMAGE = "oceanbase/oblogproxy-ce"; - - private static final int PORT = 2983; - private static final String ROOT_USER = "root"; - - private String sysPassword; - - public LogProxyContainer(String version) { - super(DockerImageName.parse(IMAGE + ":" + version)); - addExposedPorts(PORT); - setWaitStrategy(Wait.forLogMessage(".*boot success!.*", 1)); - } - - @Override - protected void configure() { - addEnv("OB_SYS_USERNAME", ROOT_USER); - addEnv("OB_SYS_PASSWORD", sysPassword); - } - - public @NotNull Set getLivenessCheckPortNumbers() { - return Collections.singleton(this.getMappedPort(PORT)); - } - - public int getPort() { - return getMappedPort(PORT); - } - - public LogProxyContainer withSysPassword(String sysPassword) { - this.sysPassword = sysPassword; - return this; - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/OceanBaseCdcMetadata.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/OceanBaseCdcMetadata.java deleted file mode 100644 index bb2469509d8..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/OceanBaseCdcMetadata.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase.testutils; - -import java.io.Serializable; - -/** OceanBase CDC metadata. */ -public interface OceanBaseCdcMetadata extends Serializable { - - String getCompatibleMode(); - - String getHostname(); - - int getPort(); - - String getUsername(); - - String getPassword(); - - String getDriverClass(); - - String getDatabase(); - - String getJdbcUrl(); - - String getTenantName(); - - String getLogProxyHost(); - - int getLogProxyPort(); - - default String getConfigUrl() { - return null; - } - - default String getRsList() { - return null; - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/OceanBaseMySQLCdcMetadata.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/OceanBaseMySQLCdcMetadata.java deleted file mode 100644 index aede8a2924f..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/OceanBaseMySQLCdcMetadata.java +++ /dev/null @@ -1,113 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase.testutils; - -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; - -/** OceanBase CDC MySQL mode metadata. */ -public class OceanBaseMySQLCdcMetadata implements OceanBaseCdcMetadata { - - private final OceanBaseContainer obServerContainer; - private final LogProxyContainer logProxyContainer; - - private String rsList; - - public OceanBaseMySQLCdcMetadata( - OceanBaseContainer obServerContainer, LogProxyContainer logProxyContainer) { - this.obServerContainer = obServerContainer; - this.logProxyContainer = logProxyContainer; - } - - @Override - public String getCompatibleMode() { - return "mysql"; - } - - @Override - public String getHostname() { - return obServerContainer.getHost(); - } - - @Override - public int getPort() { - return obServerContainer.getDatabasePort(); - } - - @Override - public String getUsername() { - return obServerContainer.getUsername(); - } - - @Override - public String getPassword() { - return obServerContainer.getPassword(); - } - - @Override - public String getDriverClass() { - return obServerContainer.getDriverClassName(); - } - - @Override - public String getDatabase() { - return obServerContainer.getDatabaseName(); - } - - @Override - public String getJdbcUrl() { - return "jdbc:mysql://" + getHostname() + ":" + getPort() + "/?useSSL=false"; - } - - @Override - public String getTenantName() { - return obServerContainer.getTenantName(); - } - - @Override - public String getLogProxyHost() { - return logProxyContainer.getHost(); - } - - @Override - public int getLogProxyPort() { - return logProxyContainer.getPort(); - } - - @Override - public String getRsList() { - if (rsList == null) { - try (Connection connection = - DriverManager.getConnection( - getJdbcUrl(), getUsername(), getPassword()); - Statement statement = connection.createStatement()) { - ResultSet rs = statement.executeQuery("SHOW PARAMETERS LIKE 'rootservice_list'"); - rsList = rs.next() ? rs.getString("VALUE") : null; - } catch (SQLException e) { - throw new RuntimeException("Failed to query rs list", e); - } - if (rsList == null) { - throw new RuntimeException("Got empty rs list"); - } - } - return rsList; - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/OceanBaseOracleCdcMetadata.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/OceanBaseOracleCdcMetadata.java deleted file mode 100644 index c68fdda037b..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/OceanBaseOracleCdcMetadata.java +++ /dev/null @@ -1,82 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase.testutils; - -/** OceanBase CDC Oracle mode metadata. */ -public class OceanBaseOracleCdcMetadata implements OceanBaseCdcMetadata { - - @Override - public String getCompatibleMode() { - return "oracle"; - } - - @Override - public String getHostname() { - return System.getenv("host"); - } - - @Override - public int getPort() { - return Integer.parseInt(System.getenv("port")); - } - - @Override - public String getUsername() { - return System.getenv("username"); - } - - @Override - public String getPassword() { - return System.getenv("password"); - } - - @Override - public String getDatabase() { - return System.getenv("schema"); - } - - @Override - public String getDriverClass() { - return "com.oceanbase.jdbc.Driver"; - } - - @Override - public String getJdbcUrl() { - return "jdbc:oceanbase://" + getHostname() + ":" + getPort() + "/" + getDatabase(); - } - - @Override - public String getTenantName() { - return System.getenv("tenant"); - } - - @Override - public String getLogProxyHost() { - return System.getenv("log_proxy_host"); - } - - @Override - public int getLogProxyPort() { - return Integer.parseInt(System.getenv("log_proxy_port")); - } - - @Override - public String getConfigUrl() { - return System.getenv("config_url"); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/UniqueDatabase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/UniqueDatabase.java deleted file mode 100644 index bb59d98d6f0..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/UniqueDatabase.java +++ /dev/null @@ -1,163 +0,0 @@ -/* - * 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.flink.cdc.connectors.oceanbase.testutils; - -import org.assertj.core.api.Assertions; - -import java.net.URL; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; -import java.sql.Statement; -import java.util.Arrays; -import java.util.List; -import java.util.Random; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -/** - * Create and populate a unique instance of an OceanBase database for each run of JUnit test. A user - * of class needs to provide a logical name for Debezium and database name. It is expected that - * there is an init file in src/test/resources/ddl/<database_name>.sql. The - * database name is enriched with a unique suffix that guarantees complete isolation between runs - * - * <database_name>_<suffix> - * - *

    This class is inspired from Debezium project. - */ -public class UniqueDatabase { - - private static final String[] CREATE_DATABASE_DDL = - new String[] {"CREATE DATABASE `$DBNAME$`;", "USE `$DBNAME$`;"}; - private static final String DROP_DATABASE_DDL = "DROP DATABASE IF EXISTS `$DBNAME$`;"; - private static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)--.*$"); - - private final OceanBaseContainer container; - private final String databaseName; - private final String templateName; - - public UniqueDatabase(OceanBaseContainer container, String databaseName) { - this(container, databaseName, Integer.toUnsignedString(new Random().nextInt(), 36)); - } - - private UniqueDatabase( - OceanBaseContainer container, String databaseName, final String identifier) { - this.container = container; - this.databaseName = databaseName + "_" + identifier; - this.templateName = databaseName; - } - - public String getHost() { - return container.getHost(); - } - - public int getDatabasePort() { - return container.getDatabasePort(); - } - - public String getDatabaseName() { - return databaseName; - } - - public String getUsername() { - return container.getUsername(); - } - - public String getPassword() { - return container.getPassword(); - } - - /** @return Fully qualified table name <databaseName>.<tableName> */ - public String qualifiedTableName(final String tableName) { - return String.format("%s.%s", databaseName, tableName); - } - - public void createAndInitialize() { - createAndInitializeWithDdlFile(String.format("ddl/%s.sql", templateName)); - } - - public void createAndInitialize(String variant) { - createAndInitializeWithDdlFile(String.format("ddl/%s/%s.sql", variant, templateName)); - } - - /** Creates the database and populates it with initialization SQL script. */ - public void createAndInitializeWithDdlFile(String ddlFile) { - final URL ddlTestFile = UniqueDatabase.class.getClassLoader().getResource(ddlFile); - Assertions.assertThat(ddlTestFile).withFailMessage("Cannot locate " + ddlFile).isNotNull(); - try { - try (Connection connection = - DriverManager.getConnection( - container.getJdbcUrl(), getUsername(), getPassword()); - Statement statement = connection.createStatement()) { - final List statements = - Arrays.stream( - Stream.concat( - Arrays.stream(CREATE_DATABASE_DDL), - Files.readAllLines( - Paths.get(ddlTestFile.toURI())) - .stream()) - .map(String::trim) - .filter(x -> !x.startsWith("--") && !x.isEmpty()) - .map( - x -> { - final Matcher m = - COMMENT_PATTERN.matcher(x); - return m.matches() ? m.group(1) : x; - }) - .map(this::convertSQL) - .collect(Collectors.joining("\n")) - .split(";")) - .map(x -> x.replace("$$", ";")) - .collect(Collectors.toList()); - for (String stmt : statements) { - statement.execute(stmt); - } - } - } catch (final Exception e) { - throw new IllegalStateException(e); - } - } - - /** Drop the database if it is existing. */ - public void dropDatabase() { - try { - try (Connection connection = - DriverManager.getConnection( - container.getJdbcUrl(), getUsername(), getPassword()); - Statement statement = connection.createStatement()) { - final String dropDatabaseStatement = convertSQL(DROP_DATABASE_DDL); - statement.execute(dropDatabaseStatement); - } - } catch (final Exception e) { - throw new IllegalStateException(e); - } - } - - public Connection getJdbcConnection() throws SQLException { - return DriverManager.getConnection( - container.getJdbcUrl(databaseName), getUsername(), getPassword()); - } - - private String convertSQL(final String sql) { - return sql.replace("$DBNAME$", databaseName); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/charset_test.sql b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/charset_test.sql new file mode 100644 index 00000000000..552270acb6c --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/charset_test.sql @@ -0,0 +1,172 @@ +-- 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. + +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: charset_test +-- ---------------------------------------------------------------------------------------------------------------- + +CREATE TABLE `ascii_test` ( + `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号', + `table_name` varchar(200) CHARACTER SET ascii DEFAULT '' COMMENT '表名称', + PRIMARY KEY (`table_id`) USING BTREE +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=ascii; + +CREATE TABLE `big5_test` ( + `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号', + `table_name` varchar(200) CHARACTER SET big5 DEFAULT '' COMMENT '表名称', + PRIMARY KEY (`table_id`) USING BTREE +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=big5; + +CREATE TABLE `gbk_test` ( + `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号', + `table_name` varchar(200) CHARACTER SET gbk DEFAULT '' COMMENT '表名称', + PRIMARY KEY (`table_id`) USING BTREE +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=gbk; + +CREATE TABLE `sjis_test` ( + `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号', + `table_name` varchar(200) CHARACTER SET sjis DEFAULT '' COMMENT '表名称', + PRIMARY KEY (`table_id`) USING BTREE +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=sjis; + +CREATE TABLE `cp932_test` ( + `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号', + `table_name` varchar(200) CHARACTER SET cp932 DEFAULT '' COMMENT '表名称', + PRIMARY KEY (`table_id`) USING BTREE +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=cp932; + +CREATE TABLE `gb2312_test` ( + `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号', + `table_name` varchar(200) CHARACTER SET gb2312 DEFAULT '' COMMENT '表名称', + PRIMARY KEY (`table_id`) USING BTREE +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=gb2312; + +CREATE TABLE `ujis_test` ( + `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号', + `table_name` varchar(200) CHARACTER SET ujis DEFAULT '' COMMENT '表名称', + PRIMARY KEY (`table_id`) USING BTREE +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=ujis; + +CREATE TABLE `euckr_test` ( + `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号', + `table_name` varchar(200) CHARACTER SET euckr DEFAULT '' COMMENT '表名称', + PRIMARY KEY (`table_id`) USING BTREE +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=euckr; + +CREATE TABLE `latin1_test` ( + `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号', + `table_name` varchar(200) CHARACTER SET latin1 DEFAULT '' COMMENT '表名称', + PRIMARY KEY (`table_id`) USING BTREE +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=latin1; + + +CREATE TABLE `latin2_test` ( + `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号', + `table_name` varchar(200) CHARACTER SET latin2 DEFAULT '' COMMENT '表名称', + PRIMARY KEY (`table_id`) USING BTREE +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=latin2; + + +CREATE TABLE `greek_test` ( + `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号', + `table_name` varchar(200) CHARACTER SET greek DEFAULT '' COMMENT '表名称', + PRIMARY KEY (`table_id`) USING BTREE +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=greek; + + +CREATE TABLE `hebrew_test` ( + `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号', + `table_name` varchar(200) CHARACTER SET hebrew DEFAULT '' COMMENT '表名称', + PRIMARY KEY (`table_id`) USING BTREE +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=hebrew; + + +CREATE TABLE `cp866_test` ( + `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号', + `table_name` varchar(200) CHARACTER SET cp866 DEFAULT '' COMMENT '表名称', + PRIMARY KEY (`table_id`) USING BTREE +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=cp866; + + +CREATE TABLE `tis620_test` ( + `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号', + `table_name` varchar(200) CHARACTER SET tis620 DEFAULT '' COMMENT '表名称', + PRIMARY KEY (`table_id`) USING BTREE +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=tis620; + + +CREATE TABLE `cp1250_test` ( + `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号', + `table_name` varchar(200) CHARACTER SET cp1250 DEFAULT '' COMMENT '表名称', + PRIMARY KEY (`table_id`) USING BTREE +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=cp1250; + +CREATE TABLE `cp1251_test` ( + `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号', + `table_name` varchar(200) CHARACTER SET cp1251 DEFAULT '' COMMENT '表名称', + PRIMARY KEY (`table_id`) USING BTREE +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=cp1251; + +CREATE TABLE `cp1257_test` ( + `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号', + `table_name` varchar(200) CHARACTER SET cp1257 DEFAULT '' COMMENT '表名称', + PRIMARY KEY (`table_id`) USING BTREE +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=cp1257; + +CREATE TABLE `macroman_test` ( + `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号', + `table_name` varchar(200) CHARACTER SET macroman DEFAULT '' COMMENT '表名称', + PRIMARY KEY (`table_id`) USING BTREE +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=macroman; + +CREATE TABLE `macce_test` ( + `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号', + `table_name` varchar(200) CHARACTER SET macce DEFAULT '' COMMENT '表名称', + PRIMARY KEY (`table_id`) USING BTREE +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=macce; + +CREATE TABLE `utf8_test` ( + `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号', + `table_name` varchar(200) CHARACTER SET utf8 DEFAULT '' COMMENT '表名称', + PRIMARY KEY (`table_id`) USING BTREE +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=utf8; + +CREATE TABLE `ucs2_test` ( + `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号', + `table_name` varchar(200) CHARACTER SET ucs2 DEFAULT '' COMMENT '表名称', + PRIMARY KEY (`table_id`) USING BTREE +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=ucs2; + +INSERT into `ascii_test` values (1, 'ascii test!?'), (2, 'Craig Marshall'), (3, '{test}'); +INSERT into `big5_test` values (1, '大五'), (2, 'Craig Marshall'), (3, '丹店'); +INSERT into `gbk_test` values (1, '测试数据'), (2, 'Craig Marshall'), (3, '另一个测试数据'); +INSERT into `sjis_test` values (1, 'ひびぴ'), (2, 'Craig Marshall'), (3, 'フブプ'); +INSERT into `cp932_test` values (1, 'ひびぴ'), (2, 'Craig Marshall'), (3, 'フブプ'); +INSERT into `gb2312_test` values (1, '测试数据'), (2, 'Craig Marshall'), (3, '另一个测试数据'); +INSERT into `ujis_test` values (1, 'ひびぴ'), (2, 'Craig Marshall'), (3, 'フブプ'); +INSERT into `euckr_test` values (1, '죠주쥬'), (2, 'Craig Marshall'), (3, '한국어'); +INSERT into `latin1_test` values (1, 'ÀÆÉ'), (2, 'Craig Marshall'), (3, 'Üæû'); +INSERT into `latin2_test` values (1, 'ÓÔŐÖ'), (2, 'Craig Marshall'), (3, 'ŠŞŤŹ'); +INSERT into `greek_test` values (1, 'αβγδε'), (2, 'Craig Marshall'), (3, 'θικλ'); +INSERT into `hebrew_test` values (1, 'בבקשה'), (2, 'Craig Marshall'), (3, 'שרפה'); +INSERT into `cp866_test` values (1, 'твой'), (2, 'Craig Marshall'), (3, 'любой'); +INSERT into `tis620_test` values (1, 'ภาษาไทย'), (2, 'Craig Marshall'), (3, 'ฆงจฉ'); +INSERT into `cp1250_test` values (1, 'ÓÔŐÖ'), (2, 'Craig Marshall'), (3, 'ŠŞŤŹ'); +INSERT into `cp1251_test` values (1, 'твой'), (2, 'Craig Marshall'), (3, 'любой'); +INSERT into `cp1257_test` values (1, 'piedzimst brīvi'), (2, 'Craig Marshall'), (3, 'apveltīti ar saprātu'); +INSERT into `macroman_test` values (1, 'ÀÆÉ'), (2, 'Craig Marshall'), (3, 'Üæû'); +INSERT into `macce_test` values (1, 'ÓÔŐÖ'), (2, 'Craig Marshall'), (3, 'ŮÚŰÜ'); +INSERT into `utf8_test` values (1, '测试数据'), (2, 'Craig Marshall'), (3, '另一个测试数据'); +INSERT into `ucs2_test` values (1, '测试数据'), (2, 'Craig Marshall'), (3, '另一个测试数据'); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/column_type_test.sql b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/column_type_test.sql deleted file mode 100644 index 1d6ae4d70dc..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/column_type_test.sql +++ /dev/null @@ -1,74 +0,0 @@ --- 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. - --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: column_type_test --- ---------------------------------------------------------------------------------------------------------------- - -DROP TABLE IF EXISTS full_types; -CREATE TABLE full_types -( - id INT AUTO_INCREMENT NOT NULL, - bit1_c BIT, - tiny1_c TINYINT(1), - boolean_c BOOLEAN, - tiny_c TINYINT, - tiny_un_c TINYINT UNSIGNED, - small_c SMALLINT, - small_un_c SMALLINT UNSIGNED, - medium_c MEDIUMINT, - medium_un_c MEDIUMINT UNSIGNED, - int11_c INT(11), - int_c INTEGER, - int_un_c INTEGER UNSIGNED, - big_c BIGINT, - big_un_c BIGINT UNSIGNED, - real_c REAL, - float_c FLOAT, - double_c DOUBLE, - decimal_c DECIMAL(8, 4), - numeric_c NUMERIC(6, 0), - big_decimal_c DECIMAL(65, 1), - date_c DATE, - time_c TIME(0), - datetime3_c DATETIME(3), - datetime6_c DATETIME(6), - timestamp_c TIMESTAMP, - timestamp3_c TIMESTAMP(3), - timestamp6_c TIMESTAMP(6), - char_c CHAR(3), - varchar_c VARCHAR(255), - file_uuid BINARY(16), - bit_c BIT(64), - text_c TEXT, - tiny_blob_c TINYBLOB, - medium_blob_c MEDIUMBLOB, - blob_c BLOB, - long_blob_c LONGBLOB, - year_c YEAR, - set_c SET ('a', 'b'), - enum_c ENUM ('red', 'green', 'blue'), - json_c JSON, - PRIMARY KEY (id) -) DEFAULT CHARSET = utf8mb4; - -INSERT INTO full_types -VALUES (DEFAULT, 0, 1, true, 127, 255, 32767, 65535, 8388607, 16777215, 2147483647, 2147483647, 4294967295, - 9223372036854775807, 18446744073709551615, 123.102, 123.102, 404.4443, 123.4567, 345.6, 34567892.1, - '2020-07-17', '18:00:22', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456', - '2020-07-17 18:00:22', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456', - 'abc', 'Hello World', unhex(replace('651aed08-390f-4893-b2f1-36923e7b7400', '-', '')), - b'0000010000000100000001000000010000000100000001000000010000000100', 'text', UNHEX(HEX(16)), UNHEX(HEX(16)), - UNHEX(HEX(16)), UNHEX(HEX(16)), 2022, 'a,b,a', 'red', '{"key1":"value1"}'); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/inventory.sql b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/inventory.sql deleted file mode 100644 index e0c56ce1e7a..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/inventory.sql +++ /dev/null @@ -1,39 +0,0 @@ --- 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. - --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: inventory --- ---------------------------------------------------------------------------------------------------------------- - -DROP TABLE IF EXISTS products; -CREATE TABLE products -( - id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, - name VARCHAR(255) NOT NULL DEFAULT 'flink', - description VARCHAR(512), - weight DECIMAL(20, 10) -); -ALTER TABLE products AUTO_INCREMENT = 101; - -INSERT INTO products -VALUES (default, "scooter", "Small 2-wheel scooter", 3.14), - (default, "car battery", "12V car battery", 8.1), - (default, "12-pack drill bits", "12-pack of drill bits with sizes ranging from #40 to #3", 0.8), - (default, "hammer", "12oz carpenter's hammer", 0.75), - (default, "hammer", "14oz carpenter's hammer", 0.875), - (default, "hammer", "16oz carpenter's hammer", 1.0), - (default, "rocks", "box of assorted rocks", 5.3), - (default, "jacket", "water resistent black wind breaker", 0.1), - (default, "spare tire", "24 inch spare tire", 22.2); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/oceanbase_ddl_test.sql b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/oceanbase_ddl_test.sql new file mode 100644 index 00000000000..cc5c30b2fda --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/oceanbase_ddl_test.sql @@ -0,0 +1,232 @@ +-- 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. + +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: oceanbase_ddl_test +-- ---------------------------------------------------------------------------------------------------------------- + +-- Create orders table with single primary key +create table orders ( + id bigint not null auto_increment, + seller_id varchar(30) DEFAULT NULL, + order_id varchar(30) DEFAULT NULL, + buyer_id varchar(30) DEFAULT NULL, + create_time datetime DEFAULT NULL, + primary key(id), + INDEX `i_seller`(`seller_id`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8 partition by key(id) partitions 3; + +-- insert some orders for testing +INSERT INTO orders +VALUES (1, 1001, 1, 102, '2022-01-16'), + (2, 1002, 2, 105, '2022-01-16'), + (3, 1004, 3, 109, '2022-01-16'), + (4, 1002, 2, 106, '2022-01-16'), + (5, 1003, 1, 107, '2022-01-16'); + +-- Create orders with multi primary keys +create table orders_with_multi_pks ( + id bigint not null auto_increment, + seller_id varchar(30) DEFAULT NULL, + order_id varchar(30) NOT NULL, + buyer_id varchar(30) DEFAULT NULL, + create_time datetime DEFAULT NULL, + primary key(id, order_id), + INDEX `g_mi_seller`(`seller_id`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8 partition by key(id, order_id) partitions 3; + +-- insert some orders for testing +INSERT INTO orders_with_multi_pks +VALUES (1, 1001, 1, 102, '2022-01-16'), + (2, 1002, 2, 105, '2022-01-16'), + (3, 1004, 3, 109, '2022-01-16'), + (4, 1002, 2, 106, '2022-01-16'), + (5, 1003, 1, 107, '2022-01-16'); + + +-- create table with full types +CREATE TABLE oceanbase_full_types ( + id INT AUTO_INCREMENT, + tiny_c TINYINT, + tiny_un_c TINYINT UNSIGNED, + small_c SMALLINT, + small_un_c SMALLINT UNSIGNED, + medium_c MEDIUMINT, + medium_un_c MEDIUMINT UNSIGNED, + int_c INTEGER , + int_un_c INTEGER UNSIGNED, + int11_c INT(11) DEFAULT 0, + big_c BIGINT, + big_un_c BIGINT UNSIGNED, + varchar_c VARCHAR(255) DEFAULT '1', + char_c CHAR(3) DEFAULT '', + real_c REAL, + float_c FLOAT, + double_c DOUBLE, + decimal_c DECIMAL(8, 4), + numeric_c NUMERIC(6, 0), + big_decimal_c DECIMAL(65, 1), + bit1_c BIT, + tiny1_c TINYINT(1), + boolean_c BOOLEAN, + date_c DATE, + time_c TIME(0), + datetime3_c DATETIME(3), + datetime6_c DATETIME(6), + timestamp_c TIMESTAMP, + file_uuid BINARY(16), + bit_c BIT(64), + text_c TEXT, + tiny_blob_c TINYBLOB, + blob_c BLOB, + medium_blob_c MEDIUMBLOB, + long_blob_c LONGBLOB, + year_c YEAR, + enum_c enum('red', 'white') default 'red', + set_c SET('a', 'b'), + json_c JSON, + point_c POINT, + geometry_c GEOMETRY, + linestring_c LINESTRING, + polygon_c POLYGON, + multipoint_c MULTIPOINT, + multiline_c MULTILINESTRING, + multipolygon_c MULTIPOLYGON, + geometrycollection_c GEOMETRYCOLLECTION, + PRIMARY KEY (id), + INDEX `g_mit_seller`(`int_c`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8 partition by Hash(id) partitions 3; + +INSERT INTO oceanbase_full_types VALUES ( + DEFAULT, 127, 255, 32767, 65535, 8388607, 16777215, 2147483647, 4294967295, 2147483647, 9223372036854775807, + 18446744073709551615, + 'Hello World', 'abc', 123.102, 123.102, 404.4443, 123.4567, 345.6, 34567892.1, 0, 1, true, + '2020-07-17', '18:00:22', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456', '2020-07-17 18:00:22', + unhex(replace('651aed08-390f-4893-b2f1-36923e7b7400','-','')), b'0000010000000100000001000000010000000100000001000000010000000100', + 'text',UNHEX(HEX(16)),UNHEX(HEX(16)),UNHEX(HEX(16)),UNHEX(HEX(16)), 2021, + 'red', 'a,b,a', '{"key1": "value1", "key2": "value2", "num1": 1.6708304E7, "num2": 16708305}', + ST_GeomFromText('POINT(1 1)'), + ST_GeomFromText('POLYGON((1 1, 2 1, 2 2, 1 2, 1 1))'), + ST_GeomFromText('LINESTRING(3 0, 3 3, 3 5)'), + ST_GeomFromText('POLYGON((1 1, 2 1, 2 2, 1 2, 1 1))'), + ST_GeomFromText('MULTIPOINT((1 1),(2 2))'), + ST_GeomFromText('MultiLineString((1 1,2 2,3 3),(4 4,5 5))'), + ST_GeomFromText('MULTIPOLYGON(((0 0, 10 0, 10 10, 0 10, 0 0)), ((5 5, 7 5, 7 7, 5 7, 5 5)))'), + ST_GeomFromText('GEOMETRYCOLLECTION(POINT(10 10), POINT(30 30), LINESTRING(15 15, 20 20))') +); + +-- Create orders_sink for testing the sink of flink-jdbc-connector +create table orders_sink ( + id bigint not null auto_increment, + seller_id varchar(30) DEFAULT NULL, + order_id varchar(30) DEFAULT NULL, + buyer_id varchar(30) DEFAULT NULL, + create_time datetime DEFAULT NULL, + primary key(id) +) ENGINE=InnoDB DEFAULT CHARSET=utf8 partition by hash(id) partitions 3; + + +-- Create and populate our users using a single insert with many rows +CREATE TABLE customers ( + id INTEGER NOT NULL PRIMARY KEY, + name VARCHAR(255) NOT NULL DEFAULT 'flink', + address VARCHAR(1024), + phone_number VARCHAR(512) +); + +INSERT INTO customers +VALUES (101,"user_1","Shanghai","123567891234"), + (102,"user_2","Shanghai","123567891234"), + (103,"user_3","Shanghai","123567891234"), + (109,"user_4","Shanghai","123567891234"), + (110,"user_5","Shanghai","123567891234"), + (111,"user_6","Shanghai","123567891234"), + (118,"user_7","Shanghai","123567891234"), + (121,"user_8","Shanghai","123567891234"), + (123,"user_9","Shanghai","123567891234"), + (1009,"user_10","Shanghai","123567891234"), + (1010,"user_11","Shanghai","123567891234"), + (1011,"user_12","Shanghai","123567891234"), + (1012,"user_13","Shanghai","123567891234"), + (1013,"user_14","Shanghai","123567891234"), + (1014,"user_15","Shanghai","123567891234"), + (1015,"user_16","Shanghai","123567891234"), + (1016,"user_17","Shanghai","123567891234"), + (1017,"user_18","Shanghai","123567891234"), + (1018,"user_19","Shanghai","123567891234"), + (1019,"user_20","Shanghai","123567891234"), + (2000,"user_21","Shanghai","123567891234"); + +-- table has same name prefix with 'customers.*' +CREATE TABLE customers_1 ( + id INTEGER NOT NULL PRIMARY KEY, + name VARCHAR(255) NOT NULL DEFAULT 'flink', + address VARCHAR(1024), + phone_number VARCHAR(512) +); + +INSERT INTO customers_1 +VALUES (101,"user_1","Shanghai","123567891234"), + (102,"user_2","Shanghai","123567891234"), + (103,"user_3","Shanghai","123567891234"), + (109,"user_4","Shanghai","123567891234"), + (110,"user_5","Shanghai","123567891234"), + (111,"user_6","Shanghai","123567891234"), + (118,"user_7","Shanghai","123567891234"), + (121,"user_8","Shanghai","123567891234"), + (123,"user_9","Shanghai","123567891234"), + (1009,"user_10","Shanghai","123567891234"), + (1010,"user_11","Shanghai","123567891234"), + (1011,"user_12","Shanghai","123567891234"), + (1012,"user_13","Shanghai","123567891234"), + (1013,"user_14","Shanghai","123567891234"), + (1014,"user_15","Shanghai","123567891234"), + (1015,"user_16","Shanghai","123567891234"), + (1016,"user_17","Shanghai","123567891234"), + (1017,"user_18","Shanghai","123567891234"), + (1018,"user_19","Shanghai","123567891234"), + (1019,"user_20","Shanghai","123567891234"), + (2000,"user_21","Shanghai","123567891234"); + + +CREATE TABLE customers_no_pk ( + id INTEGER, + name VARCHAR(255) DEFAULT 'flink', + address VARCHAR(1024), + phone_number VARCHAR(512) +); + +INSERT INTO customers_no_pk +VALUES (101,"user_1","Shanghai","123567891234"), + (102,"user_2","Shanghai","123567891234"), + (103,"user_3","Shanghai","123567891234"), + (109,"user_4","Shanghai","123567891234"), + (110,"user_5","Shanghai","123567891234"), + (111,"user_6","Shanghai","123567891234"), + (118,"user_7","Shanghai","123567891234"), + (121,"user_8","Shanghai","123567891234"), + (123,"user_9","Shanghai","123567891234"), + (1009,"user_10","Shanghai","123567891234"), + (1010,"user_11","Shanghai","123567891234"), + (1011,"user_12","Shanghai","123567891234"), + (1012,"user_13","Shanghai","123567891234"), + (1013,"user_14","Shanghai","123567891234"), + (1014,"user_15","Shanghai","123567891234"), + (1015,"user_16","Shanghai","123567891234"), + (1016,"user_17","Shanghai","123567891234"), + (1017,"user_18","Shanghai","123567891234"), + (1018,"user_19","Shanghai","123567891234"), + (1019,"user_20","Shanghai","123567891234"), + (2000,"user_21","Shanghai","123567891234"); \ No newline at end of file diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/oracle/column_type_test.sql b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/oracle/column_type_test.sql deleted file mode 100644 index bfe622cfb10..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/oracle/column_type_test.sql +++ /dev/null @@ -1,70 +0,0 @@ --- 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. - -CREATE TABLE FULL_TYPES ( - ID NUMBER(9) NOT NULL, - VAL_VARCHAR VARCHAR2(1000), - VAL_VARCHAR2 VARCHAR2(1000), - VAL_NVARCHAR2 NVARCHAR2(1000), - VAL_CHAR CHAR(3), - VAL_NCHAR NCHAR(3), - VAL_BF BINARY_FLOAT, - VAL_BD BINARY_DOUBLE, - VAL_F FLOAT, - VAL_F_10 FLOAT(10), - VAL_NUM NUMBER(10, 6), - VAL_DP FLOAT, - VAL_R FLOAT(63), - VAL_DECIMAL NUMBER(10, 6), - VAL_NUMERIC NUMBER(10, 6), - VAL_NUM_VS NUMBER, - VAL_INT NUMBER, - VAL_INTEGER NUMBER, - VAL_SMALLINT NUMBER, - VAL_NUMBER_38_NO_SCALE NUMBER(38), - VAL_NUMBER_38_SCALE_0 NUMBER(38), - VAL_NUMBER_1 NUMBER(1), - VAL_NUMBER_2 NUMBER(2), - VAL_NUMBER_4 NUMBER(4), - VAL_NUMBER_9 NUMBER(9), - VAL_NUMBER_18 NUMBER(18), - VAL_NUMBER_2_NEGATIVE_SCALE NUMBER(1, -1), - VAL_NUMBER_4_NEGATIVE_SCALE NUMBER(2, -2), - VAL_NUMBER_9_NEGATIVE_SCALE NUMBER(8, -1), - VAL_NUMBER_18_NEGATIVE_SCALE NUMBER(16, -2), - VAL_NUMBER_36_NEGATIVE_SCALE NUMBER(36, -2), - VAL_DATE DATE, - VAL_TS TIMESTAMP(6), - VAL_TS_PRECISION2 TIMESTAMP(2), - VAL_TS_PRECISION4 TIMESTAMP(4), - VAL_TS_PRECISION9 TIMESTAMP(6), - VAL_CLOB_INLINE CLOB, - VAL_BLOB_INLINE BLOB, - primary key (ID) -); - -INSERT INTO FULL_TYPES VALUES ( - 1, 'vc2', 'vc2', 'nvc2', 'c', 'nc', - 1.1, 2.22, 3.33, 8.888, 4.4444, 5.555, 6.66, 1234.567891, 1234.567891, 77.323, - 1, 22, 333, 4444, 5555, 1, 99, 9999, 999999999, 999999999999999999, - 94, 9949, 999999994, 999999999999999949, 99999999999999999999999999999999999949, - TO_DATE('2022-10-30', 'yyyy-mm-dd'), - TO_TIMESTAMP('2022-10-30 12:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'), - TO_TIMESTAMP('2022-10-30 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'), - TO_TIMESTAMP('2022-10-30 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'), - TO_TIMESTAMP('2022-10-30 12:34:56.125456789', 'yyyy-mm-dd HH24:MI:SS.FF9'), - TO_CLOB ('col_clob'), - utl_raw.cast_to_raw ('col_blob') -); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-sql-connector-oceanbase-cdc/pom.xml b/flink-cdc-connect/flink-cdc-source-connectors/flink-sql-connector-oceanbase-cdc/pom.xml index 6bc425a71b1..3774acf5f92 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-sql-connector-oceanbase-cdc/pom.xml +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-sql-connector-oceanbase-cdc/pom.xml @@ -33,6 +33,12 @@ limitations under the License. flink-connector-oceanbase-cdc ${project.version} + + + org.apache.flink + flink-connector-mysql-cdc + ${project.version} + @@ -55,19 +61,21 @@ limitations under the License. io.debezium:debezium-api io.debezium:debezium-embedded io.debezium:debezium-core - org.apache.flink:flink-cdc-base + io.debezium:debezium-ddl-parser + io.debezium:debezium-connector-mysql org.apache.flink:flink-cdc-common org.apache.flink:flink-connector-debezium - org.apache.flink:flink-connector-oceanbase-cdc - com.oceanbase:* - io.netty:* - com.google.protobuf:protobuf-java - commons-codec:commons-codec - org.lz4:lz4-java - org.apache.avro:avro - org.apache.commons:commons-lang3 + org.apache.flink:flink-connector-mysql-cdc + org.antlr:antlr4-runtime org.apache.kafka:* + com.zendesk:mysql-binlog-connector-java com.fasterxml.*:* + com.google.guava:* + com.esri.geometry:esri-geometry-api + com.zaxxer:HikariCP + + org.apache.flink:flink-shaded-guava + org.apache.flink:flink-connector-oceanbase-cdc @@ -85,12 +93,6 @@ limitations under the License. - - org.apache.commons - - org.apache.flink.cdc.connectors.shaded.org.apache.commons - - org.apache.kafka @@ -98,9 +100,9 @@ limitations under the License. - org.apache.avro + org.antlr - org.apache.flink.cdc.connectors.shaded.org.apache.avro + org.apache.flink.cdc.connectors.shaded.org.antlr @@ -115,6 +117,16 @@ limitations under the License. org.apache.flink.cdc.connectors.shaded.com.google + + com.esri.geometry + org.apache.flink.cdc.connectors.shaded.com.esri.geometry + + + com.zaxxer + + org.apache.flink.cdc.connectors.shaded.com.zaxxer + + diff --git a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/pom.xml b/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/pom.xml index a1306409ce3..f97841016d1 100644 --- a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/pom.xml +++ b/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/pom.xml @@ -86,13 +86,6 @@ limitations under the License. test-jar test - - org.apache.flink - flink-connector-oceanbase-cdc - ${project.version} - test-jar - test - org.apache.flink flink-connector-oracle-cdc @@ -347,16 +340,6 @@ limitations under the License. - - org.apache.flink - flink-sql-connector-oceanbase-cdc - ${project.version} - oceanbase-cdc-connector.jar - jar - ${project.build.directory}/dependencies - - - com.ibm.db2.jcc db2jcc diff --git a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/org/apache/flink/cdc/connectors/tests/OceanBaseE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/org/apache/flink/cdc/connectors/tests/OceanBaseE2eITCase.java deleted file mode 100644 index 49e3fcdd1ac..00000000000 --- a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/org/apache/flink/cdc/connectors/tests/OceanBaseE2eITCase.java +++ /dev/null @@ -1,187 +0,0 @@ -/* - * 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.flink.cdc.connectors.tests; - -import org.apache.flink.cdc.common.test.utils.JdbcProxy; -import org.apache.flink.cdc.common.test.utils.TestUtils; -import org.apache.flink.cdc.connectors.oceanbase.testutils.LogProxyContainer; -import org.apache.flink.cdc.connectors.oceanbase.testutils.OceanBaseCdcMetadata; -import org.apache.flink.cdc.connectors.oceanbase.testutils.OceanBaseContainer; -import org.apache.flink.cdc.connectors.oceanbase.testutils.OceanBaseMySQLCdcMetadata; -import org.apache.flink.cdc.connectors.oceanbase.testutils.UniqueDatabase; -import org.apache.flink.cdc.connectors.tests.utils.FlinkContainerTestEnvironment; - -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.testcontainers.junit.jupiter.Container; -import org.testcontainers.junit.jupiter.Testcontainers; - -import java.nio.file.Path; -import java.sql.Connection; -import java.sql.SQLException; -import java.sql.Statement; -import java.time.Duration; -import java.util.Arrays; -import java.util.List; - -import static org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestUtils.createLogProxyContainer; -import static org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestUtils.createOceanBaseContainerForCDC; - -/** End-to-end tests for oceanbase-cdc connector uber jar. */ -@Testcontainers -class OceanBaseE2eITCase extends FlinkContainerTestEnvironment { - - private static final String INTER_CONTAINER_OB_SERVER_ALIAS = "oceanbase"; - private static final String INTER_CONTAINER_LOG_PROXY_ALIAS = "oblogproxy"; - - private static final Path obCdcJar = TestUtils.getResource("oceanbase-cdc-connector.jar"); - private static final Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - - @Container - public static final OceanBaseContainer OB_SERVER = - createOceanBaseContainerForCDC() - .withNetwork(NETWORK) - .withNetworkAliases(INTER_CONTAINER_OB_SERVER_ALIAS); - - @Container - public static final LogProxyContainer LOG_PROXY = - createLogProxyContainer() - .withNetwork(NETWORK) - .withNetworkAliases(INTER_CONTAINER_LOG_PROXY_ALIAS); - - private static final OceanBaseCdcMetadata METADATA = - new OceanBaseMySQLCdcMetadata(OB_SERVER, LOG_PROXY); - - protected final UniqueDatabase obInventoryDatabase = - new UniqueDatabase(OB_SERVER, "oceanbase_inventory"); - - @BeforeEach - public void before() { - super.before(); - - obInventoryDatabase.createAndInitialize(); - } - - @AfterEach - public void after() { - super.after(); - - obInventoryDatabase.dropDatabase(); - } - - @Test - void testOceanBaseCDC() throws Exception { - List sqlLines = - Arrays.asList( - "SET 'execution.checkpointing.interval' = '3s';", - "SET 'execution.checkpointing.checkpoints-after-tasks-finish.enabled' = 'true';", - "CREATE TABLE products_source (", - " `id` INT NOT NULL,", - " name STRING,", - " description STRING,", - " weight DECIMAL(10,3),", - " enum_c STRING,", - " json_c STRING,", - " primary key (`id`) not enforced", - ") WITH (", - " 'connector' = 'oceanbase-cdc',", - " 'scan.startup.mode' = 'initial',", - " 'username' = '" + METADATA.getUsername() + "',", - " 'password' = '" + METADATA.getPassword() + "',", - " 'tenant-name' = '" + METADATA.getTenantName() + "',", - " 'table-list' = '" - + obInventoryDatabase.qualifiedTableName("products_source") - + "',", - " 'hostname' = '" + INTER_CONTAINER_OB_SERVER_ALIAS + "',", - " 'port' = '2881',", - " 'jdbc.driver' = '" + METADATA.getDriverClass() + "',", - " 'logproxy.host' = '" + INTER_CONTAINER_LOG_PROXY_ALIAS + "',", - " 'logproxy.port' = '2983',", - " 'rootserver-list' = '" + METADATA.getRsList() + "',", - " 'working-mode' = 'memory',", - " 'jdbc.properties.useSSL' = 'false'", - ");", - "CREATE TABLE ob_products_sink (", - " `id` INT NOT NULL,", - " name STRING,", - " description STRING,", - " weight DECIMAL(10,3),", - " enum_c STRING,", - " json_c STRING,", - " primary key (`id`) not enforced", - ") WITH (", - " 'connector' = 'jdbc',", - String.format( - " 'url' = 'jdbc:mysql://%s:3306/%s',", - INTER_CONTAINER_MYSQL_ALIAS, - mysqlInventoryDatabase.getDatabaseName()), - " 'table-name' = 'ob_products_sink',", - " 'username' = '" + MYSQL_TEST_USER + "',", - " 'password' = '" + MYSQL_TEST_PASSWORD + "'", - ");", - "INSERT INTO ob_products_sink", - "SELECT * FROM products_source;"); - - submitSQLJob(sqlLines, obCdcJar, jdbcJar, mysqlDriverJar); - waitUntilJobRunning(Duration.ofSeconds(30)); - - try (Connection conn = obInventoryDatabase.getJdbcConnection(); - Statement stat = conn.createStatement()) { - stat.execute( - "UPDATE products_source SET description='18oz carpenter hammer' WHERE id=106;"); - stat.execute("UPDATE products_source SET weight='5.1' WHERE id=107;"); - stat.execute( - "INSERT INTO products_source VALUES (default,'jacket','water resistent white wind breaker',0.2, null, null);"); - stat.execute( - "INSERT INTO products_source VALUES (default,'scooter','Big 2-wheel scooter ',5.18, null, null);"); - stat.execute( - "UPDATE products_source SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110;"); - stat.execute("UPDATE products_source SET weight='5.17' WHERE id=111;"); - stat.execute("DELETE FROM products_source WHERE id=111;"); - } catch (SQLException e) { - throw new RuntimeException("Update table for CDC failed.", e); - } - - String mysqlUrl = - String.format( - "jdbc:mysql://%s:%s/%s", - MYSQL.getHost(), - MYSQL.getDatabasePort(), - mysqlInventoryDatabase.getDatabaseName()); - JdbcProxy proxy = - new JdbcProxy(mysqlUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD, MYSQL_DRIVER_CLASS); - List expectResult = - Arrays.asList( - "101,scooter,Small 2-wheel scooter,3.14,red,{\"key1\": \"value1\"}", - "102,car battery,12V car battery,8.1,white,{\"key2\": \"value2\"}", - "103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8,red,{\"key3\": \"value3\"}", - "104,hammer,12oz carpenter's hammer,0.75,white,{\"key4\": \"value4\"}", - "105,hammer,14oz carpenter's hammer,0.875,red,{\"k1\": \"v1\", \"k2\": \"v2\"}", - "106,hammer,18oz carpenter hammer,1.0,null,null", - "107,rocks,box of assorted rocks,5.1,null,null", - "108,jacket,water resistent black wind breaker,0.1,null,null", - "109,spare tire,24 inch spare tire,22.2,null,null", - "110,jacket,new water resistent white wind breaker,0.5,null,null"); - proxy.checkResultWithTimeout( - expectResult, - "ob_products_sink", - new String[] {"id", "name", "description", "weight", "enum_c", "json_c"}, - 300000L); - } -} diff --git a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/resources/ddl/oceanbase_inventory.sql b/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/resources/ddl/oceanbase_inventory.sql deleted file mode 100644 index 9c4ec596974..00000000000 --- a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/resources/ddl/oceanbase_inventory.sql +++ /dev/null @@ -1,40 +0,0 @@ --- 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. - --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: inventory --- ---------------------------------------------------------------------------------------------------------------- - --- Create and populate our products using a single insert with many rows -CREATE TABLE products_source ( - id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, - name VARCHAR(255) NOT NULL DEFAULT 'flink', - description VARCHAR(512), - weight DECIMAL(10, 3), - enum_c enum('red', 'white') default 'red', - json_c JSON -); -ALTER TABLE products_source AUTO_INCREMENT = 101; - -INSERT INTO products_source -VALUES (default,"scooter","Small 2-wheel scooter",3.14, 'red', '{"key1": "value1"}'), - (default,"car battery","12V car battery",8.1, 'white', '{"key2": "value2"}'), - (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8, 'red', '{"key3": "value3"}'), - (default,"hammer","12oz carpenter's hammer",0.75, 'white', '{"key4": "value4"}'), - (default,"hammer","14oz carpenter's hammer",0.875, 'red', '{"k1": "v1", "k2": "v2"}'), - (default,"hammer","16oz carpenter's hammer",1.0, null, null), - (default,"rocks","box of assorted rocks",5.3, null, null), - (default,"jacket","water resistent black wind breaker",0.1, null, null), - (default,"spare tire","24 inch spare tire",22.2, null, null);