[FLINK-34639][cdc][oceanbase] Support debezium deserializer in OceanBase source connector (#3124)

pull/3261/head
He Wang 9 months ago committed by GitHub
parent bdca0e328b
commit 05281e5d6f
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194

@ -28,32 +28,84 @@ under the License.
OceanBase CDC 连接器允许从 OceanBase 读取快照数据和增量数据。本文介绍了如何设置 OceanBase CDC 连接器以对 OceanBase 进行 SQL 查询。
## 依赖
为了使用 OceanBase CDC 连接器,您必须提供相关的依赖信息。以下依赖信息适用于使用自动构建工具(如 Maven 或 SBT构建的项目和带有 SQL JAR 包的 SQL 客户端。
为了使用 OceanBase CDC 连接器,您必须提供相关的依赖信息。以下依赖信息适用于使用自动构建工具(如 Maven 或 SBT构建的项目和带有 SQL JAR 包的 SQL 客户端。
### OceanBase CDC 方案
```xml
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-oceanbase-cdc</artifactId>
<!-- 请使用已发布的版本依赖snapshot 版本的依赖需要本地自行编译。 -->
<version>3.1-SNAPSHOT</version>
</dependency>
名词解释:
```
- *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 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 。
如果您是要连接企业版的 OceanBase您可能需要使用 OceanBase 官方的 JDBC 驱动,这时需要引入如下依赖。
OceanBase CDC 源端读取方案:
```xml
<dependency>
<groupId>com.oceanbase</groupId>
<artifactId>oceanbase-client</artifactId>
<version>2.4.2</version>
</dependency>
```
<div class="wy-table-responsive">
<table class="colwidths-auto docutils">
<thead>
<tr>
<th class="text-left">数据库类型</th>
<th class="text-left">支持的驱动</th>
<th class="text-left">CDC 连接器</th>
<th class="text-left">其他用到的组件</th>
</tr>
</thead>
<tbody>
<tr>
<td rowspan="2">OceanBase CE</td>
<td>
MySQL Driver: 5.1.4x, 8.0.x <br>
OceanBase Driver: 2.4.x
</td>
<td>OceanBase CDC Connector</td>
<td>Log Proxy CE</td>
</tr>
<tr>
<td>MySQL Driver: 8.0.x</td>
<td>MySQL CDC Connector</td>
<td>Binlog Service CE</td>
</tr>
<tr>
<td rowspan="2">OceanBase EE (MySQL 模式)</td>
<td>
MySQL Driver: 5.1.4x, 8.0.x <br>
OceanBase Driver: 2.4.x
</td>
<td>OceanBase CDC Connector</td>
<td>Log Proxy EE</td>
</tr>
<tr>
<td>MySQL Driver: 8.0.x</td>
<td>MySQL CDC Connector</td>
<td>Binlog Service EE</td>
</tr>
<tr>
<td>OceanBase EE (Oracle 模式)</td>
<td>OceanBase Driver: 2.4.x</td>
<td>OceanBase CDC Connector</td>
<td>Log Proxy EE (CDC 模式)</td>
</tr>
</tbody>
</table>
</div>
注意: 对于使用 OceanBase 社区版或 OceanBase 企业版 MySQL 模式的用户,我们推荐参考 [MySQL CDC 的文档](mysql-cdc.md),使用 MySQL CDC 连接器搭配 Binlog 服务。
依赖
------------
为了使用 OceanBase CDC 连接器,您必须提供相关的依赖信息。以下依赖信息适用于使用自动构建工具(如 Maven 或 SBT构建的项目和带有 SQL JAR 包的 SQL 客户端。
### Maven dependency
## 下载 SQL 客户端 JAR 包
{{< artifact flink-connector-oceanbase-cdc >}}
### SQL Client JAR
```下载链接仅在已发布版本可用,请在文档网站左下角选择浏览已发布的版本。```
@ -61,19 +113,44 @@ OceanBase CDC 连接器允许从 OceanBase 读取快照数据和增量数据。
**注意:** 参考 [flink-sql-connector-oceanbase-cdc](https://mvnrepository.com/artifact/com.ververica/flink-sql-connector-oceanbase-cdc) 当前已发布的所有版本都可以在 Maven 中央仓库获取。
由于开源许可证的原因,我们不能在上述 cdc jar 文件中包含 OceanBase 的官方 JDBC 驱动,如果您需要使用它,可以从[这里](https://repo1.maven.org/maven2/com/oceanbase/oceanbase-client/2.4.2/oceanbase-client-2.4.2.jar)下载,然后放到 `<FLINK_HOME>/lib/` 目录下,同时需要将配置项 `jdbc.driver` 设为 `com.oceanbase.jdbc.Driver`
由于 MySQL Driver 和 OceanBase Driver 使用的开源协议都与 Flink CDC 项目不兼容,我们无法在 jar 包中提供驱动。 您可能需要手动配置以下依赖:
<div class="wy-table-responsive">
<table class="colwidths-auto docutils">
<thead>
<tr>
<th class="text-left">依赖名称</th>
<th class="text-left">说明</th>
</tr>
</thead>
<tbody>
<tr>
<td><a href="https://mvnrepository.com/artifact/mysql/mysql-connector-java/8.0.27">mysql:mysql-connector-java:8.0.27</a></td>
<td>用于连接到 OceanBase 数据库的 MySQL 租户。</td>
</tr>
</tbody>
<tbody>
<tr>
<td><a href="https://mvnrepository.com/artifact/com.oceanbase/oceanbase-client/2.4.9">com.oceanbase:oceanbase-client:2.4.9</a></td>
<td>用于连接到 OceanBase 数据库的 MySQL 或 Oracle 租户。</td>
</tr>
</tbody>
</table>
</div>
### 配置 OceanBase 数据库和 oblogproxy 服务
配置 OceanBase 数据库和 Log Proxy 服务
----------------------
1. 按照 [文档](https://github.com/oceanbase/oceanbase#quick-start) 配置 OceanBase 集群。
2. 在 sys 租户中,为 oblogproxy 创建一个带密码的用户。
```bash
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;
```
```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. 为你想要监控的租户创建一个用户,这个用户用来读取快照数据和变化事件数据。
4. OceanBase 社区版用户需要获取`rootserver-list`,可以使用以下命令获取:
@ -87,9 +164,10 @@ OceanBase CDC 连接器允许从 OceanBase 读取快照数据和增量数据。
mysql> show parameters like 'obconfig_url';
```
5. 按照 [文档](https://github.com/oceanbase/oblogproxy#getting-started) 配置 oblogproxy
5. 设置 OceanBase LogProxy。 对于OceanBase社区版的用户您可以按照[此文档](https://www.oceanbase.com/docs/community-oblogproxy-doc-1000000000531984)进行操作
## 创建 OceanBase CDC 表
创建 OceanBase CDC 表
----------------
使用以下命令,创建 OceanBase CDC 表:
@ -158,12 +236,13 @@ Flink SQL> CREATE TABLE orders (
您也可以访问 Flink CDC 官网文档,快速体验将数据从 OceanBase 导入到 Elasticsearch。更多信息参考 [Flink CDC 官网文档](https://nightlies.apache.org/flink/flink-cdc-docs-release-3.0/docs/connectors/legacy-flink-cdc-sources/tutorials/oceanbase-tutorial/)。
## OceanBase CDC 连接器选项
OceanBase CDC 连接器选项
----------------
OceanBase CDC 连接器包括用于 SQL 和 DataStream API 的选项,如下表所示。
*注意*:连接器支持两种方式来指定需要监听的表,两种方式同时使用时会监听两种方式匹配的所有表。
1. 使用 `database-name``table-name` 匹配正则表达式中的数据库和表名。 由于`obcdc`(以前的`liboblog`)现在只支持`fnmatch`匹配,我们不能直接使用正则过滤 changelog 事件,所以通过两个选项去匹配去指定监听表只能在`initial`启动模式下使用。
1. 使用 `database-name``table-name` 匹配正则表达式中的数据库和表名。
2. 使用 `table-list` 去匹配数据库名和表名的准确列表。
<div class="highlight">
@ -187,11 +266,12 @@ OceanBase CDC 连接器包括用于 SQL 和 DataStream API 的选项,如下表
</tr>
<tr>
<td>scan.startup.mode</td>
<td></td>
<td style="word-wrap: break-word;"></td>
<td></td>
<td style="word-wrap: break-word;">initial</td>
<td>String</td>
<td>指定 OceanBase CDC 消费者的启动模式。可取值为<code>'initial'</code>,<code>'latest-offset'</code> or
<code>'timestamp'</code></td>
<td>指定 OceanBase CDC 消费者的启动模式。可取值为
<code>'initial'</code><code>'latest-offset'</code><code>'timestamp'</code><code>'snapshot'</code>
</td>
</tr>
<tr>
<td>scan.startup.timestamp</td>
@ -216,7 +296,7 @@ OceanBase CDC 连接器包括用于 SQL 和 DataStream API 的选项,如下表
</tr>
<tr>
<td>tenant-name</td>
<td></td>
<td></td>
<td style="word-wrap: break-word;"></td>
<td>String</td>
<td>待监控 OceanBase 数据库的租户名,应该填入精确值。</td>
@ -226,14 +306,14 @@ OceanBase CDC 连接器包括用于 SQL 和 DataStream API 的选项,如下表
<td></td>
<td style="word-wrap: break-word;"></td>
<td>String</td>
<td>待监控 OceanBase 数据库的数据库名,应该是正则表达式,该选项只支持和 'initial' 模式一起使用</td>
<td>待监控 OceanBase 数据库的数据库名,应该是正则表达式。</td>
</tr>
<tr>
<td>table-name</td>
<td></td>
<td style="word-wrap: break-word;"></td>
<td>String</td>
<td>待监控 OceanBase 数据库的表名,应该是正则表达式,该选项只支持和 'initial' 模式一起使用</td>
<td>待监控 OceanBase 数据库的表名,应该是正则表达式。</td>
</tr>
<tr>
<td>table-list</td>
@ -244,14 +324,14 @@ OceanBase CDC 连接器包括用于 SQL 和 DataStream API 的选项,如下表
</tr>
<tr>
<td>hostname</td>
<td></td>
<td></td>
<td style="word-wrap: break-word;"></td>
<td>String</td>
<td>OceanBase 数据库或 OceanBbase 代理 ODP 的 IP 地址或主机名。</td>
</tr>
<tr>
<td>port</td>
<td></td>
<td></td>
<td style="word-wrap: break-word;"></td>
<td>Integer</td>
<td>
@ -278,14 +358,14 @@ OceanBase CDC 连接器包括用于 SQL 和 DataStream API 的选项,如下表
</tr>
<tr>
<td>logproxy.host</td>
<td></td>
<td></td>
<td style="word-wrap: break-word;"></td>
<td>String</td>
<td>OceanBase 日志代理服务 的 IP 地址或主机名。</td>
</tr>
<tr>
<td>logproxy.port</td>
<td></td>
<td></td>
<td style="word-wrap: break-word;"></td>
<td>Integer</td>
<td>OceanBase 日志代理服务 的端口号。</td>
@ -328,7 +408,7 @@ OceanBase CDC 连接器包括用于 SQL 和 DataStream API 的选项,如下表
<tr>
<td>jdbc.driver</td>
<td></td>
<td style="word-wrap: break-word;">com.mysql.jdbc.Driver</td>
<td style="word-wrap: break-word;">com.mysql.cj.jdbc.Driver</td>
<td>String</td>
<td>全量读取时使用的 jdbc 驱动类名。</td>
</tr>
@ -339,11 +419,19 @@ OceanBase CDC 连接器包括用于 SQL 和 DataStream API 的选项,如下表
<td>String</td>
<td>传递自定义 JDBC URL 属性的选项。用户可以传递自定义属性,如 'jdbc.properties.useSSL' = 'false'。</td>
</tr>
<tr>
<td>obcdc.properties.*</td>
<td></td>
<td style="word-wrap: break-word;"></td>
<td>String</td>
<td>传递自定义 <code>libobcdc</code> 属性的选项,如 'obcdc.properties.sort_trans_participants' = '1'。详情参见 <a href="https://www.oceanbase.com/docs/common-oceanbase-database-cn-1000000000510698">obcdc 配置项说明</a></td>
</tr>
</tbody>
</table>
</div>
## 支持的元数据
支持的元数据
----------------
在创建表时您可以使用以下格式的元数据作为只读列VIRTUAL
@ -358,13 +446,18 @@ OceanBase CDC 连接器包括用于 SQL 和 DataStream API 的选项,如下表
<tbody>
<tr>
<td>tenant_name</td>
<td>STRING NOT NULL</td>
<td>STRING</td>
<td>当前记录所属的租户名称。</td>
</tr>
<tr>
<td>database_name</td>
<td>STRING NOT NULL</td>
<td>当前记录所属的库名。</td>
<td>STRING</td>
<td>当前记录所属的 db 名。</td>
</tr>
<tr>
<td>schema_name</td>
<td>STRING</td>
<td>当前记录所属的 schema 名。</td>
</tr>
<tr>
<td>table_name</td>
@ -406,10 +499,13 @@ CREATE TABLE products (
'port' = '2881',
'rootserver-list' = '127.0.0.1:2882:2881',
'logproxy.host' = '127.0.0.1',
'logproxy.port' = '2983');
'logproxy.port' = '2983',
'working-mode' = 'memory'
);
```
## 特性
特性
--------
### At-Least-Once 处理
@ -424,6 +520,7 @@ OceanBase 数据库是一个分布式数据库,它的日志也分散在不同
- `initial`(默认):在首次启动时对受监视的数据库表执行初始快照,并继续读取最新的提交日志。
- `latest-offset`:首次启动时,不对受监视的数据库表执行快照,仅从连接器启动时读取提交日志。
- `timestamp`:在首次启动时不对受监视的数据库表执行初始快照,仅从指定的 `scan.startup.timestamp` 读取最新的提交日志。
- `snapshot`: 仅对受监视的数据库表执行初始快照。
### 消费提交日志
@ -434,65 +531,31 @@ OceanBase CDC 连接器使用 [oblogclient](https://github.com/oceanbase/oblogcl
OceanBase CDC 连接器也可以作为 DataStream Source 使用。您可以按照如下创建一个 SourceFunction
```java
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.debezium.JsonDebeziumDeserializationSchema;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.catalog.Column;
import org.apache.flink.table.catalog.ResolvedSchema;
import org.apache.flink.table.catalog.UniqueConstraint;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.cdc.connectors.oceanbase.OceanBaseSource;
import org.apache.flink.cdc.connectors.oceanbase.source.RowDataOceanBaseDeserializationSchema;
import org.apache.flink.cdc.connectors.oceanbase.table.OceanBaseDeserializationSchema;
import org.apache.flink.cdc.connectors.oceanbase.table.StartupMode;
import java.time.ZoneId;
import java.util.Arrays;
import java.util.Collections;
public class OceanBaseSourceExample {
public static void main(String[] args) throws Exception {
ResolvedSchema resolvedSchema =
new ResolvedSchema(
Arrays.asList(
Column.physical("id", DataTypes.INT().notNull()),
Column.physical("name", DataTypes.STRING().notNull())),
Collections.emptyList(),
UniqueConstraint.primaryKey("pk", Collections.singletonList("id")));
RowType physicalDataType =
(RowType) resolvedSchema.toPhysicalRowDataType().getLogicalType();
TypeInformation<RowData> resultTypeInfo = InternalTypeInfo.of(physicalDataType);
String serverTimeZone = "+00:00";
OceanBaseDeserializationSchema<RowData> deserializer =
RowDataOceanBaseDeserializationSchema.newBuilder()
.setPhysicalRowType(physicalDataType)
.setResultTypeInfo(resultTypeInfo)
.setServerTimeZone(ZoneId.of(serverTimeZone))
.build();
SourceFunction<RowData> oceanBaseSource =
OceanBaseSource.<RowData>builder()
.rsList("127.0.0.1:2882:2881")
.startupMode(StartupMode.INITIAL)
SourceFunction<String> oceanBaseSource =
OceanBaseSource.<String>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$")
.hostname("127.0.0.1")
.port(2881)
.compatibleMode("mysql")
.jdbcDriver("com.mysql.jdbc.Driver")
.logProxyHost("127.0.0.1")
.logProxyPort(2983)
.serverTimeZone(serverTimeZone)
.deserializer(deserializer)
.rsList("127.0.0.1:2882:2881")
.serverTimeZone("+08:00")
.deserializer(new JsonDebeziumDeserializationSchema())
.build();
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
@ -506,7 +569,8 @@ public class OceanBaseSourceExample {
}
```
## 数据类型映射
数据类型映射
----------------
### Mysql 模式
@ -591,7 +655,8 @@ public class OceanBaseSourceExample {
<td>
NUMERIC(p, s)<br>
DECIMAL(p, s)<br>
where 38 < p <=65 </td>
where 38 < p <=65<br>
</td>
<td>STRING</td>
<td>
DECIMAL 等同于 NUMERIC。在 OceanBase 数据库中DECIMAL 数据类型的精度最高为 65。<br>
@ -631,7 +696,7 @@ public class OceanBaseSourceExample {
</tr>
<tr>
<td>BIT(n)</td>
<td>BINARY(⌈n/8⌉)</td>
<td>BINARY(⌈(n + 7) / 8⌉)</td>
<td></td>
</tr>
<tr>

@ -28,22 +28,81 @@ under the License.
The OceanBase CDC connector allows for reading snapshot data and incremental data from OceanBase. This document describes how to set up the OceanBase CDC connector to run SQL queries against OceanBase.
### OceanBase CDC Solutions
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.
CDC Source Solutions for OceanBase:
<div class="wy-table-responsive">
<table class="colwidths-auto docutils">
<thead>
<tr>
<th class="text-left">Database</th>
<th class="text-left">Supported Driver</th>
<th class="text-left">CDC Source Connector</th>
<th class="text-left">Other Required Components</th>
</tr>
</thead>
<tbody>
<tr>
<td rowspan="2">OceanBase CE</td>
<td>
MySQL Driver: 5.1.4x, 8.0.x <br>
OceanBase Driver: 2.4.x
</td>
<td>OceanBase CDC Connector</td>
<td>Log Proxy CE</td>
</tr>
<tr>
<td>MySQL Driver: 8.0.x</td>
<td>MySQL CDC Connector</td>
<td>Binlog Service CE</td>
</tr>
<tr>
<td rowspan="2">OceanBase EE (MySQL Mode)</td>
<td>
MySQL Driver: 5.1.4x, 8.0.x <br>
OceanBase Driver: 2.4.x
</td>
<td>OceanBase CDC Connector</td>
<td>Log Proxy EE</td>
</tr>
<tr>
<td>MySQL Driver: 8.0.x</td>
<td>MySQL CDC Connector</td>
<td>Binlog Service EE</td>
</tr>
<tr>
<td>OceanBase EE (Oracle Mode)</td>
<td>OceanBase Driver: 2.4.x</td>
<td>OceanBase CDC Connector</td>
<td>Log Proxy EE (CDC Mode)</td>
</tr>
</tbody>
</table>
</div>
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.
Dependencies
------------
In order to set up the OceanBase CDC connector, the following table provides dependency information for both projects using a build automation tool (such as Maven or SBT) and SQL Client with SQL JAR bundles.
{{< artifact flink-connector-oceanbase-cdc >}}
If you want to use OceanBase JDBC driver to connect to the enterprise edition database, you should also include the following dependency in your class path.
### Maven dependency
```xml
<dependency>
<groupId>com.oceanbase</groupId>
<artifactId>oceanbase-client</artifactId>
<version>2.4.2</version>
</dependency>
```
{{< artifact flink-connector-oceanbase-cdc >}}
### SQL Client JAR
@ -53,7 +112,30 @@ Download [flink-sql-connector-oceanbase-cdc-3.0.1.jar](https://repo1.maven.org/m
**Note:** Refer to [flink-sql-connector-oceanbase-cdc](https://mvnrepository.com/artifact/com.ververica/flink-sql-connector-oceanbase-cdc), more released versions will be available in the Maven central warehouse.
Due to the license issue, we can not include the OceanBase JDBC driver in the cdc jar. If you need to use it, you can download it from [here](https://repo1.maven.org/maven2/com/oceanbase/oceanbase-client/2.4.2/oceanbase-client-2.4.2.jar) and put it under `<FLINK_HOME>/lib/`, you also need to set the start option `jdbc.driver` to `com.oceanbase.jdbc.Driver`.
Since the licenses of MySQL Driver and OceanBase Driver are incompatible with Flink CDC project, we can't provide them in prebuilt connector jar packages. You may need to configure the following dependencies manually.
<div class="wy-table-responsive">
<table class="colwidths-auto docutils">
<thead>
<tr>
<th class="text-left">Dependency Item</th>
<th class="text-left">Description</th>
</tr>
</thead>
<tbody>
<tr>
<td><a href="https://mvnrepository.com/artifact/mysql/mysql-connector-java/8.0.27">mysql:mysql-connector-java:8.0.27</a></td>
<td>Used for connecting to MySQL tenant of OceanBase.</td>
</tr>
</tbody>
<tbody>
<tr>
<td><a href="https://mvnrepository.com/artifact/com.oceanbase/oceanbase-client/2.4.9">com.oceanbase:oceanbase-client:2.4.9</a></td>
<td>Used for connecting to MySQL or Oracle tenant of OceanBase.</td>
</tr>
</tbody>
</table>
</div>
Setup OceanBase and LogProxy Server
----------------------
@ -84,7 +166,7 @@ Setup OceanBase and LogProxy Server
mysql> show parameters like 'obconfig_url';
```
5. Setup OceanBase LogProxy. For users of OceanBase Community Edition, you can follow the [quick start](https://github.com/oceanbase/oblogproxy#getting-started).
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).
How to create a OceanBase CDC table
----------------
@ -162,7 +244,7 @@ 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. As the `obcdc` (former `liboblog`) only supports `fnmatch` now, we can't use regex directly to filter change events, so these two options can only be used in `initial` startup mode.
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.
<div class="highlight">
@ -186,11 +268,11 @@ The OceanBase CDC Connector contains some options for both sql and stream api as
</tr>
<tr>
<td>scan.startup.mode</td>
<td>required</td>
<td style="word-wrap: break-word;">(none)</td>
<td>optional</td>
<td style="word-wrap: break-word;">initial</td>
<td>String</td>
<td>Specify the startup mode for OceanBase CDC consumer, valid enumerations are
<code>'initial'</code>,<code>'latest-offset'</code> or <code>'timestamp'</code>.
<code>'initial'</code>,<code>'latest-offset'</code>,<code>'timestamp'</code> or <code>'snapshot'</code>.
</td>
</tr>
<tr>
@ -216,24 +298,24 @@ The OceanBase CDC Connector contains some options for both sql and stream api as
</tr>
<tr>
<td>tenant-name</td>
<td>required</td>
<td>optional</td>
<td style="word-wrap: break-word;">(none)</td>
<td>String</td>
<td>Tenant name of OceanBase to monitor, should be exact value.</td>
<td>Tenant name of OceanBase to monitor, should be exact value. Required when 'scan.startup.mode' is not 'snapshot'.</td>
</tr>
<tr>
<td>database-name</td>
<td>optional</td>
<td style="word-wrap: break-word;">(none)</td>
<td>String</td>
<td>Database name of OceanBase to monitor, should be regular expression. Only can be used with 'initial' mode.</td>
<td>Database name of OceanBase to monitor, should be regular expression.</td>
</tr>
<tr>
<td>table-name</td>
<td>optional</td>
<td style="word-wrap: break-word;">(none)</td>
<td>String</td>
<td>Table name of OceanBase to monitor, should be regular expression. Only can be used with 'initial' mode.</td>
<td>Table name of OceanBase to monitor, should be regular expression.</td>
</tr>
<tr>
<td>table-list</td>
@ -244,14 +326,14 @@ The OceanBase CDC Connector contains some options for both sql and stream api as
</tr>
<tr>
<td>hostname</td>
<td>optional</td>
<td>required</td>
<td style="word-wrap: break-word;">(none)</td>
<td>String</td>
<td>IP address or hostname of the OceanBase database server or OceanBase Proxy server.</td>
</tr>
<tr>
<td>port</td>
<td>optional</td>
<td>required</td>
<td style="word-wrap: break-word;">(none)</td>
<td>Integer</td>
<td>Integer 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.</td>
@ -272,17 +354,17 @@ The OceanBase CDC Connector contains some options for both sql and stream api as
</tr>
<tr>
<td>logproxy.host</td>
<td>required</td>
<td>optional</td>
<td style="word-wrap: break-word;">(none)</td>
<td>String</td>
<td>Hostname or IP address of OceanBase log proxy service.</td>
<td>Hostname or IP address of OceanBase log proxy service. Required when 'scan.startup.mode' is not 'snapshot'.</td>
</tr>
<tr>
<td>logproxy.port</td>
<td>required</td>
<td>optional</td>
<td style="word-wrap: break-word;">(none)</td>
<td>Integer</td>
<td>Port number of OceanBase log proxy service.</td>
<td>Port number of OceanBase log proxy service. Required when 'scan.startup.mode' is not 'snapshot'.</td>
</tr>
<tr>
<td>logproxy.client.id</td>
@ -322,7 +404,7 @@ The OceanBase CDC Connector contains some options for both sql and stream api as
<tr>
<td>jdbc.driver</td>
<td>optional</td>
<td style="word-wrap: break-word;">com.mysql.jdbc.Driver</td>
<td style="word-wrap: break-word;">com.mysql.cj.jdbc.Driver</td>
<td>String</td>
<td>JDBC driver class for snapshot reading.</td>
</tr>
@ -360,14 +442,19 @@ The following format metadata can be exposed as read-only (VIRTUAL) columns in a
<tbody>
<tr>
<td>tenant_name</td>
<td>STRING NOT NULL</td>
<td>STRING</td>
<td>Name of the tenant that contains the row.</td>
</tr>
<tr>
<td>database_name</td>
<td>STRING NOT NULL</td>
<td>STRING</td>
<td>Name of the database that contains the row.</td>
</tr>
<tr>
<td>schema_name</td>
<td>STRING</td>
<td>Name of the schema that contains the row.</td>
</tr>
<tr>
<td>table_name</td>
<td>STRING NOT NULL</td>
@ -430,6 +517,7 @@ The config option `scan.startup.mode` specifies the startup mode for OceanBase C
- `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
@ -440,65 +528,31 @@ The OceanBase CDC Connector using [oblogclient](https://github.com/oceanbase/obl
The OceanBase CDC connector can also be a DataStream source. You can create a SourceFunction as the following shows:
```java
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.debezium.JsonDebeziumDeserializationSchema;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.catalog.Column;
import org.apache.flink.table.catalog.ResolvedSchema;
import org.apache.flink.table.catalog.UniqueConstraint;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.cdc.connectors.oceanbase.OceanBaseSource;
import org.apache.flink.cdc.connectors.oceanbase.source.RowDataOceanBaseDeserializationSchema;
import org.apache.flink.cdc.connectors.oceanbase.table.OceanBaseDeserializationSchema;
import org.apache.flink.cdc.connectors.oceanbase.table.StartupMode;
import java.time.ZoneId;
import java.util.Arrays;
import java.util.Collections;
public class OceanBaseSourceExample {
public static void main(String[] args) throws Exception {
ResolvedSchema resolvedSchema =
new ResolvedSchema(
Arrays.asList(
Column.physical("id", DataTypes.INT().notNull()),
Column.physical("name", DataTypes.STRING().notNull())),
Collections.emptyList(),
UniqueConstraint.primaryKey("pk", Collections.singletonList("id")));
RowType physicalDataType =
(RowType) resolvedSchema.toPhysicalRowDataType().getLogicalType();
TypeInformation<RowData> resultTypeInfo = InternalTypeInfo.of(physicalDataType);
String serverTimeZone = "+00:00";
OceanBaseDeserializationSchema<RowData> deserializer =
RowDataOceanBaseDeserializationSchema.newBuilder()
.setPhysicalRowType(physicalDataType)
.setResultTypeInfo(resultTypeInfo)
.setServerTimeZone(ZoneId.of(serverTimeZone))
.build();
SourceFunction<RowData> oceanBaseSource =
OceanBaseSource.<RowData>builder()
.rsList("127.0.0.1:2882:2881")
.startupMode(StartupMode.INITIAL)
SourceFunction<String> oceanBaseSource =
OceanBaseSource.<String>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$")
.hostname("127.0.0.1")
.port(2881)
.compatibleMode("mysql")
.jdbcDriver("com.mysql.jdbc.Driver")
.logProxyHost("127.0.0.1")
.logProxyPort(2983)
.serverTimeZone(serverTimeZone)
.deserializer(deserializer)
.rsList("127.0.0.1:2882:2881")
.serverTimeZone("+08:00")
.deserializer(new JsonDebeziumDeserializationSchema())
.build();
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
@ -511,6 +565,7 @@ public class OceanBaseSourceExample {
}
}
```
Data Type Mapping
----------------

@ -24,8 +24,7 @@ limitations under the License.
<version>${revision}</version>
</parent>
<properties>
<!-- Because of oceanbase docker image can not expose port quickly, so we need to specify testcontainers version to 1.15.3 -->
<jdbc.version>1.15.3</jdbc.version>
<oblogclient.version>1.1.2</oblogclient.version>
</properties>
<modelVersion>4.0.0</modelVersion>
@ -34,6 +33,13 @@ limitations under the License.
<packaging>jar</packaging>
<dependencies>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-cdc-base</artifactId>
<version>${project.version}</version>
</dependency>
<!-- Debezium dependencies -->
<dependency>
<groupId>org.apache.flink</groupId>
@ -47,12 +53,6 @@ limitations under the License.
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-cdc-common</artifactId>
<version>${project.version}</version>
</dependency>
<!-- OceanBase Log Client -->
<dependency>
<groupId>com.oceanbase</groupId>
@ -64,7 +64,13 @@ limitations under the License.
<dependency>
<groupId>mysql</groupId>
<artifactId>mysql-connector-java</artifactId>
<version>5.1.47</version>
<version>8.0.27</version>
<exclusions>
<exclusion>
<groupId>com.google.protobuf</groupId>
<artifactId>protobuf-java</artifactId>
</exclusion>
</exclusions>
</dependency>
<!-- test dependencies on Flink -->
@ -151,7 +157,7 @@ limitations under the License.
<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>jdbc</artifactId>
<version>${jdbc.version}</version>
<version>${testcontainers.version}</version>
<scope>test</scope>
</dependency>

@ -18,17 +18,19 @@
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.connectors.oceanbase.table.OceanBaseDeserializationSchema;
import org.apache.flink.cdc.connectors.oceanbase.table.StartupMode;
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 com.oceanbase.clogproxy.client.util.ClientIdGenerator;
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;
@ -47,7 +49,7 @@ public class OceanBaseSource {
public static class Builder<T> {
// common config
private StartupMode startupMode;
private StartupOptions startupOptions;
private String username;
private String password;
private String tenantName;
@ -73,11 +75,12 @@ public class OceanBaseSource {
private String configUrl;
private String workingMode;
private Properties obcdcProperties;
private Properties debeziumProperties;
private OceanBaseDeserializationSchema<T> deserializer;
private DebeziumDeserializationSchema<T> deserializer;
public Builder<T> startupMode(StartupMode startupMode) {
this.startupMode = startupMode;
public Builder<T> startupOptions(StartupOptions startupOptions) {
this.startupOptions = startupOptions;
return this;
}
@ -151,7 +154,7 @@ public class OceanBaseSource {
return this;
}
public Builder<T> logProxyPort(int logProxyPort) {
public Builder<T> logProxyPort(Integer logProxyPort) {
this.logProxyPort = logProxyPort;
return this;
}
@ -186,23 +189,44 @@ public class OceanBaseSource {
return this;
}
public Builder<T> deserializer(OceanBaseDeserializationSchema<T> deserializer) {
public Builder<T> debeziumProperties(Properties debeziumProperties) {
this.debeziumProperties = debeziumProperties;
return this;
}
public Builder<T> deserializer(DebeziumDeserializationSchema<T> deserializer) {
this.deserializer = deserializer;
return this;
}
public SourceFunction<T> build() {
switch (startupMode) {
case INITIAL:
checkNotNull(hostname, "hostname shouldn't be null on startup mode 'initial'");
checkNotNull(port, "port shouldn't be null on startup mode 'initial'");
checkNotNull(
compatibleMode,
"compatibleMode shouldn't be null on startup mode 'initial'");
checkNotNull(
jdbcDriver, "jdbcDriver shouldn't be null on startup mode 'initial'");
startupTimestamp = 0L;
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;
@ -213,15 +237,9 @@ public class OceanBaseSource {
break;
default:
throw new UnsupportedOperationException(
startupMode + " mode is not supported.");
startupOptions.startupMode + " mode is not supported.");
}
if (!startupMode.equals(StartupMode.INITIAL)
&& (StringUtils.isNotEmpty(databaseName)
|| StringUtils.isNotEmpty(tableName))) {
throw new IllegalArgumentException(
"If startup mode is not 'INITIAL', 'database-name' and 'table-name' must not be configured");
}
if (StringUtils.isNotEmpty(databaseName) || StringUtils.isNotEmpty(tableName)) {
if (StringUtils.isEmpty(databaseName) || StringUtils.isEmpty(tableName)) {
throw new IllegalArgumentException(
@ -233,57 +251,51 @@ public class OceanBaseSource {
"'database-name', 'table-name' or 'table-list' should be configured");
}
if (serverTimeZone == null) {
serverTimeZone = "+00:00";
}
ClientConf clientConf = null;
ObReaderConfig obReaderConfig = null;
if (connectTimeout == null) {
connectTimeout = Duration.ofSeconds(30);
}
if (!startupOptions.isSnapshotOnly()) {
if (logProxyClientId == null) {
logProxyClientId =
String.format(
"%s_%s_%s",
ClientIdGenerator.generate(),
Thread.currentThread().getId(),
checkNotNull(tenantName));
}
ClientConf clientConf =
ClientConf.builder()
.clientId(logProxyClientId)
.connectTimeoutMs((int) connectTimeout.toMillis())
.build();
ObReaderConfig 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(serverTimeZone);
if (obcdcProperties != null && !obcdcProperties.isEmpty()) {
Map<String, String> extraConfigs = new HashMap<>();
obcdcProperties.forEach((k, v) -> extraConfigs.put(k.toString(), v.toString()));
obReaderConfig.setExtraConfigs(extraConfigs);
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<String, String> extraConfigs = new HashMap<>();
obcdcProperties.forEach((k, v) -> extraConfigs.put(k.toString(), v.toString()));
obReaderConfig.setExtraConfigs(extraConfigs);
}
}
return new OceanBaseRichSourceFunction<>(
StartupMode.INITIAL.equals(startupMode),
startupOptions,
username,
password,
tenantName,
databaseName,
tableName,
tableList,
serverTimeZone,
connectTimeout,
hostname,
port,
@ -292,8 +304,9 @@ public class OceanBaseSource {
jdbcProperties,
logProxyHost,
logProxyPort,
clientConf,
logProxyClientId,
obReaderConfig,
debeziumProperties,
deserializer);
}
}

@ -23,8 +23,13 @@ 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.oceanbase.table.OceanBaseDeserializationSchema;
import org.apache.flink.cdc.connectors.oceanbase.table.OceanBaseRecord;
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;
@ -38,14 +43,24 @@ 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.ResultSetMetaData;
import java.sql.SQLException;
import java.time.Duration;
import java.time.Instant;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
@ -56,6 +71,7 @@ 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;
@ -72,13 +88,14 @@ public class OceanBaseRichSourceFunction<T> extends RichSourceFunction<T>
private static final Logger LOG = LoggerFactory.getLogger(OceanBaseRichSourceFunction.class);
private final boolean snapshot;
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;
@ -86,28 +103,35 @@ public class OceanBaseRichSourceFunction<T> extends RichSourceFunction<T>
private final String jdbcDriver;
private final Properties jdbcProperties;
private final String logProxyHost;
private final int logProxyPort;
private final ClientConf logProxyClientConf;
private final Integer logProxyPort;
private final String logProxyClientId;
private final ObReaderConfig obReaderConfig;
private final OceanBaseDeserializationSchema<T> deserializer;
private final Properties debeziumProperties;
private final DebeziumDeserializationSchema<T> deserializer;
private final List<OceanBaseRecord> changeRecordBuffer = new LinkedList<>();
private final List<SourceRecord> changeRecordBuffer = new LinkedList<>();
private transient Set<String> tableSet;
private transient OceanBaseConnectorConfig connectorConfig;
private transient OceanBaseSourceInfo sourceInfo;
private transient Set<TableId> 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<Long> offsetState;
private transient OutputCollector<T> outputCollector;
public OceanBaseRichSourceFunction(
boolean snapshot,
StartupOptions startupOptions,
String username,
String password,
String tenantName,
String databaseName,
String tableName,
String tableList,
String serverTimeZone,
Duration connectTimeout,
String hostname,
Integer port,
@ -115,27 +139,30 @@ public class OceanBaseRichSourceFunction<T> extends RichSourceFunction<T>
String jdbcDriver,
Properties jdbcProperties,
String logProxyHost,
int logProxyPort,
ClientConf logProxyClientConf,
Integer logProxyPort,
String logProxyClientId,
ObReaderConfig obReaderConfig,
OceanBaseDeserializationSchema<T> deserializer) {
this.snapshot = checkNotNull(snapshot);
Properties debeziumProperties,
DebeziumDeserializationSchema<T> deserializer) {
this.startupOptions = checkNotNull(startupOptions);
this.username = checkNotNull(username);
this.password = checkNotNull(password);
this.tenantName = checkNotNull(tenantName);
this.tenantName = tenantName;
this.databaseName = databaseName;
this.tableName = tableName;
this.tableList = tableList;
this.serverTimeZone = checkNotNull(serverTimeZone);
this.connectTimeout = checkNotNull(connectTimeout);
this.hostname = hostname;
this.port = port;
this.compatibleMode = compatibleMode;
this.jdbcDriver = jdbcDriver;
this.hostname = checkNotNull(hostname);
this.port = checkNotNull(port);
this.compatibleMode = checkNotNull(compatibleMode);
this.jdbcDriver = checkNotNull(jdbcDriver);
this.jdbcProperties = jdbcProperties;
this.logProxyHost = checkNotNull(logProxyHost);
this.logProxyPort = checkNotNull(logProxyPort);
this.logProxyClientConf = checkNotNull(logProxyClientConf);
this.obReaderConfig = checkNotNull(obReaderConfig);
this.logProxyHost = logProxyHost;
this.logProxyPort = logProxyPort;
this.logProxyClientId = logProxyClientId;
this.obReaderConfig = obReaderConfig;
this.debeziumProperties = debeziumProperties;
this.deserializer = checkNotNull(deserializer);
}
@ -143,6 +170,9 @@ public class OceanBaseRichSourceFunction<T> extends RichSourceFunction<T>
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
@ -152,27 +182,28 @@ public class OceanBaseRichSourceFunction<T> extends RichSourceFunction<T>
LOG.info("Start to initial table whitelist");
initTableWhiteList();
if (shouldReadSnapshot()) {
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");
}
LOG.info("Change events reading started");
readChangeRecords();
if (!startupOptions.isSnapshotOnly()) {
sourceInfo.setSnapshot(SnapshotRecord.FALSE);
LOG.info("Change events reading started");
readChangeRecords();
}
} finally {
cancel();
}
}
private boolean shouldReadSnapshot() {
return resolvedTimestamp <= 0 && snapshot;
}
private OceanBaseConnection getSnapshotConnection() {
if (snapshotConnection == null) {
snapshotConnection =
@ -201,27 +232,35 @@ public class OceanBaseRichSourceFunction<T> extends RichSourceFunction<T>
}
}
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<String> localTableSet = new HashSet<>();
final Set<TableId> localTableSet = new HashSet<>();
if (StringUtils.isNotBlank(tableList)) {
for (String s : tableList.split(",")) {
if (StringUtils.isNotBlank(s)) {
String[] schema = s.split("\\.");
localTableSet.add(String.format("%s.%s", schema[0].trim(), schema[1].trim()));
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<String> tables = getSnapshotConnection().getTables(databaseName, tableName);
LOG.info("Pattern matched tables: {}", tables);
localTableSet.addAll(tables);
List<TableId> tableIds = getSnapshotConnection().getTables(databaseName, tableName);
LOG.info("Pattern matched tables: {}", tableIds);
localTableSet.addAll(tableIds);
} catch (SQLException e) {
LOG.error(
String.format(
@ -238,50 +277,78 @@ public class OceanBaseRichSourceFunction<T> extends RichSourceFunction<T>
LOG.info("Table list: {}", localTableSet);
this.tableSet = localTableSet;
this.obReaderConfig.setTableWhiteList(String.format("%s.*.*", tenantName));
// 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(
table -> {
String[] schema = table.split("\\.");
readSnapshotRecordsByTable(schema[0], schema[1]);
});
tableSet.forEach(this::readSnapshotRecordsByTable);
}
private void readSnapshotRecordsByTable(String databaseName, String tableName) {
OceanBaseRecord.SourceInfo sourceInfo =
new OceanBaseRecord.SourceInfo(
tenantName, databaseName, tableName, resolvedTimestamp);
String fullName;
if ("mysql".equalsIgnoreCase(compatibleMode)) {
fullName = String.format("`%s`.`%s`", databaseName, tableName);
} else {
fullName = String.format("%s.%s", databaseName, tableName);
}
try {
LOG.info("Start to read snapshot from {}", fullName);
getSnapshotConnection()
.query(
"SELECT * FROM " + fullName,
rs -> {
ResultSetMetaData metaData = rs.getMetaData();
while (rs.next()) {
Map<String, Object> fieldMap = new HashMap<>();
for (int i = 0; i < metaData.getColumnCount(); i++) {
fieldMap.put(
metaData.getColumnName(i + 1), rs.getObject(i + 1));
}
OceanBaseRecord record =
new OceanBaseRecord(sourceInfo, fieldMap);
try {
deserializer.deserialize(record, outputCollector);
} catch (Exception e) {
LOG.error("Deserialize snapshot record failed ", e);
throw new FlinkRuntimeException(e);
}
}
});
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<Field> 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);
@ -295,8 +362,21 @@ public class OceanBaseRichSourceFunction<T> extends RichSourceFunction<T>
LOG.info("Restore from timestamp: {}", resolvedTimestamp);
}
logProxyClient =
new LogProxyClient(logProxyHost, logProxyPort, obReaderConfig, logProxyClientConf);
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);
@ -321,7 +401,7 @@ public class OceanBaseRichSourceFunction<T> extends RichSourceFunction<T>
if (!started) {
break;
}
OceanBaseRecord record = getChangeRecord(message);
SourceRecord record = getChangeRecord(message);
if (record != null) {
changeRecordBuffer.add(record);
}
@ -355,14 +435,14 @@ public class OceanBaseRichSourceFunction<T> extends RichSourceFunction<T>
@Override
public void onException(LogProxyClientException e) {
LOG.error("LogProxyClient exception", e);
logProxyClientException = e;
logProxyClient.stop();
}
});
LOG.info(
"Try to start LogProxyClient with client id: {}, config: {}",
logProxyClientConf.getClientId(),
clientConf.getClientId(),
obReaderConfig);
logProxyClient.start();
@ -373,20 +453,87 @@ public class OceanBaseRichSourceFunction<T> extends RichSourceFunction<T>
LOG.info("LogProxyClient started successfully");
logProxyClient.join();
if (logProxyClientException != null) {
throw new RuntimeException("LogProxyClient exception", logProxyClientException);
}
}
private OceanBaseRecord getChangeRecord(LogMessage message) {
private SourceRecord getChangeRecord(LogMessage message) {
String databaseName = message.getDbName().replace(tenantName + ".", "");
if (!tableSet.contains(String.format("%s.%s", databaseName, message.getTableName()))) {
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<Field> fields = valueSchema.fields();
Struct before, after;
Object[] beforeFieldValues, afterFieldValues;
Map<String, Object> beforeValueMap = new HashMap<>();
Map<String, Object> 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;
}
OceanBaseRecord.SourceInfo sourceInfo =
new OceanBaseRecord.SourceInfo(
tenantName,
databaseName,
message.getTableName(),
Long.parseLong(message.getSafeTimestamp()));
return new OceanBaseRecord(sourceInfo, message.getOpt(), message.getFieldList());
String encoding = field.getEncoding();
if ("binary".equalsIgnoreCase(encoding)) {
return field.getValue().getBytes();
}
return field.getValue().toString(encoding);
}
@Override

@ -1,632 +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.typeinfo.TypeInformation;
import org.apache.flink.cdc.connectors.oceanbase.table.OceanBaseAppendMetadataCollector;
import org.apache.flink.cdc.connectors.oceanbase.table.OceanBaseDeserializationSchema;
import org.apache.flink.cdc.connectors.oceanbase.table.OceanBaseMetadataConverter;
import org.apache.flink.cdc.connectors.oceanbase.table.OceanBaseRecord;
import org.apache.flink.cdc.debezium.utils.TemporalConversions;
import org.apache.flink.table.data.DecimalData;
import org.apache.flink.table.data.GenericArrayData;
import org.apache.flink.table.data.GenericRowData;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.StringData;
import org.apache.flink.table.data.TimestampData;
import org.apache.flink.table.types.logical.DecimalType;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.types.RowKind;
import org.apache.flink.util.Collector;
import com.oceanbase.oms.logmessage.ByteString;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.sql.Date;
import java.sql.Time;
import java.sql.Timestamp;
import java.time.LocalDateTime;
import java.time.ZoneId;
import java.util.Arrays;
import java.util.Map;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
* Deserialization schema from OceanBase object to Flink Table/SQL internal data structure {@link
* RowData}.
*/
public class RowDataOceanBaseDeserializationSchema
implements OceanBaseDeserializationSchema<RowData> {
private static final long serialVersionUID = 1L;
/** TypeInformation of the produced {@link RowData}. * */
private final TypeInformation<RowData> resultTypeInfo;
/**
* Runtime converter that OceanBase record data into {@link RowData} consisted of physical
* column values.
*/
private final OceanBaseDeserializationRuntimeConverter physicalConverter;
/** Whether the deserializer needs to handle metadata columns. */
private final boolean hasMetadata;
/**
* A wrapped output collector which is used to append metadata columns after physical columns.
*/
private final OceanBaseAppendMetadataCollector appendMetadataCollector;
/** Returns a builder to build {@link RowDataOceanBaseDeserializationSchema}. */
public static RowDataOceanBaseDeserializationSchema.Builder newBuilder() {
return new RowDataOceanBaseDeserializationSchema.Builder();
}
RowDataOceanBaseDeserializationSchema(
RowType physicalDataType,
OceanBaseMetadataConverter[] metadataConverters,
TypeInformation<RowData> resultTypeInfo,
ZoneId serverTimeZone) {
this.hasMetadata = checkNotNull(metadataConverters).length > 0;
this.appendMetadataCollector = new OceanBaseAppendMetadataCollector(metadataConverters);
this.physicalConverter = createConverter(checkNotNull(physicalDataType), serverTimeZone);
this.resultTypeInfo = checkNotNull(resultTypeInfo);
}
@Override
public void deserialize(OceanBaseRecord record, Collector<RowData> out) throws Exception {
RowData physicalRow;
if (record.isSnapshotRecord()) {
physicalRow = (GenericRowData) physicalConverter.convert(record.getJdbcFields());
physicalRow.setRowKind(RowKind.INSERT);
emit(record, physicalRow, out);
} else {
switch (record.getOpt()) {
case INSERT:
physicalRow =
(GenericRowData)
physicalConverter.convert(record.getLogMessageFieldsAfter());
physicalRow.setRowKind(RowKind.INSERT);
emit(record, physicalRow, out);
break;
case DELETE:
physicalRow =
(GenericRowData)
physicalConverter.convert(record.getLogMessageFieldsBefore());
physicalRow.setRowKind(RowKind.DELETE);
emit(record, physicalRow, out);
break;
case UPDATE:
physicalRow =
(GenericRowData)
physicalConverter.convert(record.getLogMessageFieldsBefore());
physicalRow.setRowKind(RowKind.UPDATE_BEFORE);
emit(record, physicalRow, out);
physicalRow =
(GenericRowData)
physicalConverter.convert(record.getLogMessageFieldsAfter());
physicalRow.setRowKind(RowKind.UPDATE_AFTER);
emit(record, physicalRow, out);
break;
default:
throw new IllegalArgumentException(
"Unsupported log message record type: " + record.getOpt());
}
}
}
private void emit(OceanBaseRecord row, RowData physicalRow, Collector<RowData> collector) {
if (!hasMetadata) {
collector.collect(physicalRow);
return;
}
appendMetadataCollector.inputRecord = row;
appendMetadataCollector.outputCollector = collector;
appendMetadataCollector.collect(physicalRow);
}
@Override
public TypeInformation<RowData> getProducedType() {
return resultTypeInfo;
}
/** Builder class of {@link RowDataOceanBaseDeserializationSchema}. */
public static class Builder {
private RowType physicalRowType;
private TypeInformation<RowData> resultTypeInfo;
private OceanBaseMetadataConverter[] metadataConverters = new OceanBaseMetadataConverter[0];
private ZoneId serverTimeZone = ZoneId.of("UTC");
public RowDataOceanBaseDeserializationSchema.Builder setPhysicalRowType(
RowType physicalRowType) {
this.physicalRowType = physicalRowType;
return this;
}
public RowDataOceanBaseDeserializationSchema.Builder setMetadataConverters(
OceanBaseMetadataConverter[] metadataConverters) {
this.metadataConverters = metadataConverters;
return this;
}
public RowDataOceanBaseDeserializationSchema.Builder setResultTypeInfo(
TypeInformation<RowData> resultTypeInfo) {
this.resultTypeInfo = resultTypeInfo;
return this;
}
public RowDataOceanBaseDeserializationSchema.Builder setServerTimeZone(
ZoneId serverTimeZone) {
this.serverTimeZone = serverTimeZone;
return this;
}
public RowDataOceanBaseDeserializationSchema build() {
return new RowDataOceanBaseDeserializationSchema(
physicalRowType, metadataConverters, resultTypeInfo, serverTimeZone);
}
}
private static OceanBaseDeserializationRuntimeConverter createConverter(
LogicalType type, ZoneId serverTimeZone) {
return wrapIntoNullableConverter(createNotNullConverter(type, serverTimeZone));
}
private static OceanBaseDeserializationRuntimeConverter wrapIntoNullableConverter(
OceanBaseDeserializationRuntimeConverter converter) {
return new OceanBaseDeserializationRuntimeConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object convert(Object object) throws Exception {
if (object == null) {
return null;
}
return converter.convert(object);
}
};
}
public static OceanBaseDeserializationRuntimeConverter createNotNullConverter(
LogicalType type, ZoneId serverTimeZone) {
switch (type.getTypeRoot()) {
case ROW:
return createRowConverter((RowType) type, serverTimeZone);
case NULL:
return convertToNull();
case BOOLEAN:
return convertToBoolean();
case TINYINT:
return convertToTinyInt();
case SMALLINT:
return convertToSmallInt();
case INTEGER:
case INTERVAL_YEAR_MONTH:
return convertToInt();
case BIGINT:
case INTERVAL_DAY_TIME:
return convertToLong();
case DATE:
return convertToDate();
case TIME_WITHOUT_TIME_ZONE:
return convertToTime();
case TIMESTAMP_WITHOUT_TIME_ZONE:
return convertToTimestamp();
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
return convertToLocalTimeZoneTimestamp(serverTimeZone);
case FLOAT:
return convertToFloat();
case DOUBLE:
return convertToDouble();
case CHAR:
case VARCHAR:
return convertToString();
case BINARY:
return convertToBinary();
case VARBINARY:
return convertToBytes();
case DECIMAL:
return createDecimalConverter((DecimalType) type);
case ARRAY:
return createArrayConverter();
default:
throw new UnsupportedOperationException("Unsupported type: " + type);
}
}
private static OceanBaseDeserializationRuntimeConverter createRowConverter(
RowType rowType, ZoneId serverTimeZone) {
final OceanBaseDeserializationRuntimeConverter[] fieldConverters =
rowType.getFields().stream()
.map(RowType.RowField::getType)
.map(logicType -> createConverter(logicType, serverTimeZone))
.toArray(OceanBaseDeserializationRuntimeConverter[]::new);
final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]);
return new OceanBaseDeserializationRuntimeConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object convert(Object object) {
int arity = fieldNames.length;
GenericRowData row = new GenericRowData(arity);
Map<String, Object> fieldMap = (Map<String, Object>) object;
for (int i = 0; i < arity; i++) {
String fieldName = fieldNames[i];
Object value = fieldMap.get(fieldName);
try {
row.setField(i, fieldConverters[i].convert(value));
} catch (Exception e) {
throw new RuntimeException(
"Failed to convert field '" + fieldName + "' with value: " + value,
e);
}
}
return row;
}
};
}
private static OceanBaseDeserializationRuntimeConverter convertToNull() {
return new OceanBaseDeserializationRuntimeConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object convert(Object object) {
return null;
}
};
}
private static OceanBaseDeserializationRuntimeConverter convertToBoolean() {
return new OceanBaseDeserializationRuntimeConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object convert(Object object) {
if (object instanceof byte[]) {
return "1".equals(new String((byte[]) object, StandardCharsets.UTF_8));
}
return Boolean.parseBoolean(object.toString()) || "1".equals(object.toString());
}
};
}
private static OceanBaseDeserializationRuntimeConverter convertToTinyInt() {
return new OceanBaseDeserializationRuntimeConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object convert(Object object) {
return Byte.parseByte(object.toString());
}
};
}
private static OceanBaseDeserializationRuntimeConverter convertToSmallInt() {
return new OceanBaseDeserializationRuntimeConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object convert(Object object) {
return Short.parseShort(object.toString());
}
};
}
private static OceanBaseDeserializationRuntimeConverter convertToInt() {
return new OceanBaseDeserializationRuntimeConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object convert(Object object) {
if (object instanceof Integer) {
return object;
} else if (object instanceof Long) {
return ((Long) object).intValue();
} else if (object instanceof Date) {
return ((Date) object).toLocalDate().getYear();
} else {
return Integer.parseInt(object.toString());
}
}
};
}
private static OceanBaseDeserializationRuntimeConverter convertToLong() {
return new OceanBaseDeserializationRuntimeConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object convert(Object object) {
if (object instanceof Integer) {
return ((Integer) object).longValue();
} else if (object instanceof Long) {
return object;
} else {
return Long.parseLong(object.toString());
}
}
};
}
private static OceanBaseDeserializationRuntimeConverter convertToDouble() {
return new OceanBaseDeserializationRuntimeConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object convert(Object object) {
if (object instanceof Float) {
return ((Float) object).doubleValue();
} else if (object instanceof Double) {
return object;
} else {
return Double.parseDouble(object.toString());
}
}
};
}
private static OceanBaseDeserializationRuntimeConverter convertToFloat() {
return new OceanBaseDeserializationRuntimeConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object convert(Object object) {
if (object instanceof Float) {
return object;
} else if (object instanceof Double) {
return ((Double) object).floatValue();
} else {
return Float.parseFloat(object.toString());
}
}
};
}
private static OceanBaseDeserializationRuntimeConverter convertToDate() {
return new OceanBaseDeserializationRuntimeConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object convert(Object object) {
if (object instanceof String) {
object = Date.valueOf((String) object);
}
return (int) TemporalConversions.toLocalDate(object).toEpochDay();
}
};
}
private static OceanBaseDeserializationRuntimeConverter convertToTime() {
return new OceanBaseDeserializationRuntimeConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object convert(Object object) {
if (object instanceof Long) {
return (int) ((Long) object / 1000_000);
}
if (object instanceof String) {
object = Time.valueOf((String) object);
}
return TemporalConversions.toLocalTime(object).toSecondOfDay() * 1000;
}
};
}
private static OceanBaseDeserializationRuntimeConverter convertToTimestamp() {
return new OceanBaseDeserializationRuntimeConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object convert(Object object) {
if (object instanceof String) {
object = Timestamp.valueOf((String) object);
}
if (object instanceof Timestamp) {
return TimestampData.fromTimestamp((Timestamp) object);
}
if (object instanceof LocalDateTime) {
return TimestampData.fromLocalDateTime((LocalDateTime) object);
}
throw new IllegalArgumentException(
"Unable to convert to TimestampData from unexpected value '"
+ object
+ "' of type "
+ object.getClass().getName());
}
};
}
private static OceanBaseDeserializationRuntimeConverter convertToLocalTimeZoneTimestamp(
ZoneId serverTimeZone) {
return new OceanBaseDeserializationRuntimeConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object convert(Object object) {
if (object instanceof String) {
object = Timestamp.valueOf((String) object);
}
if (object instanceof Timestamp) {
return TimestampData.fromInstant(
((Timestamp) object)
.toLocalDateTime()
.atZone(serverTimeZone)
.toInstant());
}
if (object instanceof LocalDateTime) {
return TimestampData.fromInstant(
((LocalDateTime) object).atZone(serverTimeZone).toInstant());
}
throw new IllegalArgumentException(
"Unable to convert to TimestampData from unexpected value '"
+ object
+ "' of type "
+ object.getClass().getName());
}
};
}
private static OceanBaseDeserializationRuntimeConverter convertToString() {
return new OceanBaseDeserializationRuntimeConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object convert(Object object) {
return StringData.fromString(object.toString());
}
};
}
private static OceanBaseDeserializationRuntimeConverter convertToBinary() {
return new OceanBaseDeserializationRuntimeConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object convert(Object object) {
if (object instanceof String) {
try {
long v = Long.parseLong((String) object);
byte[] bytes = ByteBuffer.allocate(8).putLong(v).array();
int i = 0;
while (i < Long.BYTES - 1 && bytes[i] == 0) {
i++;
}
return Arrays.copyOfRange(bytes, i, Long.BYTES);
} catch (NumberFormatException e) {
return ((String) object).getBytes(StandardCharsets.UTF_8);
}
} else if (object instanceof byte[]) {
String str = new String((byte[]) object, StandardCharsets.US_ASCII);
return str.getBytes(StandardCharsets.UTF_8);
} else if (object instanceof ByteBuffer) {
ByteBuffer byteBuffer = (ByteBuffer) object;
byte[] bytes = new byte[byteBuffer.remaining()];
byteBuffer.get(bytes);
return bytes;
} else {
throw new UnsupportedOperationException(
"Unsupported BINARY value type: " + object.getClass().getSimpleName());
}
}
};
}
private static OceanBaseDeserializationRuntimeConverter convertToBytes() {
return new OceanBaseDeserializationRuntimeConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object convert(Object object) {
if (object instanceof String) {
return ((String) object).getBytes(StandardCharsets.UTF_8);
} else if (object instanceof byte[]) {
return object;
} else if (object instanceof ByteBuffer) {
ByteBuffer byteBuffer = (ByteBuffer) object;
byte[] bytes = new byte[byteBuffer.remaining()];
byteBuffer.get(bytes);
return bytes;
} else {
throw new UnsupportedOperationException(
"Unsupported BYTES value type: " + object.getClass().getSimpleName());
}
}
};
}
private static OceanBaseDeserializationRuntimeConverter createDecimalConverter(
DecimalType decimalType) {
final int precision = decimalType.getPrecision();
final int scale = decimalType.getScale();
return new OceanBaseDeserializationRuntimeConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object convert(Object object) {
BigDecimal bigDecimal;
if (object instanceof String) {
bigDecimal = new BigDecimal((String) object);
} else if (object instanceof Long) {
bigDecimal = new BigDecimal((Long) object);
} else if (object instanceof BigInteger) {
bigDecimal = new BigDecimal((BigInteger) object);
} else if (object instanceof Double) {
bigDecimal = BigDecimal.valueOf((Double) object);
} else if (object instanceof BigDecimal) {
bigDecimal = (BigDecimal) object;
} else {
throw new IllegalArgumentException(
"Unable to convert to decimal from unexpected value '"
+ object
+ "' of type "
+ object.getClass());
}
return DecimalData.fromBigDecimal(bigDecimal, precision, scale);
}
};
}
private static OceanBaseDeserializationRuntimeConverter createArrayConverter() {
return new OceanBaseDeserializationRuntimeConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object convert(Object object) {
String s;
if (object instanceof ByteString) {
s = ((ByteString) object).toString(StandardCharsets.UTF_8.name());
} else {
s = object.toString();
}
String[] strArray = s.split(",");
StringData[] stringDataArray = new StringData[strArray.length];
for (int i = 0; i < strArray.length; i++) {
stringDataArray[i] = StringData.fromString(strArray[i]);
}
return new GenericArrayData(stringDataArray);
}
};
}
}

@ -0,0 +1,88 @@
/*
* 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<String> 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();
}
}

@ -15,12 +15,16 @@
* limitations under the License.
*/
package org.apache.flink.cdc.connectors.oceanbase.source;
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;
@ -28,11 +32,17 @@ 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;
@ -41,13 +51,19 @@ public class OceanBaseConnection extends JdbcConnection {
private static final Logger LOG = LoggerFactory.getLogger(OceanBaseConnection.class);
private static final String QUOTED_CHARACTER = "`";
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(
@ -62,9 +78,10 @@ public class OceanBaseConnection extends JdbcConnection {
ClassLoader classLoader) {
super(
config(hostname, port, user, password, timeout),
factory(jdbcDriver, jdbcProperties, classLoader),
QUOTED_CHARACTER,
QUOTED_CHARACTER);
JdbcConnection.patternBasedFactory(
formatJdbcUrl(jdbcDriver, jdbcProperties), jdbcDriver, classLoader),
getQuote(compatibleMode) + "",
getQuote(compatibleMode) + "");
this.compatibleMode = compatibleMode;
}
@ -86,7 +103,7 @@ public class OceanBaseConnection extends JdbcConnection {
combinedProperties.putAll(jdbcProperties);
}
String urlPattern =
jdbcDriver.toLowerCase().contains("oceanbase") ? OB_URL_PATTERN : MYSQL_URL_PATTERN;
OceanBaseUtils.isOceanBaseDriver(jdbcDriver) ? OB_URL_PATTERN : MYSQL_URL_PATTERN;
StringBuilder jdbcUrlStringBuilder = new StringBuilder(urlPattern);
combinedProperties.forEach(
(key, value) -> {
@ -106,10 +123,8 @@ public class OceanBaseConnection extends JdbcConnection {
return defaultJdbcProperties;
}
private static JdbcConnection.ConnectionFactory factory(
String jdbcDriver, Properties jdbcProperties, ClassLoader classLoader) {
return JdbcConnection.patternBasedFactory(
formatJdbcUrl(jdbcDriver, jdbcProperties), jdbcDriver, classLoader);
private static char getQuote(String compatibleMode) {
return "mysql".equalsIgnoreCase(compatibleMode) ? '`' : '"';
}
/**
@ -155,11 +170,11 @@ public class OceanBaseConnection extends JdbcConnection {
*
* @param dbPattern Database name pattern.
* @param tbPattern Table name pattern.
* @return Table list.
* @return TableId list.
* @throws SQLException If a database access error occurs.
*/
public List<String> getTables(String dbPattern, String tbPattern) throws SQLException {
List<String> result = new ArrayList<>();
public List<TableId> getTables(String dbPattern, String tbPattern) throws SQLException {
List<TableId> result = new ArrayList<>();
DatabaseMetaData metaData = connection().getMetaData();
switch (compatibleMode.toLowerCase()) {
case "mysql":
@ -171,27 +186,28 @@ public class OceanBaseConnection extends JdbcConnection {
for (String dbName : dbNames) {
List<String> tableNames =
getResultList(
metaData.getTables(dbName, null, null, new String[] {"TABLE"}),
metaData.getTables(dbName, null, null, supportedTableTypes()),
"TABLE_NAME");
tableNames.stream()
.filter(tbName -> Pattern.matches(tbPattern, tbName))
.forEach(tbName -> result.add(dbName + "." + tbName));
.forEach(tbName -> result.add(new TableId(dbName, null, tbName)));
}
break;
case "oracle":
dbNames = getResultList(metaData.getSchemas(), "TABLE_SCHEM");
dbNames =
dbNames.stream()
.filter(dbName -> Pattern.matches(dbPattern, dbName))
List<String> schemaNames = getResultList(metaData.getSchemas(), "TABLE_SCHEM");
schemaNames =
schemaNames.stream()
.filter(schemaName -> Pattern.matches(dbPattern, schemaName))
.collect(Collectors.toList());
for (String dbName : dbNames) {
for (String schemaName : schemaNames) {
List<String> tableNames =
getResultList(
metaData.getTables(null, dbName, null, new String[] {"TABLE"}),
metaData.getTables(
null, schemaName, null, supportedTableTypes()),
"TABLE_NAME");
tableNames.stream()
.filter(tbName -> Pattern.matches(tbPattern, tbName))
.forEach(tbName -> result.add(dbName + "." + tbName));
.forEach(tbName -> result.add(new TableId(null, schemaName, tbName)));
}
break;
default:
@ -207,4 +223,112 @@ public class OceanBaseConnection extends JdbcConnection {
}
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<TableId> capturedTables)
throws SQLException {
Set<TableId> tableIdsBefore = new HashSet<>(tables.tableIds());
DatabaseMetaData metadata = connection().getMetaData();
Map<TableId, List<Column>> 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<TableId, List<Column>> tableEntry : columnsByTable.entrySet()) {
// First get the primary key information, which must be done for *each* table ...
List<String> pkColumnNames = readPrimaryKeyNames(metadata, tableEntry.getKey());
// Then define the table ...
List<Column> 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;
}
}
}

@ -0,0 +1,74 @@
/*
* 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<DeserializationRuntimeConverter> createUserDefinedConverter(
LogicalType logicalType, ZoneId serverTimeZone) {
switch (logicalType.getTypeRoot()) {
case ARRAY:
return createArrayConverter();
default:
return Optional.empty();
}
}
};
}
private static Optional<DeserializationRuntimeConverter> 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));
}
});
}
}

@ -0,0 +1,509 @@
/*
* 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);
}
}
}

@ -0,0 +1,101 @@
/*
* 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<TableId> 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(","));
}
}

@ -0,0 +1,74 @@
/*
* 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<OceanBaseSourceInfo> {
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;
}
}

@ -0,0 +1,53 @@
/*
* 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());
}
}

@ -0,0 +1,81 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
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<TableId, TableChanges.TableChange> 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<TableId> tableIdSet = new HashSet<>();
tableIdSet.add(tableId);
final Map<TableId, TableChanges.TableChange> 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);
}
}

@ -1,57 +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.common.annotation.Internal;
import org.apache.flink.table.data.GenericRowData;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.utils.JoinedRowData;
import org.apache.flink.util.Collector;
import java.io.Serializable;
/** Emits a row with physical fields and metadata fields. */
@Internal
public class OceanBaseAppendMetadataCollector implements Collector<RowData>, Serializable {
private static final long serialVersionUID = 1L;
private final OceanBaseMetadataConverter[] metadataConverters;
public transient OceanBaseRecord inputRecord;
public transient Collector<RowData> outputCollector;
public OceanBaseAppendMetadataCollector(OceanBaseMetadataConverter[] metadataConverters) {
this.metadataConverters = metadataConverters;
}
@Override
public void collect(RowData physicalRow) {
GenericRowData metaRow = new GenericRowData(metadataConverters.length);
for (int i = 0; i < metadataConverters.length; i++) {
Object meta = metadataConverters[i].read(inputRecord);
metaRow.setField(i, meta);
}
RowData outRow = new JoinedRowData(physicalRow.getRowKind(), physicalRow, metaRow);
outputCollector.collect(outRow);
}
@Override
public void close() {
// nothing to do
}
}

@ -1,37 +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.java.typeutils.ResultTypeQueryable;
import org.apache.flink.cdc.common.annotation.PublicEvolving;
import org.apache.flink.util.Collector;
import java.io.Serializable;
/**
* The deserialization schema describes how to turn the OceanBase record into data types (Java/Scala
* objects) that are processed by Flink.
*
* @param <T> The type created by the deserialization schema.
*/
@PublicEvolving
public interface OceanBaseDeserializationSchema<T> extends Serializable, ResultTypeQueryable<T> {
/** Deserialize the OceanBase record, it is represented in {@link OceanBaseRecord}. */
void deserialize(OceanBaseRecord record, Collector<T> out) throws Exception;
}

@ -1,29 +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.common.annotation.Internal;
import java.io.Serializable;
/** A converter converts OceanBase record metadata into Flink internal data structures. */
@FunctionalInterface
@Internal
public interface OceanBaseMetadataConverter extends Serializable {
Object read(OceanBaseRecord record);
}

@ -17,37 +17,65 @@
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().notNull(),
new OceanBaseMetadataConverter() {
DataTypes.STRING().nullable(),
new MetadataConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object read(OceanBaseRecord record) {
return StringData.fromString(record.getSourceInfo().getTenant());
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().notNull(),
new OceanBaseMetadataConverter() {
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(OceanBaseRecord record) {
return StringData.fromString(record.getSourceInfo().getDatabase());
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));
}
}),
@ -55,12 +83,15 @@ public enum OceanBaseReadableMetadata {
TABLE(
"table_name",
DataTypes.STRING().notNull(),
new OceanBaseMetadataConverter() {
new MetadataConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object read(OceanBaseRecord record) {
return StringData.fromString(record.getSourceInfo().getTable());
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));
}
}),
@ -71,13 +102,15 @@ public enum OceanBaseReadableMetadata {
OP_TS(
"op_ts",
DataTypes.TIMESTAMP_LTZ(3).notNull(),
new OceanBaseMetadataConverter() {
new MetadataConverter() {
private static final long serialVersionUID = 1L;
@Override
public Object read(OceanBaseRecord record) {
public Object read(SourceRecord record) {
Struct messageStruct = (Struct) record.value();
Struct sourceStruct = messageStruct.getStruct(Envelope.FieldName.SOURCE);
return TimestampData.fromEpochMillis(
record.getSourceInfo().getTimestampS() * 1000);
sourceStruct.getInt64(OceanBaseSourceInfo.TIMESTAMP_KEY));
}
});
@ -85,9 +118,9 @@ public enum OceanBaseReadableMetadata {
private final DataType dataType;
private final OceanBaseMetadataConverter converter;
private final MetadataConverter converter;
OceanBaseReadableMetadata(String key, DataType dataType, OceanBaseMetadataConverter converter) {
OceanBaseReadableMetadata(String key, DataType dataType, MetadataConverter converter) {
this.key = key;
this.dataType = dataType;
this.converter = converter;
@ -101,7 +134,7 @@ public enum OceanBaseReadableMetadata {
return dataType;
}
public OceanBaseMetadataConverter getConverter() {
public MetadataConverter getConverter() {
return converter;
}
}

@ -1,133 +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 com.oceanbase.oms.logmessage.DataMessage;
import java.io.Serializable;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/** An internal data structure representing record of OceanBase. */
public class OceanBaseRecord implements Serializable {
private static final long serialVersionUID = 1L;
private final SourceInfo sourceInfo;
private final boolean isSnapshotRecord;
private final Map<String, Object> jdbcFields;
private final DataMessage.Record.Type opt;
private final Map<String, Object> logMessageFieldsBefore;
private final Map<String, Object> logMessageFieldsAfter;
public OceanBaseRecord(SourceInfo sourceInfo, Map<String, Object> jdbcFields) {
this.sourceInfo = sourceInfo;
this.isSnapshotRecord = true;
this.jdbcFields = jdbcFields;
this.opt = null;
this.logMessageFieldsBefore = null;
this.logMessageFieldsAfter = null;
}
public OceanBaseRecord(
SourceInfo sourceInfo,
DataMessage.Record.Type opt,
List<DataMessage.Record.Field> logMessageFieldList) {
this.sourceInfo = sourceInfo;
this.isSnapshotRecord = false;
this.jdbcFields = null;
this.opt = opt;
this.logMessageFieldsBefore = new HashMap<>();
this.logMessageFieldsAfter = new HashMap<>();
for (DataMessage.Record.Field field : logMessageFieldList) {
if (field.isPrev()) {
logMessageFieldsBefore.put(field.getFieldname(), getFieldStringValue(field));
} else {
logMessageFieldsAfter.put(field.getFieldname(), getFieldStringValue(field));
}
}
}
private String getFieldStringValue(DataMessage.Record.Field field) {
if (field.getValue() == null) {
return null;
}
String encoding = field.getEncoding();
if ("binary".equalsIgnoreCase(encoding)) {
return field.getValue().toString("utf8");
}
return field.getValue().toString(encoding);
}
public SourceInfo getSourceInfo() {
return sourceInfo;
}
public boolean isSnapshotRecord() {
return isSnapshotRecord;
}
public Map<String, Object> getJdbcFields() {
return jdbcFields;
}
public DataMessage.Record.Type getOpt() {
return opt;
}
public Map<String, Object> getLogMessageFieldsBefore() {
return logMessageFieldsBefore;
}
public Map<String, Object> getLogMessageFieldsAfter() {
return logMessageFieldsAfter;
}
/** Information about the source of record. */
public static class SourceInfo implements Serializable {
private static final long serialVersionUID = 1L;
private final String tenant;
private final String database;
private final String table;
private final long timestampS;
public SourceInfo(String tenant, String database, String table, long timestampS) {
this.tenant = tenant;
this.database = database;
this.table = table;
this.timestampS = timestampS;
}
public String getTenant() {
return tenant;
}
public String getDatabase() {
return database;
}
public String getTable() {
return table;
}
public long getTimestampS() {
return timestampS;
}
}
}

@ -18,8 +18,12 @@
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.RowDataOceanBaseDeserializationSchema;
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;
@ -47,7 +51,7 @@ public class OceanBaseTableSource implements ScanTableSource, SupportsReadingMet
private final ResolvedSchema physicalSchema;
private final StartupMode startupMode;
private final StartupOptions startupOptions;
private final String username;
private final String password;
private final String tenantName;
@ -58,7 +62,7 @@ public class OceanBaseTableSource implements ScanTableSource, SupportsReadingMet
private final String serverTimeZone;
private final String hostname;
private final Integer port;
private final int port;
private final String compatibleMode;
private final String jdbcDriver;
private final Properties jdbcProperties;
@ -71,6 +75,7 @@ public class OceanBaseTableSource implements ScanTableSource, SupportsReadingMet
private final String configUrl;
private final String workingMode;
private final Properties obcdcProperties;
private final Properties debeziumProperties;
// --------------------------------------------------------------------------------------------
// Mutable attributes
@ -84,7 +89,7 @@ public class OceanBaseTableSource implements ScanTableSource, SupportsReadingMet
public OceanBaseTableSource(
ResolvedSchema physicalSchema,
StartupMode startupMode,
StartupOptions startupOptions,
String username,
String password,
String tenantName,
@ -94,7 +99,7 @@ public class OceanBaseTableSource implements ScanTableSource, SupportsReadingMet
String serverTimeZone,
Duration connectTimeout,
String hostname,
Integer port,
int port,
String compatibleMode,
String jdbcDriver,
Properties jdbcProperties,
@ -105,30 +110,32 @@ public class OceanBaseTableSource implements ScanTableSource, SupportsReadingMet
String rsList,
String configUrl,
String workingMode,
Properties obcdcProperties) {
Properties obcdcProperties,
Properties debeziumProperties) {
this.physicalSchema = physicalSchema;
this.startupMode = checkNotNull(startupMode);
this.startupOptions = checkNotNull(startupOptions);
this.username = checkNotNull(username);
this.password = checkNotNull(password);
this.tenantName = checkNotNull(tenantName);
this.tenantName = tenantName;
this.databaseName = databaseName;
this.tableName = tableName;
this.tableList = tableList;
this.serverTimeZone = serverTimeZone;
this.connectTimeout = connectTimeout;
this.hostname = hostname;
this.hostname = checkNotNull(hostname);
this.port = port;
this.compatibleMode = compatibleMode;
this.jdbcDriver = jdbcDriver;
this.jdbcProperties = jdbcProperties;
this.logProxyHost = checkNotNull(logProxyHost);
this.logProxyPort = checkNotNull(logProxyPort);
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();
@ -143,20 +150,25 @@ public class OceanBaseTableSource implements ScanTableSource, SupportsReadingMet
public ScanRuntimeProvider getScanRuntimeProvider(ScanContext context) {
RowType physicalDataType =
(RowType) physicalSchema.toPhysicalRowDataType().getLogicalType();
OceanBaseMetadataConverter[] metadataConverters = getMetadataConverters();
MetadataConverter[] metadataConverters = getMetadataConverters();
TypeInformation<RowData> resultTypeInfo = context.createTypeInformation(producedDataType);
RowDataOceanBaseDeserializationSchema deserializer =
RowDataOceanBaseDeserializationSchema.newBuilder()
DebeziumDeserializationSchema<RowData> deserializer =
RowDataDebeziumDeserializeSchema.newBuilder()
.setPhysicalRowType(physicalDataType)
.setMetadataConverters(metadataConverters)
.setResultTypeInfo(resultTypeInfo)
.setServerTimeZone(ZoneId.of(serverTimeZone))
.setServerTimeZone(
serverTimeZone == null
? ZoneId.systemDefault()
: ZoneId.of(serverTimeZone))
.setUserDefinedConverterFactory(
OceanBaseDeserializationConverterFactory.instance())
.build();
OceanBaseSource.Builder<RowData> builder =
OceanBaseSource.<RowData>builder()
.startupMode(startupMode)
.startupOptions(startupOptions)
.username(username)
.password(password)
.tenantName(tenantName)
@ -178,13 +190,14 @@ public class OceanBaseTableSource implements ScanTableSource, SupportsReadingMet
.configUrl(configUrl)
.workingMode(workingMode)
.obcdcProperties(obcdcProperties)
.debeziumProperties(debeziumProperties)
.deserializer(deserializer);
return SourceFunctionProvider.of(builder.build(), false);
}
protected OceanBaseMetadataConverter[] getMetadataConverters() {
protected MetadataConverter[] getMetadataConverters() {
if (metadataKeys.isEmpty()) {
return new OceanBaseMetadataConverter[0];
return new MetadataConverter[0];
}
return metadataKeys.stream()
.map(
@ -194,7 +207,7 @@ public class OceanBaseTableSource implements ScanTableSource, SupportsReadingMet
.findFirst()
.orElseThrow(IllegalStateException::new))
.map(OceanBaseReadableMetadata::getConverter)
.toArray(OceanBaseMetadataConverter[]::new);
.toArray(MetadataConverter[]::new);
}
@Override
@ -217,7 +230,7 @@ public class OceanBaseTableSource implements ScanTableSource, SupportsReadingMet
OceanBaseTableSource source =
new OceanBaseTableSource(
physicalSchema,
startupMode,
startupOptions,
username,
password,
tenantName,
@ -238,7 +251,8 @@ public class OceanBaseTableSource implements ScanTableSource, SupportsReadingMet
rsList,
configUrl,
workingMode,
obcdcProperties);
obcdcProperties,
debeziumProperties);
source.metadataKeys = metadataKeys;
source.producedDataType = producedDataType;
return source;
@ -254,7 +268,7 @@ public class OceanBaseTableSource implements ScanTableSource, SupportsReadingMet
}
OceanBaseTableSource that = (OceanBaseTableSource) o;
return Objects.equals(this.physicalSchema, that.physicalSchema)
&& Objects.equals(this.startupMode, that.startupMode)
&& Objects.equals(this.startupOptions, that.startupOptions)
&& Objects.equals(this.username, that.username)
&& Objects.equals(this.password, that.password)
&& Objects.equals(this.tenantName, that.tenantName)
@ -276,6 +290,7 @@ public class OceanBaseTableSource implements ScanTableSource, SupportsReadingMet
&& 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);
}
@ -284,7 +299,7 @@ public class OceanBaseTableSource implements ScanTableSource, SupportsReadingMet
public int hashCode() {
return Objects.hash(
physicalSchema,
startupMode,
startupOptions,
username,
password,
tenantName,
@ -306,6 +321,7 @@ public class OceanBaseTableSource implements ScanTableSource, SupportsReadingMet
configUrl,
workingMode,
obcdcProperties,
debeziumProperties,
producedDataType,
metadataKeys);
}

@ -17,12 +17,16 @@
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;
@ -35,19 +39,13 @@ 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<String> SCAN_STARTUP_MODE =
ConfigOptions.key("scan.startup.mode")
.stringType()
.noDefaultValue()
.withDescription(
"Optional startup mode for OceanBase CDC consumer, valid enumerations are "
+ "\"initial\", \"latest-offset\" or \"timestamp\"");
public static final ConfigOption<String> USERNAME =
ConfigOptions.key("username")
.stringType()
@ -124,9 +122,9 @@ public class OceanBaseTableSourceFactory implements DynamicTableSourceFactory {
public static final ConfigOption<String> JDBC_DRIVER =
ConfigOptions.key("jdbc.driver")
.stringType()
.defaultValue("com.mysql.jdbc.Driver")
.defaultValue("com.mysql.cj.jdbc.Driver")
.withDescription(
"JDBC driver class name, use 'com.mysql.jdbc.Driver' by default.");
"JDBC driver class name, use 'com.mysql.cj.jdbc.Driver' by default.");
public static final ConfigOption<String> LOG_PROXY_HOST =
ConfigOptions.key("logproxy.host")
@ -181,14 +179,16 @@ public class OceanBaseTableSourceFactory implements DynamicTableSourceFactory {
public DynamicTableSource createDynamicTableSource(Context context) {
final FactoryUtil.TableFactoryHelper helper =
FactoryUtil.createTableFactoryHelper(this, context);
helper.validateExcept(JdbcUrlUtils.PROPERTIES_PREFIX, OBCDC_PROPERTIES_PREFIX);
helper.validateExcept(
JdbcUrlUtils.PROPERTIES_PREFIX,
OBCDC_PROPERTIES_PREFIX,
DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX);
ResolvedSchema physicalSchema = context.getCatalogTable().getResolvedSchema();
ReadableConfig config = helper.getOptions();
validate(config);
StartupMode startupMode = StartupMode.getStartupMode(config.get(SCAN_STARTUP_MODE));
StartupOptions startupOptions = getStartupOptions(config);
String username = config.get(USERNAME);
String password = config.get(PASSWORD);
@ -205,6 +205,8 @@ public class OceanBaseTableSourceFactory implements DynamicTableSourceFactory {
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);
@ -217,7 +219,7 @@ public class OceanBaseTableSourceFactory implements DynamicTableSourceFactory {
return new OceanBaseTableSource(
physicalSchema,
startupMode,
startupOptions,
username,
password,
tenantName,
@ -238,7 +240,8 @@ public class OceanBaseTableSourceFactory implements DynamicTableSourceFactory {
rsList,
configUrl,
workingMode,
getProperties(context.getCatalogTable().getOptions(), OBCDC_PROPERTIES_PREFIX));
getProperties(context.getCatalogTable().getOptions(), OBCDC_PROPERTIES_PREFIX),
DebeziumOptions.getDebeziumProperties(context.getCatalogTable().getOptions()));
}
@Override
@ -249,28 +252,28 @@ public class OceanBaseTableSourceFactory implements DynamicTableSourceFactory {
@Override
public Set<ConfigOption<?>> requiredOptions() {
Set<ConfigOption<?>> options = new HashSet<>();
options.add(SCAN_STARTUP_MODE);
options.add(USERNAME);
options.add(PASSWORD);
options.add(TENANT_NAME);
options.add(LOG_PROXY_HOST);
options.add(LOG_PROXY_PORT);
options.add(HOSTNAME);
options.add(PORT);
return options;
}
@Override
public Set<ConfigOption<?>> optionalOptions() {
Set<ConfigOption<?>> 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(HOSTNAME);
options.add(PORT);
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);
@ -278,26 +281,54 @@ public class OceanBaseTableSourceFactory implements DynamicTableSourceFactory {
return options;
}
private void validate(ReadableConfig config) {
String startupMode = config.get(SCAN_STARTUP_MODE);
if (StartupMode.getStartupMode(startupMode).equals(StartupMode.INITIAL)) {
String compatibleMode =
Objects.requireNonNull(
config.get(COMPATIBLE_MODE),
"'compatible-mode' is required for 'initial' startup mode.");
String jdbcDriver =
Objects.requireNonNull(
config.get(JDBC_DRIVER),
"'jdbc.driver' is required for 'initial' startup mode.");
if (compatibleMode.equalsIgnoreCase("oracle")) {
if (!jdbcDriver.toLowerCase().contains("oceanbase")) {
throw new IllegalArgumentException(
"OceanBase JDBC driver is required for OceanBase Enterprise Edition.");
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);
}
Objects.requireNonNull(
config.get(CONFIG_URL),
"'config-url' is required for OceanBase Enterprise Edition.");
}
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);
}
}

@ -1,55 +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.table.api.ValidationException;
/** Startup modes for the OceanBase CDC Consumer. */
public enum StartupMode {
/**
* Performs an initial snapshot on the monitored database tables upon first startup, and
* continue to read the commit log.
*/
INITIAL,
/**
* Never to perform snapshot on the monitored database tables upon first startup, just read from
* the end of the commit log which means only have the changes since the connector was started.
*/
LATEST_OFFSET,
/**
* Never to perform snapshot on the monitored database tables upon first startup, and directly
* read commit log from the specified timestamp.
*/
TIMESTAMP;
public static StartupMode getStartupMode(String modeString) {
switch (modeString.toLowerCase()) {
case "initial":
return INITIAL;
case "latest-offset":
return LATEST_OFFSET;
case "timestamp":
return TIMESTAMP;
default:
throw new ValidationException(
String.format("Invalid startup mode '%s'.", modeString));
}
}
}

@ -15,15 +15,13 @@
* limitations under the License.
*/
package org.apache.flink.cdc.connectors.oceanbase.source;
package org.apache.flink.cdc.connectors.oceanbase.utils;
import java.io.Serializable;
/** Utils for OceanBase. */
public class OceanBaseUtils {
/**
* Runtime converter that converts objects of OceanBase into objects of Flink Table & SQL internal
* data structures.
*/
public interface OceanBaseDeserializationRuntimeConverter extends Serializable {
Object convert(Object object) throws Exception;
public static boolean isOceanBaseDriver(String driverClass) {
return "com.oceanbase.jdbc.Driver".equals(driverClass)
|| "com.alipay.oceanbase.jdbc.Driver".equals(driverClass);
}
}

@ -17,171 +17,124 @@
package org.apache.flink.cdc.connectors.oceanbase;
import org.apache.flink.runtime.minicluster.RpcServiceSharing;
import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
import org.apache.flink.table.planner.factories.TestValuesTableFactory;
import org.apache.flink.table.utils.LegacyRowResource;
import org.apache.flink.test.util.MiniClusterWithClientResource;
import org.apache.flink.util.TestLogger;
import org.awaitility.Awaitility;
import org.awaitility.core.ConditionTimeoutException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testcontainers.containers.GenericContainer;
import org.testcontainers.containers.output.Slf4jLogConsumer;
import org.testcontainers.containers.wait.strategy.Wait;
import org.testcontainers.lifecycle.Startables;
import org.junit.Rule;
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.TimeUnit;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
/** Basic class for testing OceanBase source. */
public class OceanBaseTestBase extends TestLogger {
private static final Logger LOG = LoggerFactory.getLogger(OceanBaseTestBase.class);
public abstract class OceanBaseTestBase extends TestLogger {
private static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)--.*$");
private static final Duration CONTAINER_STARTUP_TIMEOUT = Duration.ofMinutes(4);
public static final String NETWORK_MODE = "host";
// --------------------------------------------------------------------------------------------
// Attributes about host and port when network is on 'host' mode.
// --------------------------------------------------------------------------------------------
protected static int getObServerSqlPort() {
return 2881;
protected static final int DEFAULT_PARALLELISM = 4;
@Rule
public final MiniClusterWithClientResource miniClusterResource =
new MiniClusterWithClientResource(
new MiniClusterResourceConfiguration.Builder()
.setNumberTaskManagers(1)
.setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM)
.setRpcServiceSharing(RpcServiceSharing.DEDICATED)
.withHaLeadershipControl()
.build());
@ClassRule public static LegacyRowResource usesLegacyRows = LegacyRowResource.INSTANCE;
protected final String compatibleMode;
protected final String username;
protected final String password;
protected final String hostname;
protected final int port;
protected final String logProxyHost;
protected final int logProxyPort;
protected final String tenant;
public OceanBaseTestBase(
String compatibleMode,
String username,
String password,
String hostname,
int port,
String logProxyHost,
int logProxyPort,
String tenant) {
this.compatibleMode = compatibleMode;
this.username = username;
this.password = password;
this.hostname = hostname;
this.port = port;
this.logProxyHost = logProxyHost;
this.logProxyPort = logProxyPort;
this.tenant = tenant;
}
protected static int getLogProxyPort() {
return 2983;
protected String commonOptionsString() {
return String.format(
" 'connector' = 'oceanbase-cdc', "
+ " 'username' = '%s', "
+ " 'password' = '%s', "
+ " 'hostname' = '%s', "
+ " 'port' = '%s', "
+ " 'compatible-mode' = '%s'",
username, password, hostname, port, compatibleMode);
}
public static String getRsList() {
return "127.0.0.1:2882:2881";
protected String logProxyOptionsString() {
return String.format(
" 'working-mode' = 'memory',"
+ " 'tenant-name' = '%s',"
+ " 'logproxy.host' = '%s',"
+ " 'logproxy.port' = '%s'",
tenant, logProxyHost, logProxyPort);
}
// --------------------------------------------------------------------------------------------
// Attributes about user.
// From OceanBase 4.0.0.0 CE, we can only fetch the commit log of non-sys tenant.
// --------------------------------------------------------------------------------------------
public static final String OB_SYS_PASSWORD = "pswd";
protected static String getTenant() {
return "test";
protected String initialOptionsString() {
return " 'scan.startup.mode' = 'initial', "
+ commonOptionsString()
+ ", "
+ logProxyOptionsString();
}
protected static String getUsername() {
return "root@" + getTenant();
protected String snapshotOptionsString() {
return " 'scan.startup.mode' = 'snapshot', " + commonOptionsString();
}
protected static String getPassword() {
return "test";
}
protected abstract Connection getJdbcConnection() throws SQLException;
@ClassRule
public static final GenericContainer<?> OB_SERVER =
new GenericContainer<>("oceanbase/oceanbase-ce:4.2.0.0")
.withNetworkMode(NETWORK_MODE)
.withEnv("MODE", "slim")
.withEnv("OB_ROOT_PASSWORD", OB_SYS_PASSWORD)
.waitingFor(Wait.forLogMessage(".*boot success!.*", 1))
.withStartupTimeout(CONTAINER_STARTUP_TIMEOUT)
.withLogConsumer(new Slf4jLogConsumer(LOG));
@ClassRule
public static final GenericContainer<?> LOG_PROXY =
new GenericContainer<>("whhe/oblogproxy:1.1.3_4x")
.withNetworkMode(NETWORK_MODE)
.withEnv("OB_SYS_PASSWORD", OB_SYS_PASSWORD)
.waitingFor(Wait.forLogMessage(".*boot success!.*", 1))
.withStartupTimeout(CONTAINER_STARTUP_TIMEOUT)
.withLogConsumer(new Slf4jLogConsumer(LOG));
@BeforeClass
public static void startContainers() {
LOG.info("Starting containers...");
Startables.deepStart(Stream.of(OB_SERVER, LOG_PROXY)).join();
LOG.info("Containers are started.");
try (Connection connection =
DriverManager.getConnection(getJdbcUrl(""), getUsername(), "");
protected void setGlobalTimeZone(String serverTimeZone) throws SQLException {
try (Connection connection = getJdbcConnection();
Statement statement = connection.createStatement()) {
statement.execute(String.format("ALTER USER root IDENTIFIED BY '%s'", getPassword()));
} catch (SQLException e) {
LOG.error("Set test user password failed.", e);
throw new RuntimeException(e);
}
}
@AfterClass
public static void stopContainers() {
LOG.info("Stopping containers...");
Stream.of(OB_SERVER, LOG_PROXY).forEach(GenericContainer::stop);
LOG.info("Containers are stopped.");
}
public static String getJdbcUrl(String databaseName) {
return "jdbc:mysql://"
+ OB_SERVER.getHost()
+ ":"
+ getObServerSqlPort()
+ "/"
+ databaseName
+ "?useSSL=false";
}
protected static Connection getJdbcConnection(String databaseName) throws SQLException {
return DriverManager.getConnection(getJdbcUrl(databaseName), getUsername(), getPassword());
}
private static void dropTestDatabase(Connection connection, String databaseName) {
try {
Awaitility.await(String.format("Dropping database %s", databaseName))
.atMost(120, TimeUnit.SECONDS)
.until(
() -> {
try {
String sql =
String.format(
"DROP DATABASE IF EXISTS %s", databaseName);
connection.createStatement().execute(sql);
return true;
} catch (SQLException e) {
LOG.warn(
String.format(
"DROP DATABASE %s failed: {}", databaseName),
e.getMessage());
return false;
}
});
} catch (ConditionTimeoutException e) {
throw new IllegalStateException("Failed to drop test database", e);
statement.execute(String.format("SET GLOBAL time_zone = '%s';", serverTimeZone));
}
}
protected void initializeTable(String sqlFile) {
final String ddlFile = String.format("ddl/%s.sql", sqlFile);
final String ddlFile = String.format("ddl/%s/%s.sql", compatibleMode, sqlFile);
final URL ddlTestFile = getClass().getClassLoader().getResource(ddlFile);
assertNotNull("Cannot locate " + ddlFile, ddlTestFile);
try (Connection connection = getJdbcConnection("");
try (Connection connection = getJdbcConnection();
Statement statement = connection.createStatement()) {
dropTestDatabase(connection, sqlFile);
final List<String> statements =
Arrays.stream(
Files.readAllLines(Paths.get(ddlTestFile.toURI())).stream()
@ -203,4 +156,29 @@ public class OceanBaseTestBase extends TestLogger {
throw new RuntimeException(e);
}
}
public static void waitForSinkSize(String sinkName, int expectedSize)
throws InterruptedException {
while (sinkSize(sinkName) < expectedSize) {
Thread.sleep(100);
}
}
public static int sinkSize(String sinkName) {
synchronized (TestValuesTableFactory.class) {
try {
return TestValuesTableFactory.getRawResults(sinkName).size();
} catch (IllegalArgumentException e) {
// job is not started yet
return 0;
}
}
}
public static void assertContainsInAnyOrder(List<String> expected, List<String> actual) {
assertTrue(expected != null && actual != null);
assertTrue(
String.format("expected: %s, actual: %s", expected, actual),
actual.containsAll(expected));
}
}

@ -18,62 +18,141 @@
package org.apache.flink.cdc.connectors.oceanbase.table;
import org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestBase;
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.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.table.utils.LegacyRowResource;
import org.apache.flink.test.util.MiniClusterWithClientResource;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testcontainers.containers.GenericContainer;
import org.testcontainers.containers.output.Slf4jLogConsumer;
import org.testcontainers.containers.wait.strategy.Wait;
import org.testcontainers.lifecycle.Startables;
import org.testcontainers.utility.MountableFile;
import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.SQLException;
import java.sql.Statement;
import java.time.Duration;
import java.time.ZoneId;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.stream.Stream;
import static org.junit.Assert.assertTrue;
/** Integration tests for OceanBase MySQL mode table source. */
@RunWith(Parameterized.class)
public class OceanBaseMySQLModeITCase extends OceanBaseTestBase {
/** Integration tests for OceanBase change stream event SQL source. */
public class OceanBaseConnectorITCase extends OceanBaseTestBase {
private static final int DEFAULT_PARALLELISM = 2;
private static final Logger LOG = LoggerFactory.getLogger(OceanBaseMySQLModeITCase.class);
private final StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment()
.setParallelism(DEFAULT_PARALLELISM);
StreamExecutionEnvironment.getExecutionEnvironment();
private final StreamTableEnvironment tEnv =
StreamTableEnvironment.create(
env, EnvironmentSettings.newInstance().inStreamingMode().build());
@ClassRule public static LegacyRowResource usesLegacyRows = LegacyRowResource.INSTANCE;
private static final String NETWORK_MODE = "host";
private static final String OB_SYS_PASSWORD = "123456";
@ClassRule
public static final GenericContainer<?> OB_SERVER =
new GenericContainer<>("oceanbase/oceanbase-ce:4.2.0.0")
.withNetworkMode(NETWORK_MODE)
.withEnv("MODE", "slim")
.withEnv("OB_ROOT_PASSWORD", OB_SYS_PASSWORD)
.withEnv("OB_DATAFILE_SIZE", "1G")
.withEnv("OB_LOG_DISK_SIZE", "4G")
.withCopyFileToContainer(
MountableFile.forClasspathResource("ddl/mysql/docker_init.sql"),
"/root/boot/init.d/init.sql")
.waitingFor(Wait.forLogMessage(".*boot success!.*", 1))
.withStartupTimeout(Duration.ofMinutes(4))
.withLogConsumer(new Slf4jLogConsumer(LOG));
@ClassRule
public static final GenericContainer<?> LOG_PROXY =
new GenericContainer<>("whhe/oblogproxy:1.1.3_4x")
.withNetworkMode(NETWORK_MODE)
.withEnv("OB_SYS_PASSWORD", OB_SYS_PASSWORD)
.waitingFor(Wait.forLogMessage(".*boot success!.*", 1))
.withStartupTimeout(Duration.ofMinutes(1))
.withLogConsumer(new Slf4jLogConsumer(LOG));
@BeforeClass
public static void startContainers() {
LOG.info("Starting containers...");
Startables.deepStart(Stream.of(OB_SERVER, LOG_PROXY)).join();
LOG.info("Containers are started.");
}
@Rule
public final MiniClusterWithClientResource miniClusterResource =
new MiniClusterWithClientResource(
new MiniClusterResourceConfiguration.Builder()
.setNumberTaskManagers(1)
.setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM)
.setRpcServiceSharing(RpcServiceSharing.DEDICATED)
.withHaLeadershipControl()
.build());
@AfterClass
public static void stopContainers() {
LOG.info("Stopping containers...");
Stream.of(OB_SERVER, LOG_PROXY).forEach(GenericContainer::stop);
LOG.info("Containers are stopped.");
}
@Before
public void before() {
TestValuesTableFactory.clearAllData();
env.enableCheckpointing(1000);
env.getCheckpointConfig().setCheckpointingMode(CheckpointingMode.AT_LEAST_ONCE);
env.getCheckpointConfig().setMinPauseBetweenCheckpoints(500);
}
private final String rsList;
public OceanBaseMySQLModeITCase(
String username,
String password,
String hostname,
int port,
String logProxyHost,
int logProxyPort,
String tenant,
String rsList) {
super("mysql", username, password, hostname, port, logProxyHost, logProxyPort, tenant);
this.rsList = rsList;
}
@Parameterized.Parameters
public static List<Object[]> parameters() {
return Collections.singletonList(
new Object[] {
"root@test",
"123456",
"127.0.0.1",
2881,
"127.0.0.1",
2983,
"test",
"127.0.0.1:2882:2881"
});
}
@Override
protected String logProxyOptionsString() {
return super.logProxyOptionsString()
+ " , "
+ String.format(" 'rootserver-list' = '%s'", rsList);
}
@Override
protected Connection getJdbcConnection() throws SQLException {
return DriverManager.getConnection(
"jdbc:mysql://" + hostname + ":" + port + "/?useSSL=false", username, password);
}
@Test
public void testTableList() throws Exception {
initializeTable("inventory");
@ -87,29 +166,11 @@ public class OceanBaseConnectorITCase extends OceanBaseTestBase {
+ " weight DECIMAL(20, 10),"
+ " PRIMARY KEY (`id`) NOT ENFORCED"
+ ") WITH ("
+ " 'connector' = 'oceanbase-cdc',"
+ " 'scan.startup.mode' = 'initial',"
+ " 'username' = '%s',"
+ " 'password' = '%s',"
+ " 'tenant-name' = '%s',"
+ " 'table-list' = '%s',"
+ " 'hostname' = '%s',"
+ " 'port' = '%s',"
+ " 'logproxy.host' = '%s',"
+ " 'logproxy.port' = '%s',"
+ " 'rootserver-list' = '%s',"
+ " 'working-mode' = 'memory',"
+ " 'jdbc.properties.useSSL' = 'false'"
+ initialOptionsString()
+ ", "
+ " 'table-list' = '%s'"
+ ")",
getUsername(),
getPassword(),
getTenant(),
"inventory.products",
OB_SERVER.getHost(),
getObServerSqlPort(),
LOG_PROXY.getHost(),
getLogProxyPort(),
getRsList());
"inventory.products");
String sinkDDL =
"CREATE TABLE sink ("
@ -132,19 +193,19 @@ public class OceanBaseConnectorITCase extends OceanBaseTestBase {
waitForSinkSize("sink", 9);
int snapshotSize = sinkSize("sink");
try (Connection connection = getJdbcConnection("inventory");
try (Connection connection = getJdbcConnection();
Statement statement = connection.createStatement()) {
statement.execute(
"UPDATE products SET description='18oz carpenter hammer' WHERE id=106;");
statement.execute("UPDATE products SET weight='5.1' WHERE id=107;");
"UPDATE inventory.products SET description='18oz carpenter hammer' WHERE id=106;");
statement.execute("UPDATE inventory.products SET weight='5.1' WHERE id=107;");
statement.execute(
"INSERT INTO products VALUES (default,'jacket','water resistent white wind breaker',0.2);"); // 110
"INSERT INTO inventory.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);");
"INSERT INTO inventory.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;");
"UPDATE inventory.products SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110;");
statement.execute("UPDATE inventory.products SET weight='5.17' WHERE id=111;");
statement.execute("DELETE FROM inventory.products WHERE id=111;");
}
waitForSinkSize("sink", snapshotSize + 7);
@ -197,7 +258,7 @@ public class OceanBaseConnectorITCase extends OceanBaseTestBase {
@Test
public void testMetadataColumns() throws Exception {
initializeTable("inventory_meta");
initializeTable("inventory");
String sourceDDL =
String.format(
@ -211,31 +272,13 @@ public class OceanBaseConnectorITCase extends OceanBaseTestBase {
+ " weight DECIMAL(20, 10),"
+ " PRIMARY KEY (`id`) NOT ENFORCED"
+ ") WITH ("
+ " 'connector' = 'oceanbase-cdc',"
+ " 'scan.startup.mode' = 'initial',"
+ " 'username' = '%s',"
+ " 'password' = '%s',"
+ " 'tenant-name' = '%s',"
+ initialOptionsString()
+ ","
+ " 'database-name' = '%s',"
+ " 'table-name' = '%s',"
+ " 'hostname' = '%s',"
+ " 'port' = '%s',"
+ " 'logproxy.host' = '%s',"
+ " 'logproxy.port' = '%s',"
+ " 'rootserver-list' = '%s',"
+ " 'working-mode' = 'memory',"
+ " 'jdbc.properties.useSSL' = 'false'"
+ " 'table-name' = '%s'"
+ ")",
getUsername(),
getPassword(),
getTenant(),
"^inventory_meta$",
"^products$",
OB_SERVER.getHost(),
getObServerSqlPort(),
LOG_PROXY.getHost(),
getLogProxyPort(),
getRsList());
"^inventory$",
"^products$");
String sinkDDL =
"CREATE TABLE sink ("
@ -261,10 +304,10 @@ public class OceanBaseConnectorITCase extends OceanBaseTestBase {
waitForSinkSize("sink", 9);
int snapshotSize = sinkSize("sink");
try (Connection connection = getJdbcConnection("inventory_meta");
try (Connection connection = getJdbcConnection();
Statement statement = connection.createStatement()) {
statement.execute(
"UPDATE products SET description='18oz carpenter hammer' WHERE id=106;");
"UPDATE inventory.products SET description='18oz carpenter hammer' WHERE id=106;");
}
waitForSinkSize("sink", snapshotSize + 1);
@ -272,35 +315,35 @@ public class OceanBaseConnectorITCase extends OceanBaseTestBase {
List<String> expected =
Arrays.asList(
"+I("
+ getTenant()
+ ",inventory_meta,products,101,scooter,Small 2-wheel scooter,3.1400000000)",
+ tenant
+ ",inventory,products,101,scooter,Small 2-wheel scooter,3.1400000000)",
"+I("
+ getTenant()
+ ",inventory_meta,products,102,car battery,12V car battery,8.1000000000)",
+ tenant
+ ",inventory,products,102,car battery,12V car battery,8.1000000000)",
"+I("
+ getTenant()
+ ",inventory_meta,products,103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8000000000)",
+ tenant
+ ",inventory,products,103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8000000000)",
"+I("
+ getTenant()
+ ",inventory_meta,products,104,hammer,12oz carpenter's hammer,0.7500000000)",
+ tenant
+ ",inventory,products,104,hammer,12oz carpenter's hammer,0.7500000000)",
"+I("
+ getTenant()
+ ",inventory_meta,products,105,hammer,14oz carpenter's hammer,0.8750000000)",
+ tenant
+ ",inventory,products,105,hammer,14oz carpenter's hammer,0.8750000000)",
"+I("
+ getTenant()
+ ",inventory_meta,products,106,hammer,16oz carpenter's hammer,1.0000000000)",
+ tenant
+ ",inventory,products,106,hammer,16oz carpenter's hammer,1.0000000000)",
"+I("
+ getTenant()
+ ",inventory_meta,products,107,rocks,box of assorted rocks,5.3000000000)",
+ tenant
+ ",inventory,products,107,rocks,box of assorted rocks,5.3000000000)",
"+I("
+ getTenant()
+ ",inventory_meta,products,108,jacket,water resistent black wind breaker,0.1000000000)",
+ tenant
+ ",inventory,products,108,jacket,water resistent black wind breaker,0.1000000000)",
"+I("
+ getTenant()
+ ",inventory_meta,products,109,spare tire,24 inch spare tire,22.2000000000)",
+ tenant
+ ",inventory,products,109,spare tire,24 inch spare tire,22.2000000000)",
"+U("
+ getTenant()
+ ",inventory_meta,products,106,hammer,18oz carpenter hammer,1.0000000000)");
+ tenant
+ ",inventory,products,106,hammer,18oz carpenter hammer,1.0000000000)");
List<String> actual = TestValuesTableFactory.getRawResults("sink");
assertContainsInAnyOrder(expected, actual);
result.getJobClient().get().cancel().get();
@ -309,11 +352,9 @@ public class OceanBaseConnectorITCase extends OceanBaseTestBase {
@Test
public void testAllDataTypes() throws Exception {
String serverTimeZone = "+00:00";
try (Connection connection = getJdbcConnection("");
Statement statement = connection.createStatement()) {
statement.execute(String.format("SET GLOBAL time_zone = '%s';", serverTimeZone));
}
setGlobalTimeZone(serverTimeZone);
tEnv.getConfig().setLocalTimeZone(ZoneId.of(serverTimeZone));
initializeTable("column_type_test");
String sourceDDL =
String.format(
@ -343,9 +384,9 @@ public class OceanBaseConnectorITCase extends OceanBaseTestBase {
+ " time_c TIME(0),\n"
+ " datetime3_c TIMESTAMP(3),\n"
+ " datetime6_c TIMESTAMP(6),\n"
+ " timestamp_c TIMESTAMP_LTZ,\n"
+ " timestamp3_c TIMESTAMP_LTZ(3),\n"
+ " timestamp6_c TIMESTAMP_LTZ(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"
@ -361,34 +402,15 @@ public class OceanBaseConnectorITCase extends OceanBaseTestBase {
+ " json_c STRING,\n"
+ " primary key (`id`) not enforced"
+ ") WITH ("
+ " 'connector' = 'oceanbase-cdc',"
+ " 'scan.startup.mode' = 'initial',"
+ " 'username' = '%s',"
+ " 'password' = '%s',"
+ " 'tenant-name' = '%s',"
+ initialOptionsString()
+ ","
+ " 'database-name' = '%s',"
+ " 'table-name' = '%s',"
+ " 'server-time-zone' = '%s',"
+ " 'hostname' = '%s',"
+ " 'port' = '%s',"
+ " 'logproxy.host' = '%s',"
+ " 'logproxy.port' = '%s',"
+ " 'rootserver-list' = '%s',"
+ " 'working-mode' = 'memory',"
+ " 'jdbc.properties.useSSL' = 'false',"
+ " 'obcdc.properties.sort_trans_participants' = '1'"
+ " 'server-time-zone' = '%s'"
+ ")",
getUsername(),
getPassword(),
getTenant(),
"^column_type_test$",
"^full_types$",
serverTimeZone,
OB_SERVER.getHost(),
getObServerSqlPort(),
LOG_PROXY.getHost(),
getLogProxyPort(),
getRsList());
serverTimeZone);
String sinkDDL =
"CREATE TABLE sink ("
+ " `id` INT NOT NULL,\n"
@ -421,7 +443,7 @@ public class OceanBaseConnectorITCase extends OceanBaseTestBase {
+ " timestamp6_c TIMESTAMP(6),\n"
+ " char_c CHAR(3),\n"
+ " varchar_c VARCHAR(255),\n"
+ " file_uuid BINARY(16),\n"
+ " file_uuid STRING,\n"
+ " bit_c BINARY(8),\n"
+ " text_c STRING,\n"
+ " tiny_blob_c BYTES,\n"
@ -441,23 +463,66 @@ public class OceanBaseConnectorITCase extends OceanBaseTestBase {
tEnv.executeSql(sourceDDL);
tEnv.executeSql(sinkDDL);
TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM ob_source");
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("column_type_test");
try (Connection connection = getJdbcConnection();
Statement statement = connection.createStatement()) {
statement.execute(
"UPDATE full_types SET timestamp_c = '2020-07-17 18:33:22' WHERE id=1;");
"UPDATE column_type_test.full_types SET timestamp_c = '2020-07-17 18:33:22' WHERE id=1;");
}
waitForSinkSize("sink", snapshotSize + 1);
List<String> 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,[101, 26, -17, -65, -67, 8, 57, 15, 72, -17, -65, -67, -17, -65, -67, -17, -65, -67, 54, -17, -65, -67, 62, 123, 116, 0],[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,[101, 26, -17, -65, -67, 8, 57, 15, 72, -17, -65, -67, -17, -65, -67, -17, -65, -67, 54, -17, -65, -67, 62, 123, 116, 0],[4, 4, 4, 4, 4, 4, 4, 4],text,[16],[16],[16],[16],2022,[a, b],red,{\"key1\": \"value1\"})");
"+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<String> actual = TestValuesTableFactory.getRawResults("sink");
assertContainsInAnyOrder(expected, actual);
@ -475,11 +540,9 @@ public class OceanBaseConnectorITCase extends OceanBaseTestBase {
}
public void testTimeDataTypes(String serverTimeZone) throws Exception {
try (Connection connection = getJdbcConnection("");
Statement statement = connection.createStatement()) {
statement.execute(String.format("SET GLOBAL time_zone = '%s';", serverTimeZone));
}
setGlobalTimeZone(serverTimeZone);
tEnv.getConfig().setLocalTimeZone(ZoneId.of(serverTimeZone));
initializeTable("column_type_test");
String sourceDDL =
String.format(
@ -489,36 +552,18 @@ public class OceanBaseConnectorITCase extends OceanBaseTestBase {
+ " time_c TIME(0),\n"
+ " datetime3_c TIMESTAMP(3),\n"
+ " datetime6_c TIMESTAMP(6),\n"
+ " timestamp_c TIMESTAMP_LTZ,\n"
+ " timestamp_c TIMESTAMP,\n"
+ " primary key (`id`) not enforced"
+ ") WITH ("
+ " 'connector' = 'oceanbase-cdc',"
+ " 'scan.startup.mode' = 'initial',"
+ " 'username' = '%s',"
+ " 'password' = '%s',"
+ " 'tenant-name' = '%s',"
+ initialOptionsString()
+ ","
+ " 'database-name' = '%s',"
+ " 'table-name' = '%s',"
+ " 'server-time-zone' = '%s',"
+ " 'hostname' = '%s',"
+ " 'port' = '%s',"
+ " 'logproxy.host' = '%s',"
+ " 'logproxy.port' = '%s',"
+ " 'rootserver-list' = '%s',"
+ " 'working-mode' = 'memory',"
+ " 'jdbc.properties.useSSL' = 'false'"
+ " 'server-time-zone' = '%s'"
+ ")",
getUsername(),
getPassword(),
getTenant(),
"column_type_test",
"full_types",
serverTimeZone,
OB_SERVER.getHost(),
getObServerSqlPort(),
LOG_PROXY.getHost(),
getLogProxyPort(),
getRsList());
serverTimeZone);
String sinkDDL =
"CREATE TABLE sink ("
@ -546,10 +591,10 @@ public class OceanBaseConnectorITCase extends OceanBaseTestBase {
waitForSinkSize("sink", 1);
int snapshotSize = sinkSize("sink");
try (Connection connection = getJdbcConnection("column_type_test");
try (Connection connection = getJdbcConnection();
Statement statement = connection.createStatement()) {
statement.execute(
"UPDATE full_types SET timestamp_c = '2020-07-17 18:33:22' WHERE id=1;");
"UPDATE column_type_test.full_types SET timestamp_c = '2020-07-17 18:33:22' WHERE id=1;");
}
waitForSinkSize("sink", snapshotSize + 1);
@ -564,28 +609,57 @@ public class OceanBaseConnectorITCase extends OceanBaseTestBase {
result.getJobClient().get().cancel().get();
}
private static void waitForSinkSize(String sinkName, int expectedSize)
throws InterruptedException {
while (sinkSize(sinkName) < expectedSize) {
Thread.sleep(100);
}
}
@Test
public void testSnapshotOnly() throws Exception {
initializeTable("inventory");
private static int sinkSize(String sinkName) {
synchronized (TestValuesTableFactory.class) {
try {
return TestValuesTableFactory.getRawResults(sinkName).size();
} catch (IllegalArgumentException e) {
// job is not started yet
return 0;
}
}
}
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'"
+ ")",
"inventory.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'"
+ ")";
public static void assertContainsInAnyOrder(List<String> expected, List<String> actual) {
assertTrue(expected != null && actual != null);
assertTrue(
String.format("expected: %s, actual: %s", expected, actual),
actual.containsAll(expected));
tEnv.executeSql(sourceDDL);
tEnv.executeSql(sinkDDL);
TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM ob_source");
waitForSinkSize("sink", 9);
List<String> 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<String> actual = TestValuesTableFactory.getRawResults("sink");
assertContainsInAnyOrder(expected, actual);
}
}

@ -0,0 +1,267 @@
/*
* 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.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.Ignore;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.SQLException;
import java.sql.Statement;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
/** Integration tests for OceanBase Oracle mode table source. */
@Ignore("Test ignored before oceanbase-xe docker image is available")
@RunWith(Parameterized.class)
public class OceanBaseOracleModeITCase extends OceanBaseTestBase {
private final StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
private final StreamTableEnvironment tEnv =
StreamTableEnvironment.create(
env, EnvironmentSettings.newInstance().inStreamingMode().build());
private final String schema;
private final String configUrl;
public OceanBaseOracleModeITCase(
String username,
String password,
String hostname,
int port,
String logProxyHost,
int logProxyPort,
String tenant,
String schema,
String configUrl) {
super("oracle", username, password, hostname, port, logProxyHost, logProxyPort, tenant);
this.schema = schema;
this.configUrl = configUrl;
}
@Parameterized.Parameters
public static List<Object[]> parameters() {
return Collections.singletonList(
new Object[] {
"SYS@test",
"123456",
"127.0.0.1",
2881,
"127.0.0.1",
2983,
"test",
"SYS",
"http://127.0.0.1:8080/services?Action=ObRootServiceInfo&ObCluster=obcluster"
});
}
@Override
protected String commonOptionsString() {
return super.commonOptionsString() + " , " + " 'jdbc.driver' = 'com.oceanbase.jdbc.Driver'";
}
@Override
protected String logProxyOptionsString() {
return super.logProxyOptionsString()
+ " , "
+ String.format(" 'config-url' = '%s'", configUrl);
}
@Override
protected Connection getJdbcConnection() throws SQLException {
return DriverManager.getConnection(
"jdbc:oceanbase://" + hostname + ":" + port + "/" + schema, username, password);
}
@Test
public void testAllDataTypes() throws Exception {
initializeTable("column_type_test");
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<String> 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<String> actual = TestValuesTableFactory.getRawResults("sink");
assertContainsInAnyOrder(expected, actual);
result.getJobClient().get().cancel().get();
}
}

@ -17,6 +17,7 @@
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;
@ -85,7 +86,7 @@ public class OceanBaseTableFactoryTest {
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.jdbc.Driver";
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";
@ -104,7 +105,7 @@ public class OceanBaseTableFactoryTest {
OceanBaseTableSource expectedSource =
new OceanBaseTableSource(
SCHEMA,
StartupMode.LATEST_OFFSET,
StartupOptions.latest(),
USERNAME,
PASSWORD,
TENANT_NAME,
@ -113,8 +114,8 @@ public class OceanBaseTableFactoryTest {
TABLE_LIST,
SERVER_TIME_ZONE,
Duration.parse("PT" + CONNECT_TIMEOUT),
null,
null,
HOSTNAME,
PORT,
COMPATIBLE_MODE,
DRIVER_CLASS,
new Properties(),
@ -125,6 +126,7 @@ public class OceanBaseTableFactoryTest {
RS_LIST,
null,
WORKING_MODE,
new Properties(),
new Properties());
assertEquals(expectedSource, actualSource);
}
@ -136,8 +138,6 @@ public class OceanBaseTableFactoryTest {
options.put("database-name", DATABASE_NAME);
options.put("table-name", TABLE_NAME);
options.put("table-list", TABLE_LIST);
options.put("hostname", HOSTNAME);
options.put("port", String.valueOf(PORT));
options.put("compatible-mode", COMPATIBLE_MODE);
options.put("jdbc.driver", DRIVER_CLASS);
options.put("logproxy.client.id", LOG_PROXY_CLIENT_ID);
@ -147,7 +147,7 @@ public class OceanBaseTableFactoryTest {
OceanBaseTableSource expectedSource =
new OceanBaseTableSource(
SCHEMA,
StartupMode.INITIAL,
StartupOptions.initial(),
USERNAME,
PASSWORD,
TENANT_NAME,
@ -168,6 +168,7 @@ public class OceanBaseTableFactoryTest {
RS_LIST,
null,
WORKING_MODE,
new Properties(),
new Properties());
assertEquals(expectedSource, actualSource);
}
@ -190,7 +191,7 @@ public class OceanBaseTableFactoryTest {
OceanBaseTableSource expectedSource =
new OceanBaseTableSource(
SCHEMA_WITH_METADATA,
StartupMode.LATEST_OFFSET,
StartupOptions.latest(),
USERNAME,
PASSWORD,
TENANT_NAME,
@ -199,8 +200,8 @@ public class OceanBaseTableFactoryTest {
TABLE_LIST,
SERVER_TIME_ZONE,
Duration.parse("PT" + CONNECT_TIMEOUT),
null,
null,
HOSTNAME,
PORT,
COMPATIBLE_MODE,
DRIVER_CLASS,
new Properties(),
@ -211,6 +212,7 @@ public class OceanBaseTableFactoryTest {
RS_LIST,
null,
WORKING_MODE,
new Properties(),
new Properties());
expectedSource.producedDataType = SCHEMA_WITH_METADATA.toSourceRowDataType();
expectedSource.metadataKeys =
@ -240,6 +242,8 @@ public class OceanBaseTableFactoryTest {
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));

@ -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.
-- ----------------------------------------------------------------------------------------------------------------
-- DATABASE: inventory_meta
-- ----------------------------------------------------------------------------------------------------------------
CREATE DATABASE inventory_meta;
USE inventory_meta;
-- Create and populate our products using a single insert with many rows
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);

@ -17,9 +17,10 @@
-- DATABASE: column_type_test
-- ----------------------------------------------------------------------------------------------------------------
CREATE DATABASE column_type_test;
CREATE DATABASE IF NOT EXISTS column_type_test;
USE column_type_test;
DROP TABLE IF EXISTS full_types;
CREATE TABLE full_types
(
id INT AUTO_INCREMENT NOT NULL,

@ -0,0 +1,17 @@
-- 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.
-- Set the root user password of test tenant
ALTER USER root IDENTIFIED BY '123456';

@ -17,10 +17,10 @@
-- DATABASE: inventory
-- ----------------------------------------------------------------------------------------------------------------
CREATE DATABASE inventory;
CREATE DATABASE IF NOT EXISTS inventory;
USE inventory;
-- Create and populate our products using a single insert with many rows
DROP TABLE IF EXISTS products;
CREATE TABLE products
(
id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,

@ -0,0 +1,70 @@
-- 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')
);

@ -55,6 +55,7 @@ limitations under the License.
<include>io.debezium:debezium-api</include>
<include>io.debezium:debezium-embedded</include>
<include>io.debezium:debezium-core</include>
<include>org.apache.flink:flink-cdc-base</include>
<include>org.apache.flink:flink-connector-debezium</include>
<include>org.apache.flink:flink-connector-oceanbase-cdc</include>
<include>com.oceanbase:*</include>

@ -85,7 +85,6 @@ limitations under the License.
<slf4j.version>1.7.36</slf4j.version>
<log4j.version>2.17.1</log4j.version>
<spotless.version>2.4.2</spotless.version>
<oblogclient.version>1.1.0</oblogclient.version>
<commons-lang3.version>3.12.0</commons-lang3.version>
<json-path.version>2.7.0</json-path.version>
<junit5.version>5.10.1</junit5.version>

Loading…
Cancel
Save