From 3eda5b9b1ec27fc266c03fe9a4efdd27c551bd36 Mon Sep 17 00:00:00 2001 From: He Wang Date: Mon, 11 Mar 2024 21:24:31 +0800 Subject: [PATCH] [FLINK-34639] Support DebeziumDeserializationSchema in OceanBase source connector --- .../cdc-connectors/oceanbase-cdc.md | 120 +++- .../flink-connector-oceanbase-cdc/pom.xml | 13 +- .../connectors/oceanbase/OceanBaseSource.java | 151 +++-- .../oceanbase/source/OceanBaseConnection.java | 166 ++++- ...anBaseDeserializationConverterFactory.java | 74 ++ .../source/OceanBaseRichSourceFunction.java | 287 +++++--- .../oceanbase/source/OceanBaseSchema.java | 80 +++ .../source/OceanBaseValueConverters.java | 509 ++++++++++++++ ...RowDataOceanBaseDeserializationSchema.java | 632 ------------------ .../config/OceanBaseConnectorConfig.java | 93 +++ .../schema/OceanBaseDatabaseSchema.java | 53 ++ .../source/schema/OceanBaseSourceInfo.java | 101 +++ .../OceanBaseSourceInfoStructMaker.java | 68 ++ .../OceanBaseAppendMetadataCollector.java | 57 -- .../table/OceanBaseDeserializationSchema.java | 37 - .../table/OceanBaseMetadataConverter.java | 29 - .../table/OceanBaseReadableMetadata.java | 67 +- .../oceanbase/table/OceanBaseRecord.java | 133 ---- .../oceanbase/table/OceanBaseTableSource.java | 62 +- .../table/OceanBaseTableSourceFactory.java | 107 +-- .../oceanbase/table/StartupMode.java | 55 -- .../OceanBaseUtils.java} | 16 +- .../oceanbase/OceanBaseTestBase.java | 226 +++---- ...ase.java => OceanBaseMySQLModeITCase.java} | 446 ++++++------ .../table/OceanBaseOracleModeITCase.java | 267 ++++++++ .../table/OceanBaseTableFactoryTest.java | 22 +- .../src/test/resources/ddl/inventory_meta.sql | 42 -- .../ddl/{ => mysql}/column_type_test.sql | 3 +- .../test/resources/ddl/mysql/docker_init.sql | 17 + .../resources/ddl/{ => mysql}/inventory.sql | 4 +- .../resources/ddl/oracle/column_type_test.sql | 70 ++ .../flink-sql-connector-oceanbase-cdc/pom.xml | 1 + 32 files changed, 2412 insertions(+), 1596 deletions(-) create mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseDeserializationConverterFactory.java create mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseSchema.java create mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseValueConverters.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/RowDataOceanBaseDeserializationSchema.java create mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/config/OceanBaseConnectorConfig.java create mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseDatabaseSchema.java create mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseSourceInfo.java create mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseSourceInfoStructMaker.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseAppendMetadataCollector.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseDeserializationSchema.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMetadataConverter.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseRecord.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/StartupMode.java rename flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/{source/OceanBaseDeserializationRuntimeConverter.java => utils/OceanBaseUtils.java} (69%) rename flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/{OceanBaseConnectorITCase.java => OceanBaseMySQLModeITCase.java} (59%) create mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseOracleModeITCase.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/inventory_meta.sql rename flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/{ => mysql}/column_type_test.sql (97%) create mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/docker_init.sql rename flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/{ => mysql}/inventory.sql (95%) create mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/oracle/column_type_test.sql diff --git a/docs/content/docs/connectors/cdc-connectors/oceanbase-cdc.md b/docs/content/docs/connectors/cdc-connectors/oceanbase-cdc.md index 1e7887e8a08..699520d3af1 100644 --- a/docs/content/docs/connectors/cdc-connectors/oceanbase-cdc.md +++ b/docs/content/docs/connectors/cdc-connectors/oceanbase-cdc.md @@ -28,22 +28,82 @@ 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. +- *OceanBase Log Proxy CE*: OceanBase Log Proxy Community Edition. It's a proxy service which can fetch the commit log data of OceanBase cluster. It has been open sourced at https://github.com/oceanbase/oblogproxy. +- *OceanBase Log Proxy EE*: OceanBase Log Proxy Enterprise Edition. It's only available for users of OceanBase Cloud, you can contact the provider support for more details. +- *OceanBase Binlog Service CE*: OceanBase Binlog Service Community Edition. It's a solution composed of several open source components, which is compatible with MySQL Replication Protocol. See [docs (Chinese)](https://www.oceanbase.com/docs/community-oblogproxy-doc-1000000000531983). +- *OceanBase Binlog Service EE*: OceanBase Binlog Service Enterprise Edition. It's only available for users of Alibaba Cloud, see [docs](https://www.alibabacloud.com/help/en/apsaradb-for-oceanbase/latest/binlog-overview). +- *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. + +Solutions: + +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
DatabaseSupported DriverCDC ConnectorOther Required Components
OceanBase CE + MySQL Driver: 5.1.47 (builtin), 8.0.x
+ OceanBase Driver: 2.4.x +
OceanBase CDC ConnectorOceanBase Log Proxy CE (CDC Mode)
MySQL Driver: 8.0.27 (builtin)MySQL CDC ConnectorOceanBase Binlog Service CE
OceanBase EE (MySQL Mode) + MySQL Driver: 5.1.47 (builtin), 8.0.x
+ OceanBase Driver: 2.4.x +
OceanBase CDC ConnectorOceanBase Log Proxy EE (CDC Mode)
MySQL Driver: 8.0.27 (builtin)MySQL CDC ConnectorOceanBase Binlog Service EE
OceanBase EE (Oracle Mode)OceanBase Driver: 2.4.xOceanBase CDC ConnectorOceanBase Log Proxy EE (CDC Mode)
+
+ +Note: +- If you prefer to use MySQL CDC Connector, you can check the docs for more details at [mysql-cdc](mysql-cdc.md). +- When you want to use a driver other than the builtin driver, you need to manually introduce it to your application. + 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 - - com.oceanbase - oceanbase-client - 2.4.2 - -``` +{{< artifact flink-connector-oceanbase-cdc >}} ### SQL Client JAR @@ -84,7 +144,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 +222,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.
@@ -186,11 +246,11 @@ The OceanBase CDC Connector contains some options for both sql and stream api as scan.startup.mode - required - (none) + optional + initial String Specify the startup mode for OceanBase CDC consumer, valid enumerations are - 'initial','latest-offset' or 'timestamp'. + 'initial','latest-offset','timestamp' or 'snapshot'. @@ -216,24 +276,24 @@ The OceanBase CDC Connector contains some options for both sql and stream api as tenant-name - required + optional (none) String - Tenant name of OceanBase to monitor, should be exact value. + Tenant name of OceanBase to monitor, should be exact value. Required when 'scan.startup.mode' is not 'snapshot'. database-name optional (none) String - Database name of OceanBase to monitor, should be regular expression. Only can be used with 'initial' mode. + Database name of OceanBase to monitor, should be regular expression. table-name optional (none) String - Table name of OceanBase to monitor, should be regular expression. Only can be used with 'initial' mode. + Table name of OceanBase to monitor, should be regular expression. table-list @@ -244,14 +304,14 @@ The OceanBase CDC Connector contains some options for both sql and stream api as hostname - optional + required (none) String IP address or hostname of the OceanBase database server or OceanBase Proxy server. port - optional + required (none) Integer 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. @@ -272,17 +332,17 @@ The OceanBase CDC Connector contains some options for both sql and stream api as logproxy.host - required + optional (none) String - Hostname or IP address of OceanBase log proxy service. + Hostname or IP address of OceanBase log proxy service. Required when 'scan.startup.mode' is not 'snapshot'. logproxy.port - required + optional (none) Integer - Port number of OceanBase log proxy service. + Port number of OceanBase log proxy service. Required when 'scan.startup.mode' is not 'snapshot'. logproxy.client.id @@ -360,14 +420,19 @@ The following format metadata can be exposed as read-only (VIRTUAL) columns in a tenant_name - STRING NOT NULL + STRING Name of the tenant that contains the row. database_name - STRING NOT NULL + STRING Name of the database that contains the row. + + schema_name + STRING + Name of the schema that contains the row. + table_name STRING NOT NULL @@ -430,6 +495,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 diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/pom.xml b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/pom.xml index f755eb21b5b..6a0f777cbc5 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/pom.xml +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/pom.xml @@ -34,6 +34,13 @@ limitations under the License. jar + + + com.ververica + flink-cdc-base + ${project.version} + + com.ververica @@ -47,12 +54,6 @@ limitations under the License. - - com.ververica - flink-cdc-common - ${project.version} - - com.oceanbase diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSource.java index 2af7536c968..4a27c5fabec 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSource.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSource.java @@ -18,9 +18,9 @@ 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; @@ -47,7 +47,7 @@ public static Builder builder() { public static class Builder { // common config - private StartupMode startupMode; + private StartupOptions startupOptions; private String username; private String password; private String tenantName; @@ -73,11 +73,12 @@ public static class Builder { private String configUrl; private String workingMode; private Properties obcdcProperties; + private Properties debeziumProperties; - private OceanBaseDeserializationSchema deserializer; + private DebeziumDeserializationSchema deserializer; - public Builder startupMode(StartupMode startupMode) { - this.startupMode = startupMode; + public Builder startupOptions(StartupOptions startupOptions) { + this.startupOptions = startupOptions; return this; } @@ -151,7 +152,7 @@ public Builder logProxyHost(String logProxyHost) { return this; } - public Builder logProxyPort(int logProxyPort) { + public Builder logProxyPort(Integer logProxyPort) { this.logProxyPort = logProxyPort; return this; } @@ -186,23 +187,44 @@ public Builder obcdcProperties(Properties obcdcProperties) { return this; } - public Builder deserializer(OceanBaseDeserializationSchema deserializer) { + public Builder debeziumProperties(Properties debeziumProperties) { + this.debeziumProperties = debeziumProperties; + return this; + } + + public Builder deserializer(DebeziumDeserializationSchema deserializer) { this.deserializer = deserializer; return this; } public SourceFunction 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.jdbc.Driver"; + } + + if (connectTimeout == null) { + connectTimeout = Duration.ofSeconds(30); + } + + if (serverTimeZone == null) { + serverTimeZone = "+00:00"; + } + + switch (startupOptions.startupMode) { + case SNAPSHOT: break; + case INITIAL: case LATEST_OFFSET: startupTimestamp = 0L; break; @@ -213,15 +235,9 @@ public SourceFunction build() { 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 +249,61 @@ public SourceFunction build() { "'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 extraConfigs = new HashMap<>(); - obcdcProperties.forEach((k, v) -> extraConfigs.put(k.toString(), v.toString())); - obReaderConfig.setExtraConfigs(extraConfigs); + checkNotNull(logProxyHost); + checkNotNull(logProxyPort); + checkNotNull(tenantName); + + if (logProxyClientId == null) { + logProxyClientId = + String.format( + "%s_%s_%s", + ClientIdGenerator.generate(), + Thread.currentThread().getId(), + tenantName); + } + clientConf = + ClientConf.builder() + .clientId(logProxyClientId) + .connectTimeoutMs((int) connectTimeout.toMillis()) + .build(); + + 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.setTableWhiteList(tenantName + ".*.*"); + obReaderConfig.setUsername(username); + obReaderConfig.setPassword(password); + obReaderConfig.setStartTimestamp(startupTimestamp); + obReaderConfig.setTimezone(serverTimeZone); + + if (obcdcProperties != null && !obcdcProperties.isEmpty()) { + Map extraConfigs = new HashMap<>(); + obcdcProperties.forEach((k, v) -> extraConfigs.put(k.toString(), v.toString())); + obReaderConfig.setExtraConfigs(extraConfigs); + } } return new OceanBaseRichSourceFunction<>( - StartupMode.INITIAL.equals(startupMode), + startupOptions, username, password, tenantName, databaseName, tableName, tableList, + serverTimeZone, connectTimeout, hostname, port, @@ -294,6 +314,7 @@ public SourceFunction build() { logProxyPort, clientConf, obReaderConfig, + debeziumProperties, deserializer); } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseConnection.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseConnection.java index 89f538d621b..99994d12895 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseConnection.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseConnection.java @@ -17,10 +17,14 @@ package org.apache.flink.cdc.connectors.oceanbase.source; +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.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 OceanBaseConnection( 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 @@ private static String formatJdbcUrl(String jdbcDriver, Properties jdbcProperties 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 @@ private static Properties initializeDefaultJdbcProperties() { 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 Optional getCurrentTimestamp() throws SQLException { * * @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 getTables(String dbPattern, String tbPattern) throws SQLException { - List result = new ArrayList<>(); + public List getTables(String dbPattern, String tbPattern) throws SQLException { + List result = new ArrayList<>(); DatabaseMetaData metaData = connection().getMetaData(); switch (compatibleMode.toLowerCase()) { case "mysql": @@ -171,27 +186,28 @@ public List getTables(String dbPattern, String tbPattern) throws SQLExce for (String dbName : dbNames) { List 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 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 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 @@ private List getResultList(ResultSet resultSet, String columnName) throw } return result; } + + @Override + protected String[] supportedTableTypes() { + return new String[] {"TABLE"}; + } + + @Override + public String quotedTableIdString(TableId tableId) { + return tableId.toQuotedString(getQuote(compatibleMode)); + } + + public void readSchemaForCapturedTables( + Tables tables, + String databaseCatalog, + String schemaNamePattern, + Tables.ColumnNameFilter columnFilter, + boolean removeTablesNotFoundInJdbc, + Set capturedTables) + throws SQLException { + + Set tableIdsBefore = new HashSet<>(tables.tableIds()); + + DatabaseMetaData metadata = connection().getMetaData(); + Map> columnsByTable = new HashMap<>(); + + for (TableId tableId : capturedTables) { + try (ResultSet columnMetadata = + metadata.getColumns( + databaseCatalog, schemaNamePattern, tableId.table(), null)) { + while (columnMetadata.next()) { + // add all whitelisted columns + readTableColumn(columnMetadata, tableId, columnFilter) + .ifPresent( + column -> { + columnsByTable + .computeIfAbsent(tableId, t -> new ArrayList<>()) + .add(column.create()); + }); + } + } + } + + // Read the metadata for the primary keys ... + for (Map.Entry> tableEntry : columnsByTable.entrySet()) { + // First get the primary key information, which must be done for *each* table ... + List pkColumnNames = readPrimaryKeyNames(metadata, tableEntry.getKey()); + + // Then define the table ... + List columns = tableEntry.getValue(); + Collections.sort(columns); + tables.overwriteTable(tableEntry.getKey(), columns, pkColumnNames, null); + } + + if (removeTablesNotFoundInJdbc) { + // Remove any definitions for tables that were not found in the database metadata ... + tableIdsBefore.removeAll(columnsByTable.keySet()); + tableIdsBefore.forEach(tables::removeTable); + } + } + + @Override + protected int resolveNativeType(String typeName) { + String upperCaseTypeName = typeName.toUpperCase(); + if (upperCaseTypeName.startsWith("JSON")) { + return Types.VARCHAR; + } + if (upperCaseTypeName.startsWith("NCHAR")) { + return Types.NCHAR; + } + if (upperCaseTypeName.startsWith("NVARCHAR2")) { + return Types.NVARCHAR; + } + if (upperCaseTypeName.startsWith("TIMESTAMP")) { + if (upperCaseTypeName.contains("WITH TIME ZONE")) { + return TYPE_TIMESTAMP_WITH_TIME_ZONE; + } + if (upperCaseTypeName.contains("WITH LOCAL TIME ZONE")) { + return TYPE_TIMESTAMP_WITH_LOCAL_TIME_ZONE; + } + return Types.TIMESTAMP; + } + if (upperCaseTypeName.startsWith("INTERVAL")) { + if (upperCaseTypeName.contains("TO MONTH")) { + return TYPE_INTERVAL_YEAR_TO_MONTH; + } + if (upperCaseTypeName.contains("TO SECOND")) { + return TYPE_INTERVAL_DAY_TO_SECOND; + } + } + return Column.UNSET_INT_VALUE; + } + + @Override + protected int resolveJdbcType(int metadataJdbcType, int nativeType) { + switch (metadataJdbcType) { + case TYPE_BINARY_FLOAT: + return Types.REAL; + case TYPE_BINARY_DOUBLE: + return Types.DOUBLE; + case TYPE_TIMESTAMP_WITH_TIME_ZONE: + case TYPE_TIMESTAMP_WITH_LOCAL_TIME_ZONE: + case TYPE_INTERVAL_YEAR_TO_MONTH: + case TYPE_INTERVAL_DAY_TO_SECOND: + return Types.OTHER; + default: + return nativeType == Column.UNSET_INT_VALUE ? metadataJdbcType : nativeType; + } + } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseDeserializationConverterFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseDeserializationConverterFactory.java new file mode 100644 index 00000000000..4bc8f6e2338 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseDeserializationConverterFactory.java @@ -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; + +import org.apache.flink.cdc.debezium.table.DeserializationRuntimeConverter; +import org.apache.flink.cdc.debezium.table.DeserializationRuntimeConverterFactory; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.LogicalType; + +import org.apache.kafka.connect.data.Schema; + +import java.time.ZoneId; +import java.util.Optional; + +/** Used to create {@link DeserializationRuntimeConverterFactory} specified to OceanBase. */ +public class OceanBaseDeserializationConverterFactory { + + public static DeserializationRuntimeConverterFactory instance() { + return new DeserializationRuntimeConverterFactory() { + + private static final long serialVersionUID = 1L; + + @Override + public Optional createUserDefinedConverter( + LogicalType logicalType, ZoneId serverTimeZone) { + switch (logicalType.getTypeRoot()) { + case ARRAY: + return createArrayConverter(); + default: + return Optional.empty(); + } + } + }; + } + + private static Optional createArrayConverter() { + return Optional.of( + new DeserializationRuntimeConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(Object dbzObj, Schema schema) throws Exception { + if (dbzObj instanceof String) { + String[] enums = ((String) dbzObj).split(","); + StringData[] elements = new StringData[enums.length]; + for (int i = 0; i < enums.length; i++) { + elements[i] = StringData.fromString(enums[i]); + } + return new GenericArrayData(elements); + } + throw new IllegalArgumentException( + String.format( + "Unable convert to Flink ARRAY type from unexpected value '%s'", + dbzObj)); + } + }); + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseRichSourceFunction.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseRichSourceFunction.java index 92df4515fcc..203b78ec2c3 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseRichSourceFunction.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseRichSourceFunction.java @@ -23,8 +23,11 @@ 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.schema.OceanBaseDatabaseSchema; +import org.apache.flink.cdc.connectors.oceanbase.source.schema.OceanBaseSourceInfo; +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 +41,23 @@ import com.oceanbase.clogproxy.client.config.ObReaderConfig; import com.oceanbase.clogproxy.client.exception.LogProxyClientException; import com.oceanbase.clogproxy.client.listener.RecordListener; +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; @@ -72,13 +84,14 @@ public class OceanBaseRichSourceFunction extends RichSourceFunction 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,14 +99,19 @@ public class OceanBaseRichSourceFunction extends RichSourceFunction private final String jdbcDriver; private final Properties jdbcProperties; private final String logProxyHost; - private final int logProxyPort; + private final Integer logProxyPort; private final ClientConf logProxyClientConf; private final ObReaderConfig obReaderConfig; - private final OceanBaseDeserializationSchema deserializer; + private final Properties debeziumProperties; + private final DebeziumDeserializationSchema deserializer; - private final List changeRecordBuffer = new LinkedList<>(); + private final List changeRecordBuffer = new LinkedList<>(); - private transient Set tableSet; + private transient OceanBaseConnectorConfig connectorConfig; + private transient OceanBaseSourceInfo sourceInfo; + private transient Set tableSet; + private transient OceanBaseSchema obSchema; + private transient OceanBaseDatabaseSchema databaseSchema; private transient volatile long resolvedTimestamp; private transient volatile OceanBaseConnection snapshotConnection; private transient LogProxyClient logProxyClient; @@ -101,13 +119,14 @@ public class OceanBaseRichSourceFunction extends RichSourceFunction private transient OutputCollector 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 +134,30 @@ public OceanBaseRichSourceFunction( String jdbcDriver, Properties jdbcProperties, String logProxyHost, - int logProxyPort, + Integer logProxyPort, ClientConf logProxyClientConf, ObReaderConfig obReaderConfig, - OceanBaseDeserializationSchema deserializer) { - this.snapshot = checkNotNull(snapshot); + Properties debeziumProperties, + DebeziumDeserializationSchema 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.logProxyClientConf = logProxyClientConf; + this.obReaderConfig = obReaderConfig; + this.debeziumProperties = debeziumProperties; this.deserializer = checkNotNull(deserializer); } @@ -143,6 +165,9 @@ public OceanBaseRichSourceFunction( 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 +177,28 @@ public void run(SourceContext ctx) throws Exception { 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 +227,35 @@ private void closeSnapshotConnection() { } } + private TableId tableId(String databaseName, String tableName) { + if ("mysql".equalsIgnoreCase(compatibleMode)) { + return new TableId(databaseName, null, tableName); + } + return new TableId(null, databaseName, tableName); + } + private void initTableWhiteList() { if (tableSet != null && !tableSet.isEmpty()) { return; } - final Set localTableSet = new HashSet<>(); + final Set 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 tables = getSnapshotConnection().getTables(databaseName, tableName); - LOG.info("Pattern matched tables: {}", tables); - localTableSet.addAll(tables); + List tableIds = getSnapshotConnection().getTables(databaseName, tableName); + LOG.info("Pattern matched tables: {}", tableIds); + localTableSet.addAll(tableIds); } catch (SQLException e) { LOG.error( String.format( @@ -238,50 +272,74 @@ private void initTableWhiteList() { LOG.info("Table list: {}", localTableSet); this.tableSet = localTableSet; - this.obReaderConfig.setTableWhiteList(String.format("%s.*.*", tenantName)); + } + + 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 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 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(); + sourceInfo.setSourceTime(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); @@ -321,7 +379,7 @@ public void notify(LogMessage message) { if (!started) { break; } - OceanBaseRecord record = getChangeRecord(message); + SourceRecord record = getChangeRecord(message); if (record != null) { changeRecordBuffer.add(record); } @@ -375,18 +433,81 @@ public void onException(LogProxyClientException e) { logProxyClient.join(); } - 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.now()); + Struct source = sourceInfo.struct(); + + TableSchema tableSchema = getTableSchema(tableId); + Struct struct; + Schema valueSchema = tableSchema.valueSchema(); + List fields = valueSchema.fields(); + Struct before, after; + Object[] beforeFieldValues, afterFieldValues; + Map beforeValueMap = new HashMap<>(); + Map afterValueMap = new HashMap<>(); + message.getFieldList() + .forEach( + field -> { + if (field.isPrev()) { + beforeValueMap.put(field.getFieldname(), getFieldValue(field)); + } else { + afterValueMap.put(field.getFieldname(), getFieldValue(field)); + } + }); + switch (message.getOpt()) { + case INSERT: + afterFieldValues = new Object[fields.size()]; + for (Field field : fields) { + afterFieldValues[field.index()] = afterValueMap.get(field.name()); + } + after = tableSchema.valueFromColumnData(afterFieldValues); + struct = tableSchema.getEnvelopeSchema().create(after, source, Instant.now()); + break; + case DELETE: + beforeFieldValues = new Object[fields.size()]; + for (Field field : fields) { + beforeFieldValues[field.index()] = beforeValueMap.get(field.name()); + } + before = tableSchema.valueFromColumnData(beforeFieldValues); + struct = tableSchema.getEnvelopeSchema().delete(before, source, Instant.now()); + break; + case UPDATE: + beforeFieldValues = new Object[fields.size()]; + afterFieldValues = new Object[fields.size()]; + for (Field field : fields) { + beforeFieldValues[field.index()] = beforeValueMap.get(field.name()); + afterFieldValues[field.index()] = afterValueMap.get(field.name()); + } + before = tableSchema.valueFromColumnData(beforeFieldValues); + after = tableSchema.valueFromColumnData(afterFieldValues); + struct = + tableSchema + .getEnvelopeSchema() + .update(before, after, source, Instant.now()); + break; + default: + throw new UnsupportedOperationException(); + } + return new SourceRecord( + null, null, tableId.identifier(), null, null, null, struct.schema(), struct); + } + + private Object getFieldValue(DataMessage.Record.Field field) { + if (field.getValue() == null) { return null; } - 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 diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseSchema.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseSchema.java new file mode 100644 index 00000000000..0444b36efe7 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseSchema.java @@ -0,0 +1,80 @@ +/* + * 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.util.FlinkRuntimeException; + +import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.Table; +import io.debezium.relational.TableId; +import io.debezium.relational.Tables; +import io.debezium.relational.history.TableChanges; + +import java.sql.SQLException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +/** A component used to get schema by table path. */ +public class OceanBaseSchema { + + private final Map schemasByTableId; + + public OceanBaseSchema() { + this.schemasByTableId = new HashMap<>(); + } + + public TableChanges.TableChange getTableSchema(JdbcConnection connection, TableId tableId) { + TableChanges.TableChange schema = schemasByTableId.get(tableId); + if (schema == null) { + schema = readTableSchema(connection, tableId); + schemasByTableId.put(tableId, schema); + } + return schema; + } + + private TableChanges.TableChange readTableSchema(JdbcConnection jdbc, TableId tableId) { + OceanBaseConnection connection = (OceanBaseConnection) jdbc; + Set tableIdSet = new HashSet<>(); + tableIdSet.add(tableId); + + final Map tableChangeMap = new HashMap<>(); + Tables tables = new Tables(); + tables.overwriteTable(tables.editOrCreateTable(tableId).create()); + + try { + connection.readSchemaForCapturedTables( + tables, tableId.catalog(), tableId.schema(), null, false, tableIdSet); + Table table = tables.forTable(tableId); + TableChanges.TableChange tableChange = + new TableChanges.TableChange(TableChanges.TableChangeType.CREATE, table); + tableChangeMap.put(tableId, tableChange); + } catch (SQLException e) { + throw new FlinkRuntimeException( + String.format("Failed to read schema for table %s ", tableId), e); + } + + if (!tableChangeMap.containsKey(tableId)) { + throw new FlinkRuntimeException( + String.format("Can't obtain schema for table %s ", tableId)); + } + + return tableChangeMap.get(tableId); + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseValueConverters.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseValueConverters.java new file mode 100644 index 00000000000..2365751741d --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseValueConverters.java @@ -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; + +import org.apache.flink.cdc.connectors.oceanbase.source.config.OceanBaseConnectorConfig; + +import io.debezium.config.CommonConnectorConfig; +import io.debezium.data.Bits; +import io.debezium.data.SpecialValueDecimal; +import io.debezium.jdbc.JdbcValueConverters; +import io.debezium.relational.Column; +import io.debezium.relational.ValueConverter; +import io.debezium.time.MicroTimestamp; +import io.debezium.time.NanoTimestamp; +import org.apache.kafka.connect.data.Decimal; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.SchemaBuilder; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.sql.Blob; +import java.sql.Clob; +import java.sql.Date; +import java.sql.SQLException; +import java.sql.Time; +import java.sql.Timestamp; +import java.sql.Types; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.time.temporal.ChronoField; +import java.util.Locale; + +/** JdbcValueConverters for OceanBase. */ +public class OceanBaseValueConverters extends JdbcValueConverters { + + public static final String EMPTY_BLOB_FUNCTION = "EMPTY_BLOB()"; + public static final String EMPTY_CLOB_FUNCTION = "EMPTY_CLOB()"; + + private static final DateTimeFormatter TIMESTAMP_FORMATTER = + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .appendPattern("yyyy-MM-dd HH:mm:ss") + .optionalStart() + .appendPattern(".") + .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, false) + .optionalEnd() + .toFormatter(); + + private static final DateTimeFormatter TIMESTAMP_AM_PM_SHORT_FORMATTER = + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .appendPattern("dd-MMM-yy hh.mm.ss") + .optionalStart() + .appendPattern(".") + .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, false) + .optionalEnd() + .appendPattern(" a") + .toFormatter(Locale.ENGLISH); + + private final String compatibleMode; + private final String serverTimezone; + + public OceanBaseValueConverters(OceanBaseConnectorConfig connectorConfig) { + super( + connectorConfig.getDecimalMode(), + connectorConfig.getTemporalPrecisionMode(), + ZoneOffset.UTC, + x -> x, + BigIntUnsignedMode.PRECISE, + connectorConfig.binaryHandlingMode()); + this.compatibleMode = connectorConfig.getCompatibleMode(); + this.serverTimezone = connectorConfig.getServerTimeZone(); + } + + @Override + protected int getTimePrecision(Column column) { + if ("mysql".equalsIgnoreCase(compatibleMode)) { + return super.getTimePrecision(column); + } + return column.scale().orElse(0); + } + + protected boolean isUnsignedColumn(Column column) { + return column.typeName().toUpperCase().contains("UNSIGNED"); + } + + @Override + public SchemaBuilder schemaBuilder(Column column) { + logger.debug( + "Building schema for column {} of type {} named {} with constraints ({},{})", + column.name(), + column.jdbcType(), + column.typeName(), + column.length(), + column.scale()); + + switch (column.jdbcType()) { + case Types.BIT: + if (column.length() > 1) { + return Bits.builder(column.length()); + } + return SchemaBuilder.bool(); + case Types.TINYINT: + if (column.length() == 1) { + return SchemaBuilder.bool(); + } + if (isUnsignedColumn(column)) { + return SchemaBuilder.int16(); + } + return SchemaBuilder.int8(); + case Types.SMALLINT: + if (isUnsignedColumn(column)) { + return SchemaBuilder.int32(); + } + return SchemaBuilder.int16(); + case Types.INTEGER: + if (!column.typeName().toUpperCase().startsWith("MEDIUMINT") + && isUnsignedColumn(column)) { + return SchemaBuilder.int64(); + } + return SchemaBuilder.int32(); + case Types.BIGINT: + if (isUnsignedColumn(column)) { + return Decimal.builder(0); + } + return SchemaBuilder.int64(); + case Types.FLOAT: + return getDecimalSchema(column); + case Types.NUMERIC: + case Types.DECIMAL: + if ("mysql".equalsIgnoreCase(compatibleMode)) { + return getDecimalSchema(column); + } + return getNumericSchema(column); + case Types.REAL: + return SchemaBuilder.float32(); + case Types.DOUBLE: + return SchemaBuilder.float64(); + case Types.DATE: + if ("mysql".equalsIgnoreCase(compatibleMode)) { + if (column.typeName().equalsIgnoreCase("YEAR")) { + return io.debezium.time.Year.builder(); + } + if (adaptiveTimePrecisionMode || adaptiveTimeMicrosecondsPrecisionMode) { + return io.debezium.time.Date.builder(); + } + return org.apache.kafka.connect.data.Date.builder(); + } + return getTimestampSchema(column); + case Types.TIME: + if (adaptiveTimeMicrosecondsPrecisionMode) { + return io.debezium.time.MicroTime.builder(); + } + if (adaptiveTimePrecisionMode) { + if (getTimePrecision(column) <= 3) { + return io.debezium.time.Time.builder(); + } + if (getTimePrecision(column) <= 6) { + return io.debezium.time.MicroTime.builder(); + } + return io.debezium.time.NanoTime.builder(); + } + return org.apache.kafka.connect.data.Time.builder(); + case Types.TIMESTAMP: + return getTimestampSchema(column); + case Types.CHAR: + case Types.VARCHAR: + case Types.LONGVARCHAR: + case Types.NCHAR: + case Types.NVARCHAR: + case Types.CLOB: + return SchemaBuilder.string(); + case Types.BINARY: + case Types.VARBINARY: + case Types.LONGVARBINARY: + case Types.BLOB: + return binaryMode.getSchema(); + default: + return super.schemaBuilder(column); + } + } + + protected SchemaBuilder getNumericSchema(Column column) { + if (column.scale().isPresent()) { + int scale = column.scale().get(); + if (scale <= 0) { + int width = column.length() - scale; + if (width < 3) { + return SchemaBuilder.int8(); + } else if (width < 5) { + return SchemaBuilder.int16(); + } else if (width < 10) { + return SchemaBuilder.int32(); + } else if (width < 19) { + return SchemaBuilder.int64(); + } + } + } + return getDecimalSchema(column); + } + + protected SchemaBuilder getDecimalSchema(Column column) { + return SpecialValueDecimal.builder(decimalMode, column.length(), column.scale().orElse(0)); + } + + protected SchemaBuilder getTimestampSchema(Column column) { + if (adaptiveTimePrecisionMode || adaptiveTimeMicrosecondsPrecisionMode) { + if (getTimePrecision(column) <= 3) { + return io.debezium.time.Timestamp.builder(); + } + if (getTimePrecision(column) <= 6) { + return MicroTimestamp.builder(); + } + return NanoTimestamp.builder(); + } + return org.apache.kafka.connect.data.Timestamp.builder(); + } + + @Override + public ValueConverter converter(Column column, Field fieldDefn) { + switch (column.jdbcType()) { + case Types.BIT: + return convertBits(column, fieldDefn); + case Types.TINYINT: + if (column.length() == 1) { + return data -> convertBit(column, fieldDefn, data); + } + if (isUnsignedColumn(column)) { + return data -> convertSmallInt(column, fieldDefn, data); + } + return data -> convertTinyInt(column, fieldDefn, data); + case Types.SMALLINT: + if (isUnsignedColumn(column)) { + return data -> convertInteger(column, fieldDefn, data); + } + return data -> convertSmallInt(column, fieldDefn, data); + case Types.INTEGER: + if (column.typeName().toUpperCase().startsWith("MEDIUMINT")) { + return data -> convertInteger(column, fieldDefn, data); + } + if (isUnsignedColumn(column)) { + return data -> convertBigInt(column, fieldDefn, data); + } + return data -> convertInteger(column, fieldDefn, data); + case Types.BIGINT: + if (isUnsignedColumn(column)) { + switch (bigIntUnsignedMode) { + case LONG: + return (data) -> convertBigInt(column, fieldDefn, data); + case PRECISE: + return (data) -> convertUnsignedBigint(column, fieldDefn, data); + } + } + return (data) -> convertBigInt(column, fieldDefn, data); + case Types.FLOAT: + return data -> convertDecimal(column, fieldDefn, data); + case Types.NUMERIC: + case Types.DECIMAL: + if ("mysql".equalsIgnoreCase(compatibleMode)) { + return data -> convertDecimal(column, fieldDefn, data); + } + return data -> convertNumeric(column, fieldDefn, data); + case Types.REAL: + return data -> convertReal(column, fieldDefn, data); + case Types.DOUBLE: + return data -> convertDouble(column, fieldDefn, data); + case Types.DATE: + if ("mysql".equalsIgnoreCase(compatibleMode)) { + if (column.typeName().equalsIgnoreCase("YEAR")) { + return (data) -> convertYearToInt(column, fieldDefn, data); + } + if (adaptiveTimePrecisionMode || adaptiveTimeMicrosecondsPrecisionMode) { + return (data) -> convertDateToEpochDays(column, fieldDefn, data); + } + return (data) -> convertDateToEpochDaysAsDate(column, fieldDefn, data); + } + return (data) -> convertTimestamp(column, fieldDefn, data); + case Types.TIME: + return (data) -> convertTime(column, fieldDefn, data); + case Types.TIMESTAMP: + return data -> convertTimestamp(column, fieldDefn, data); + case Types.CHAR: + case Types.VARCHAR: + case Types.LONGVARCHAR: + case Types.NCHAR: + case Types.NVARCHAR: + case Types.CLOB: + return data -> convertString(column, fieldDefn, data); + case Types.BINARY: + case Types.VARBINARY: + case Types.LONGVARBINARY: + case Types.BLOB: + return (data) -> convertBinary(column, fieldDefn, data, binaryMode); + default: + return super.converter(column, fieldDefn); + } + } + + @Override + protected Object convertBits(Column column, Field fieldDefn, Object data, int numBytes) { + if (data instanceof String) { + return ByteBuffer.allocate(numBytes).putLong(Long.parseLong((String) data)).array(); + } + return super.convertBits(column, fieldDefn, data, numBytes); + } + + @Override + protected Object convertBit(Column column, Field fieldDefn, Object data) { + if (data instanceof String) { + return Boolean.parseBoolean((String) data) || "1".equals(data); + } + return super.convertBit(column, fieldDefn, data); + } + + @Override + protected Object convertTinyInt(Column column, Field fieldDefn, Object data) { + if (data instanceof String) { + return Byte.parseByte((String) data); + } + if (data instanceof Number) { + return ((Number) data).byteValue(); + } + throw new IllegalArgumentException( + "Unexpected value for JDBC type " + + column.jdbcType() + + " and column " + + column + + ": class=" + + data.getClass()); + } + + @Override + protected Object convertBigInt(Column column, Field fieldDefn, Object data) { + if (data instanceof String) { + return new BigInteger((String) data).longValue(); + } + return super.convertBigInt(column, fieldDefn, data); + } + + protected Object convertUnsignedBigint(Column column, Field fieldDefn, Object data) { + if (data instanceof String) { + return new BigDecimal((String) data); + } + if (data instanceof BigInteger) { + return new BigDecimal((BigInteger) data); + } + return convertDecimal(column, fieldDefn, data); + } + + @Override + protected Object convertReal(Column column, Field fieldDefn, Object data) { + if (data instanceof String) { + return Float.parseFloat((String) data); + } + return super.convertReal(column, fieldDefn, data); + } + + @Override + protected Object convertDouble(Column column, Field fieldDefn, Object data) { + if (data instanceof String) { + return Double.parseDouble((String) data); + } + return super.convertDouble(column, fieldDefn, data); + } + + @Override + protected Object convertNumeric(Column column, Field fieldDefn, Object data) { + if (column.scale().isPresent()) { + int scale = column.scale().get(); + + if (scale <= 0) { + int width = column.length() - scale; + if (width < 3) { + return convertTinyInt(column, fieldDefn, data); + } else if (width < 5) { + return convertSmallInt(column, fieldDefn, data); + } else if (width < 10) { + return convertInteger(column, fieldDefn, data); + } else if (width < 19) { + return convertBigInt(column, fieldDefn, data); + } + } + } + return convertDecimal(column, fieldDefn, data); + } + + protected Object convertYearToInt(Column column, Field fieldDefn, Object data) { + if (data instanceof Date) { + return ((Date) data).toLocalDate().getYear(); + } + return convertInteger(column, fieldDefn, data); + } + + @Override + protected Object convertDateToEpochDays(Column column, Field fieldDefn, Object data) { + if (data instanceof String) { + data = Date.valueOf((String) data); + } + return super.convertDateToEpochDays(column, fieldDefn, data); + } + + @Override + protected Object convertDateToEpochDaysAsDate(Column column, Field fieldDefn, Object data) { + if (data instanceof String) { + data = Date.valueOf((String) data); + } + return super.convertDateToEpochDaysAsDate(column, fieldDefn, data); + } + + @Override + protected Object convertTime(Column column, Field fieldDefn, Object data) { + if (data instanceof String) { + data = Time.valueOf((String) data); + } + return super.convertTime(column, fieldDefn, data); + } + + protected Object convertTimestamp(Column column, Field fieldDefn, Object data) { + if (data instanceof String) { + if ("mysql".equalsIgnoreCase(compatibleMode)) { + data = Timestamp.valueOf(((String) data).trim()); + } else { + data = resolveTimestampStringAsInstant((String) data); + } + } + if (adaptiveTimePrecisionMode || adaptiveTimeMicrosecondsPrecisionMode) { + if (getTimePrecision(column) <= 3) { + return convertTimestampToEpochMillis(column, fieldDefn, data); + } + if (getTimePrecision(column) <= 6) { + return convertTimestampToEpochMicros(column, fieldDefn, data); + } + return convertTimestampToEpochNanos(column, fieldDefn, data); + } + return convertTimestampToEpochMillisAsDate(column, fieldDefn, data); + } + + protected Instant resolveTimestampStringAsInstant(String dateText) { + LocalDateTime dateTime; + if (dateText.indexOf(" AM") > 0 || dateText.indexOf(" PM") > 0) { + dateTime = LocalDateTime.from(TIMESTAMP_AM_PM_SHORT_FORMATTER.parse(dateText.trim())); + } else { + dateTime = LocalDateTime.from(TIMESTAMP_FORMATTER.parse(dateText.trim())); + } + return dateTime.atZone(ZoneId.of(serverTimezone)).toInstant(); + } + + @Override + protected Object convertString(Column column, Field fieldDefn, Object data) { + if (data instanceof Clob) { + try { + Clob clob = (Clob) data; + return clob.getSubString(1, (int) clob.length()); + } catch (SQLException e) { + throw new RuntimeException("Couldn't convert value for column " + column.name(), e); + } + } + if (data instanceof String) { + String s = (String) data; + if (EMPTY_CLOB_FUNCTION.equals(s)) { + return column.isOptional() ? null : ""; + } + } + return super.convertString(column, fieldDefn, data); + } + + @Override + protected Object convertBinary( + Column column, + Field fieldDefn, + Object data, + CommonConnectorConfig.BinaryHandlingMode mode) { + try { + if (data instanceof Blob) { + Blob blob = (Blob) data; + data = blob.getBytes(1, Long.valueOf(blob.length()).intValue()); + } + if (data instanceof String) { + String str = (String) data; + if (EMPTY_BLOB_FUNCTION.equals(str)) { + data = column.isOptional() ? null : ""; + } + } + return super.convertBinary(column, fieldDefn, data, mode); + } catch (SQLException e) { + throw new RuntimeException("Couldn't convert value for column " + column.name(), e); + } + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/RowDataOceanBaseDeserializationSchema.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/RowDataOceanBaseDeserializationSchema.java deleted file mode 100644 index 9b36fce8c76..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/RowDataOceanBaseDeserializationSchema.java +++ /dev/null @@ -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 { - - private static final long serialVersionUID = 1L; - - /** TypeInformation of the produced {@link RowData}. * */ - private final TypeInformation 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 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 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 collector) { - if (!hasMetadata) { - collector.collect(physicalRow); - return; - } - - appendMetadataCollector.inputRecord = row; - appendMetadataCollector.outputCollector = collector; - appendMetadataCollector.collect(physicalRow); - } - - @Override - public TypeInformation getProducedType() { - return resultTypeInfo; - } - - /** Builder class of {@link RowDataOceanBaseDeserializationSchema}. */ - public static class Builder { - private RowType physicalRowType; - private TypeInformation 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 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 fieldMap = (Map) 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); - } - }; - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/config/OceanBaseConnectorConfig.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/config/OceanBaseConnectorConfig.java new file mode 100644 index 00000000000..c4cf07afc97 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/config/OceanBaseConnectorConfig.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.connectors.oceanbase.source.config; + +import org.apache.flink.cdc.connectors.oceanbase.source.schema.OceanBaseSourceInfoStructMaker; + +import io.debezium.config.Configuration; +import io.debezium.connector.SourceInfoStructMaker; +import io.debezium.relational.ColumnFilterMode; +import io.debezium.relational.RelationalDatabaseConnectorConfig; +import io.debezium.relational.TableId; +import io.debezium.relational.Tables; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Properties; + +/** Debezium connector config. */ +public class OceanBaseConnectorConfig extends RelationalDatabaseConnectorConfig { + + protected static final String LOGICAL_NAME = "oceanbase_cdc_connector"; + protected static final int DEFAULT_SNAPSHOT_FETCH_SIZE = Integer.MIN_VALUE; + protected static final List BUILT_IN_DB_NAMES = + Collections.unmodifiableList( + Arrays.asList( + "information_schema", + "mysql", + "oceanbase", + "LBACSYS", + "ORAAUDITOR", + "SYS")); + + private final String compatibleMode; + private final String serverTimeZone; + + public OceanBaseConnectorConfig( + String compatibleMode, String serverTimeZone, Properties properties) { + super( + Configuration.from(properties), + LOGICAL_NAME, + Tables.TableFilter.fromPredicate( + tableId -> + "mysql".equalsIgnoreCase(compatibleMode) + ? !BUILT_IN_DB_NAMES.contains(tableId.catalog()) + : !BUILT_IN_DB_NAMES.contains(tableId.schema())), + TableId::identifier, + DEFAULT_SNAPSHOT_FETCH_SIZE, + "mysql".equalsIgnoreCase(compatibleMode) + ? ColumnFilterMode.CATALOG + : ColumnFilterMode.SCHEMA); + this.compatibleMode = compatibleMode; + this.serverTimeZone = serverTimeZone; + } + + public String getCompatibleMode() { + return compatibleMode; + } + + public String getServerTimeZone() { + return serverTimeZone; + } + + @Override + public String getConnectorName() { + return "oceanbase"; + } + + @Override + public String getContextName() { + return "OceanBase"; + } + + @Override + protected SourceInfoStructMaker getSourceInfoStructMaker(Version version) { + return new OceanBaseSourceInfoStructMaker(); + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseDatabaseSchema.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseDatabaseSchema.java new file mode 100644 index 00000000000..af077882593 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseDatabaseSchema.java @@ -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.OceanBaseValueConverters; +import org.apache.flink.cdc.connectors.oceanbase.source.config.OceanBaseConnectorConfig; + +import io.debezium.relational.RelationalDatabaseSchema; +import io.debezium.relational.TableSchemaBuilder; +import io.debezium.relational.Tables; +import io.debezium.schema.TopicSelector; + +/** OceanBase database schema. */ +public class OceanBaseDatabaseSchema extends RelationalDatabaseSchema { + + public OceanBaseDatabaseSchema( + OceanBaseConnectorConfig connectorConfig, + Tables.TableFilter tableFilter, + boolean tableIdCaseInsensitive) { + super( + connectorConfig, + TopicSelector.defaultSelector( + connectorConfig, + (tableId, prefix, delimiter) -> + String.join(delimiter, prefix, tableId.identifier())), + tableFilter, + connectorConfig.getColumnFilter(), + new TableSchemaBuilder( + new OceanBaseValueConverters(connectorConfig), + connectorConfig.schemaNameAdjustmentMode().createAdjuster(), + connectorConfig.customConverterRegistry(), + connectorConfig.getSourceInfoStructMaker().schema(), + connectorConfig.getSanitizeFieldNames(), + false), + tableIdCaseInsensitive, + connectorConfig.getKeyMapper()); + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseSourceInfo.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseSourceInfo.java new file mode 100644 index 00000000000..03ed2435802 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseSourceInfo.java @@ -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.schema; + +import org.apache.flink.cdc.connectors.oceanbase.source.config.OceanBaseConnectorConfig; + +import io.debezium.connector.common.BaseSourceInfo; +import io.debezium.relational.TableId; + +import java.time.Instant; +import java.util.Collections; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +/** OceanBase source info. */ +public class OceanBaseSourceInfo extends BaseSourceInfo { + + public static final String TENANT_KEY = "tenant"; + public static final String TRANSACTION_ID_KEY = "transaction_id"; + + private final String tenant; + + private Instant sourceTime; + private Set tableIds; + private String transactionId; + + public OceanBaseSourceInfo(OceanBaseConnectorConfig config, String tenant) { + super(config); + this.tenant = tenant; + } + + public String tenant() { + return tenant; + } + + @Override + protected Instant timestamp() { + return sourceTime; + } + + public void setSourceTime(Instant sourceTime) { + this.sourceTime = sourceTime; + } + + public void beginTransaction(String transactionId) { + this.transactionId = transactionId; + } + + public void commitTransaction() { + this.transactionId = null; + } + + public String transactionId() { + return transactionId; + } + + public void tableEvent(TableId tableId) { + this.tableIds = Collections.singleton(tableId); + } + + @Override + protected String database() { + return (tableIds != null) ? tableIds.iterator().next().catalog() : null; + } + + public String tableSchema() { + return (tableIds == null || tableIds.isEmpty()) + ? null + : tableIds.stream() + .filter(Objects::nonNull) + .map(TableId::schema) + .filter(Objects::nonNull) + .distinct() + .collect(Collectors.joining(",")); + } + + public String table() { + return (tableIds == null || tableIds.isEmpty()) + ? null + : tableIds.stream() + .filter(Objects::nonNull) + .map(TableId::table) + .collect(Collectors.joining(",")); + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseSourceInfoStructMaker.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseSourceInfoStructMaker.java new file mode 100644 index 00000000000..e852bfb4111 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseSourceInfoStructMaker.java @@ -0,0 +1,68 @@ +/* + * 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 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; + +/** The {@link SourceInfoStructMaker} implementation for OceanBase. */ +public class OceanBaseSourceInfoStructMaker implements SourceInfoStructMaker { + private final Schema schema; + + public OceanBaseSourceInfoStructMaker() { + this.schema = + SchemaBuilder.struct() + .field(OceanBaseSourceInfo.TABLE_NAME_KEY, Schema.STRING_SCHEMA) + .field(OceanBaseSourceInfo.TIMESTAMP_KEY, Schema.INT64_SCHEMA) + .field(OceanBaseSourceInfo.TENANT_KEY, Schema.OPTIONAL_STRING_SCHEMA) + .field(OceanBaseSourceInfo.DATABASE_NAME_KEY, Schema.OPTIONAL_STRING_SCHEMA) + .field(OceanBaseSourceInfo.SCHEMA_NAME_KEY, Schema.OPTIONAL_STRING_SCHEMA) + .field( + OceanBaseSourceInfo.TRANSACTION_ID_KEY, + Schema.OPTIONAL_STRING_SCHEMA) + .build(); + } + + @Override + public Schema schema() { + return schema; + } + + @Override + public Struct struct(OceanBaseSourceInfo sourceInfo) { + Struct source = new Struct(schema); + source.put(OceanBaseSourceInfo.TABLE_NAME_KEY, sourceInfo.table()); + source.put(OceanBaseSourceInfo.TIMESTAMP_KEY, sourceInfo.timestamp().toEpochMilli()); + + if (sourceInfo.tenant() != null) { + source.put(OceanBaseSourceInfo.TENANT_KEY, sourceInfo.tenant()); + } + if (sourceInfo.database() != null) { + source.put(OceanBaseSourceInfo.DATABASE_NAME_KEY, sourceInfo.database()); + } + if (sourceInfo.tableSchema() != null) { + source.put(OceanBaseSourceInfo.SCHEMA_NAME_KEY, sourceInfo.tableSchema()); + } + if (sourceInfo.transactionId() != null) { + source.put(OceanBaseSourceInfo.TRANSACTION_ID_KEY, sourceInfo.transactionId()); + } + return source; + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseAppendMetadataCollector.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseAppendMetadataCollector.java deleted file mode 100644 index 72cc5794359..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseAppendMetadataCollector.java +++ /dev/null @@ -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, Serializable { - private static final long serialVersionUID = 1L; - - private final OceanBaseMetadataConverter[] metadataConverters; - - public transient OceanBaseRecord inputRecord; - public transient Collector 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 - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseDeserializationSchema.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseDeserializationSchema.java deleted file mode 100644 index f87208c267b..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseDeserializationSchema.java +++ /dev/null @@ -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 The type created by the deserialization schema. - */ -@PublicEvolving -public interface OceanBaseDeserializationSchema extends Serializable, ResultTypeQueryable { - - /** Deserialize the OceanBase record, it is represented in {@link OceanBaseRecord}. */ - void deserialize(OceanBaseRecord record, Collector out) throws Exception; -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMetadataConverter.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMetadataConverter.java deleted file mode 100644 index 953f860c851..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMetadataConverter.java +++ /dev/null @@ -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); -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseReadableMetadata.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseReadableMetadata.java index 021465a717d..0377a6f28f7 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseReadableMetadata.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseReadableMetadata.java @@ -17,37 +17,65 @@ package org.apache.flink.cdc.connectors.oceanbase.table; +import org.apache.flink.cdc.connectors.oceanbase.source.schema.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 Object read(OceanBaseRecord record) { 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 Object read(OceanBaseRecord record) { 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 Object read(OceanBaseRecord record) { 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 DataType getDataType() { return dataType; } - public OceanBaseMetadataConverter getConverter() { + public MetadataConverter getConverter() { return converter; } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseRecord.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseRecord.java deleted file mode 100644 index 716372c29ed..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseRecord.java +++ /dev/null @@ -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 jdbcFields; - private final DataMessage.Record.Type opt; - private final Map logMessageFieldsBefore; - private final Map logMessageFieldsAfter; - - public OceanBaseRecord(SourceInfo sourceInfo, Map 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 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 getJdbcFields() { - return jdbcFields; - } - - public DataMessage.Record.Type getOpt() { - return opt; - } - - public Map getLogMessageFieldsBefore() { - return logMessageFieldsBefore; - } - - public Map 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; - } - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSource.java index 0172e9e6b60..8efe51fc861 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSource.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSource.java @@ -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.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 OceanBaseTableSource( String serverTimeZone, Duration connectTimeout, String hostname, - Integer port, + int port, String compatibleMode, String jdbcDriver, Properties jdbcProperties, @@ -105,30 +110,32 @@ public OceanBaseTableSource( 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 ChangelogMode getChangelogMode() { public ScanRuntimeProvider getScanRuntimeProvider(ScanContext context) { RowType physicalDataType = (RowType) physicalSchema.toPhysicalRowDataType().getLogicalType(); - OceanBaseMetadataConverter[] metadataConverters = getMetadataConverters(); + MetadataConverter[] metadataConverters = getMetadataConverters(); TypeInformation resultTypeInfo = context.createTypeInformation(producedDataType); - RowDataOceanBaseDeserializationSchema deserializer = - RowDataOceanBaseDeserializationSchema.newBuilder() + DebeziumDeserializationSchema 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 builder = OceanBaseSource.builder() - .startupMode(startupMode) + .startupOptions(startupOptions) .username(username) .password(password) .tenantName(tenantName) @@ -178,13 +190,14 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext context) { .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 @@ protected OceanBaseMetadataConverter[] getMetadataConverters() { .findFirst() .orElseThrow(IllegalStateException::new)) .map(OceanBaseReadableMetadata::getConverter) - .toArray(OceanBaseMetadataConverter[]::new); + .toArray(MetadataConverter[]::new); } @Override @@ -217,7 +230,7 @@ public DynamicTableSource copy() { OceanBaseTableSource source = new OceanBaseTableSource( physicalSchema, - startupMode, + startupOptions, username, password, tenantName, @@ -238,7 +251,8 @@ public DynamicTableSource copy() { rsList, configUrl, workingMode, - obcdcProperties); + obcdcProperties, + debeziumProperties); source.metadataKeys = metadataKeys; source.producedDataType = producedDataType; return source; @@ -254,7 +268,7 @@ public boolean equals(Object o) { } 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 boolean equals(Object o) { && 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 boolean equals(Object o) { public int hashCode() { return Objects.hash( physicalSchema, - startupMode, + startupOptions, username, password, tenantName, @@ -306,6 +321,7 @@ public int hashCode() { configUrl, workingMode, obcdcProperties, + debeziumProperties, producedDataType, metadataKeys); } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSourceFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSourceFactory.java index 2de2a4b2c84..14743f05113 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSourceFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSourceFactory.java @@ -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.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 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 USERNAME = ConfigOptions.key("username") .stringType() @@ -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 DynamicTableSource createDynamicTableSource(Context context) { 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 DynamicTableSource createDynamicTableSource(Context context) { return new OceanBaseTableSource( physicalSchema, - startupMode, + startupOptions, username, password, tenantName, @@ -238,7 +240,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { 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 String factoryIdentifier() { @Override public Set> requiredOptions() { Set> 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> optionalOptions() { Set> options = new HashSet<>(); + options.add(SCAN_STARTUP_MODE); options.add(SCAN_STARTUP_TIMESTAMP); options.add(DATABASE_NAME); options.add(TABLE_NAME); options.add(TABLE_LIST); - options.add(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 Set> optionalOptions() { 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); } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/StartupMode.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/StartupMode.java deleted file mode 100644 index a01246b3c8b..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/StartupMode.java +++ /dev/null @@ -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)); - } - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseDeserializationRuntimeConverter.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/OceanBaseUtils.java similarity index 69% rename from flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseDeserializationRuntimeConverter.java rename to flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/OceanBaseUtils.java index f114657051b..7cb738b68b7 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseDeserializationRuntimeConverter.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/OceanBaseUtils.java @@ -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); + } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java index 1aa96221500..c3a4c2ba5d4 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java @@ -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 statements = Arrays.stream( Files.readAllLines(Paths.get(ddlTestFile.toURI())).stream() @@ -203,4 +156,29 @@ protected void initializeTable(String sqlFile) { 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 expected, List actual) { + assertTrue(expected != null && actual != null); + assertTrue( + String.format("expected: %s, actual: %s", expected, actual), + actual.containsAll(expected)); + } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseConnectorITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMySQLModeITCase.java similarity index 59% rename from flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseConnectorITCase.java rename to flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMySQLModeITCase.java index 8f6e0d9a2ac..4388b60af5e 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseConnectorITCase.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMySQLModeITCase.java @@ -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 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 void testTableList() throws Exception { + " 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 void testTableList() throws Exception { 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 void testTableList() throws Exception { @Test public void testMetadataColumns() throws Exception { - initializeTable("inventory_meta"); + initializeTable("inventory"); String sourceDDL = String.format( @@ -211,31 +272,13 @@ public void testMetadataColumns() throws Exception { + " 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 void testMetadataColumns() throws Exception { 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 void testMetadataColumns() throws Exception { List 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 actual = TestValuesTableFactory.getRawResults("sink"); assertContainsInAnyOrder(expected, actual); result.getJobClient().get().cancel().get(); @@ -309,11 +352,9 @@ public void testMetadataColumns() throws Exception { @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 void testAllDataTypes() throws Exception { + " 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 void testAllDataTypes() throws Exception { + " 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 void testAllDataTypes() throws Exception { + " 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 void testAllDataTypes() throws Exception { 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 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 actual = TestValuesTableFactory.getRawResults("sink"); assertContainsInAnyOrder(expected, actual); @@ -475,11 +540,9 @@ public void testTimezoneShanghai() throws Exception { } 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 void testTimeDataTypes(String serverTimeZone) throws Exception { + " 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 void testTimeDataTypes(String serverTimeZone) throws Exception { 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 void testTimeDataTypes(String serverTimeZone) throws Exception { 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 expected, List 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 expected = + Arrays.asList( + "+I(101,scooter,Small 2-wheel scooter,3.1400000000)", + "+I(102,car battery,12V car battery,8.1000000000)", + "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8000000000)", + "+I(104,hammer,12oz carpenter's hammer,0.7500000000)", + "+I(105,hammer,14oz carpenter's hammer,0.8750000000)", + "+I(106,hammer,16oz carpenter's hammer,1.0000000000)", + "+I(107,rocks,box of assorted rocks,5.3000000000)", + "+I(108,jacket,water resistent black wind breaker,0.1000000000)", + "+I(109,spare tire,24 inch spare tire,22.2000000000)"); + List actual = TestValuesTableFactory.getRawResults("sink"); + assertContainsInAnyOrder(expected, actual); } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseOracleModeITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseOracleModeITCase.java new file mode 100644 index 00000000000..b11da43aec9 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseOracleModeITCase.java @@ -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 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 expected = + Arrays.asList( + "+I(1,vc2,vc2,nvc2,c ,nc ,1.1,2.22,3.33,8.888,4.444400,5.555,6.66,1234.567891,1234.567891,77.323,1,22,333,4444,5555,true,99,9999,999999999,999999999999999999,90,9900,999999990,999999999999999900,99999999999999999999999999999999999900,2022-10-30T00:00,2022-10-30T12:34:56.007890,2022-10-30T12:34:56.130,2022-10-30T12:34:56.125500,2022-10-30T12:34:56.125457,col_clob,col_blob)", + "+U(1,vc2,vc2,nvc2,c ,nc ,1.1,2.22,3.33,8.888,4.444400,5.555,6.66,1234.567891,1234.567891,77.323,1,22,333,4444,5555,true,99,9999,999999999,999999999999999999,90,9900,999999990,999999999999999900,99999999999999999999999999999999999900,2022-10-30T00:00,2022-10-30T12:34:56.125450,2022-10-30T12:34:56.130,2022-10-30T12:34:56.125500,2022-10-30T12:34:56.125457,col_clob,col_blob)"); + + List actual = TestValuesTableFactory.getRawResults("sink"); + assertContainsInAnyOrder(expected, actual); + result.getJobClient().get().cancel().get(); + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableFactoryTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableFactoryTest.java index f759a902bff..9861df5dce3 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableFactoryTest.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableFactoryTest.java @@ -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; @@ -104,7 +105,7 @@ public void testCommonProperties() { OceanBaseTableSource expectedSource = new OceanBaseTableSource( SCHEMA, - StartupMode.LATEST_OFFSET, + StartupOptions.latest(), USERNAME, PASSWORD, TENANT_NAME, @@ -113,8 +114,8 @@ public void testCommonProperties() { 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 void testCommonProperties() { RS_LIST, null, WORKING_MODE, + new Properties(), new Properties()); assertEquals(expectedSource, actualSource); } @@ -136,8 +138,6 @@ public void testOptionalProperties() { 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 void testOptionalProperties() { OceanBaseTableSource expectedSource = new OceanBaseTableSource( SCHEMA, - StartupMode.INITIAL, + StartupOptions.initial(), USERNAME, PASSWORD, TENANT_NAME, @@ -168,6 +168,7 @@ public void testOptionalProperties() { RS_LIST, null, WORKING_MODE, + new Properties(), new Properties()); assertEquals(expectedSource, actualSource); } @@ -190,7 +191,7 @@ public void testMetadataColumns() { OceanBaseTableSource expectedSource = new OceanBaseTableSource( SCHEMA_WITH_METADATA, - StartupMode.LATEST_OFFSET, + StartupOptions.latest(), USERNAME, PASSWORD, TENANT_NAME, @@ -199,8 +200,8 @@ public void testMetadataColumns() { 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 void testMetadataColumns() { RS_LIST, null, WORKING_MODE, + new Properties(), new Properties()); expectedSource.producedDataType = SCHEMA_WITH_METADATA.toSourceRowDataType(); expectedSource.metadataKeys = @@ -240,6 +242,8 @@ private Map getRequiredOptions() { 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)); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/inventory_meta.sql b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/inventory_meta.sql deleted file mode 100644 index 85b40fa8010..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/inventory_meta.sql +++ /dev/null @@ -1,42 +0,0 @@ --- Licensed to the Apache Software Foundation (ASF) under one or more --- contributor license agreements. See the NOTICE file distributed with --- this work for additional information regarding copyright ownership. --- The ASF licenses this file to You under the Apache License, Version 2.0 --- (the "License"); you may not use this file except in compliance with --- the License. You may obtain a copy of the License at --- --- http://www.apache.org/licenses/LICENSE-2.0 --- --- Unless required by applicable law or agreed to in writing, software --- distributed under the License is distributed on an "AS IS" BASIS, --- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. --- See the License for the specific language governing permissions and --- limitations under the License. - --- ---------------------------------------------------------------------------------------------------------------- --- 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); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/column_type_test.sql b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/column_type_test.sql similarity index 97% rename from flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/column_type_test.sql rename to flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/column_type_test.sql index 98ac5255cc0..2840c9f83de 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/column_type_test.sql +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/column_type_test.sql @@ -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, diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/docker_init.sql b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/docker_init.sql new file mode 100644 index 00000000000..0db9c71db4a --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/docker_init.sql @@ -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'; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/inventory.sql b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/inventory.sql similarity index 95% rename from flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/inventory.sql rename to flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/inventory.sql index bb977ad7158..2252ee19573 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/inventory.sql +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/inventory.sql @@ -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, diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/oracle/column_type_test.sql b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/oracle/column_type_test.sql new file mode 100644 index 00000000000..bfe622cfb10 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/oracle/column_type_test.sql @@ -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') +); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-sql-connector-oceanbase-cdc/pom.xml b/flink-cdc-connect/flink-cdc-source-connectors/flink-sql-connector-oceanbase-cdc/pom.xml index fe142db585d..df736950733 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-sql-connector-oceanbase-cdc/pom.xml +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-sql-connector-oceanbase-cdc/pom.xml @@ -55,6 +55,7 @@ limitations under the License. io.debezium:debezium-api io.debezium:debezium-embedded io.debezium:debezium-core + com.ververica:flink-cdc-base com.ververica:flink-connector-debezium com.ververica:flink-connector-oceanbase-cdc mysql:mysql-connector-java