From 0237401ee02ff142333fd78dd5b68242380c9a7d Mon Sep 17 00:00:00 2001 From: linjc13 Date: Fri, 18 Apr 2025 17:05:46 +0800 Subject: [PATCH] [FLINK-36796][pipeline-connector][oracle]add oracle pipeline connector. --- .../connectors/pipeline-connectors/oracle.md | 287 ++++++ .../connectors/pipeline-connectors/oracle.md | 291 ++++++ .../pom.xml | 266 +++++ .../cdc/connectors/oracle/dto/ColumnInfo.java | 69 ++ .../factory/OracleDataSourceFactory.java | 245 +++++ .../oracle/source/OracleDataSource.java | 266 +++++ .../source/OracleDataSourceOptions.java | 373 +++++++ .../source/OracleEventDeserializer.java | 175 ++++ .../oracle/source/OracleMetadataAccessor.java | 88 ++ .../source/OracleSchemaDataTypeInference.java | 45 + .../source/parser/BaseParserListener.java | 96 ++ .../ColumnDefinitionParserListener.java | 312 ++++++ .../OracleAlterTableParserListener.java | 410 ++++++++ .../source/parser/OracleAntlrDdlParser.java | 55 ++ .../parser/OracleAntlrDdlParserListener.java | 70 ++ .../reader/OracleDebeziumSourceFunction.java | 124 +++ .../reader/OraclePipelineRecordEmitter.java | 91 ++ .../source/reader/OracleSourceReader.java | 210 ++++ .../reader/OracleTableSourceReader.java | 49 + .../oracle/utils/DebeziumUtils.java | 63 ++ .../oracle/utils/OracleSchemaUtils.java | 232 +++++ .../oracle/utils/OracleTypeUtils.java | 133 +++ ....apache.flink.cdc.common.factories.Factory | 15 + .../source/OracleDataSourceFactoryTest.java | 121 +++ .../oracle/source/OracleFullTypesITCase.java | 338 +++++++ .../source/OracleMetadataAccessorITCase.java | 224 +++++ .../oracle/source/OraclePipelineITCase.java | 910 ++++++++++++++++++ .../oracle/source/OracleSourceITCase.java | 741 ++++++++++++++ .../oracle/source/OracleSourceTestBase.java | 196 ++++ .../oracle/testutils/OracleTestUtils.java | 124 +++ .../oracle/testutils/RecordsFormatter.java | 101 ++ .../oracle/testutils/TestTable.java | 99 ++ .../containers/OracleContainer.java | 239 +++++ .../test/resources/ddl/column_type_test.sql | 83 ++ .../src/test/resources/ddl/customer.sql | 80 ++ .../src/test/resources/ddl/product.sql | 53 + .../src/test/resources/log4j2-test.properties | 26 + .../flink-cdc-pipeline-connectors/pom.xml | 1 + .../base/source/IncrementalSource.java | 22 +- .../IncrementalSourceRecordEmitter.java | 23 +- .../DebeziumEventDeserializationSchema.java | 3 +- .../DebeziumSchemaDataTypeInference.java | 3 + .../oracle/source/OracleSourceBuilder.java | 12 + 43 files changed, 7351 insertions(+), 13 deletions(-) create mode 100644 docs/content.zh/docs/connectors/pipeline-connectors/oracle.md create mode 100644 docs/content/docs/connectors/pipeline-connectors/oracle.md create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/pom.xml create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/dto/ColumnInfo.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/factory/OracleDataSourceFactory.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleDataSource.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleDataSourceOptions.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleEventDeserializer.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleMetadataAccessor.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleSchemaDataTypeInference.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/parser/BaseParserListener.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/parser/ColumnDefinitionParserListener.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/parser/OracleAlterTableParserListener.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/parser/OracleAntlrDdlParser.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/parser/OracleAntlrDdlParserListener.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/reader/OracleDebeziumSourceFunction.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/reader/OraclePipelineRecordEmitter.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/reader/OracleSourceReader.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/reader/OracleTableSourceReader.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/utils/DebeziumUtils.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/utils/OracleSchemaUtils.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/utils/OracleTypeUtils.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/resources/META-INF/services/org.apache.flink.cdc.common.factories.Factory create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleDataSourceFactoryTest.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleFullTypesITCase.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleMetadataAccessorITCase.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OraclePipelineITCase.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceITCase.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceTestBase.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/testutils/OracleTestUtils.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/testutils/RecordsFormatter.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/testutils/TestTable.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/testcontainers/containers/OracleContainer.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/resources/ddl/column_type_test.sql create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/resources/ddl/customer.sql create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/resources/ddl/product.sql create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/resources/log4j2-test.properties diff --git a/docs/content.zh/docs/connectors/pipeline-connectors/oracle.md b/docs/content.zh/docs/connectors/pipeline-connectors/oracle.md new file mode 100644 index 00000000000..5bec69aafbd --- /dev/null +++ b/docs/content.zh/docs/connectors/pipeline-connectors/oracle.md @@ -0,0 +1,287 @@ +--- +title: "ORACLE" +weight: 2 +type: docs +aliases: +- /connectors/pipeline-connectors/oracle +--- + + +# Oracle Connector + +Oracle CDC Pipeline 连接器允许从 Oracle 数据库读取快照数据和增量数据,并提供端到端的整库数据同步能力。 本文描述了如何设置 Oracle CDC Pipeline 连接器。 + + +## 示例 + +从 Oracle 读取数据同步到 Doris 的 Pipeline 可以定义如下: + +```yaml +source: + type: oracle + name: Oracle Source + hostname: 127.0.0.1 + port: 1521 + username: debezium + password: dbz + database: ORCLDB + tables: adb.\.*, bdb.user_table_[0-9]+, [app|web].order_\.* + +sink: + type: doris + name: Doris Sink + fenodes: 127.0.0.1:8030 + username: root + password: pass + +pipeline: + name: Oracle to Doris Pipeline + parallelism: 4 +``` + +## 连接器配置项 + +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
OptionRequiredDefaultTypeDescription
hostnamerequired(none)String Oracle 数据库服务器的 IP 地址或主机名。
portoptional1521IntegerOracle 数据库服务器的整数端口号。
usernamerequired(none)String连接到 Oracle 数据库服务器时要使用的 Oracle 用户的名称。
passwordrequired(none)String连接 Oracle 数据库服务器时使用的密码。
tablesrequired(none)String需要监视的 Oracle 数据库的表名。表名支持正则表达式,以监视满足正则表达式的多个表。
+ 需要注意的是,点号(.)被视为数据库和表名的分隔符。 如果需要在正则表达式中使用点(.)来匹配任何字符,必须使用反斜杠对点进行转义。
+ 例如,db0.\.*, db1.user_table_[0-9]+, db[1-2].[app|web]order_\.*
schema-change.enabledoptionaltrueBoolean是否发送模式更改事件,下游 sink 可以响应模式变更事件实现表结构同步,默认为true。
scan.incremental.snapshot.chunk.sizeoptional8096Integer表快照的块大小(行数),读取表的快照时,捕获的表被拆分为多个块。
scan.snapshot.fetch.sizeoptional1024Integer读取表快照时每次读取数据的最大条数。
scan.startup.modeoptionalinitialString Oracle CDC 消费者可选的启动模式, + 合法的模式为 "initial","latest-offset"。
debezium.*optional(none)String将 Debezium 的属性传递给 Debezium 嵌入式引擎,该引擎用于从 Oracle 服务器捕获数据更改。 + 例如: 'debezium.snapshot.mode' = 'never'. + 查看更多关于 Debezium 的 Oracle 连接器属性
scan.incremental.close-idle-reader.enabledoptionalfalseBoolean是否在快照结束后关闭空闲的 Reader。 此特性需要 flink 版本大于等于 1.14 并且 'execution.checkpointing.checkpoints-after-tasks-finish.enabled' 需要设置为 true。
+ 若 flink 版本大于等于 1.15,'execution.checkpointing.checkpoints-after-tasks-finish.enabled' 默认值变更为 true,可以不用显式配置 'execution.checkpointing.checkpoints-after-tasks-finish.enabled' = true。
metadata.listoptionalfalseString + 可额外读取的SourceRecord中元数据的列表,后续可直接使用在transform模块,英文逗号 `,` 分割。目前可用值包含:op_ts。 +
+
+ +## 启动模式 + +配置选项`scan.startup.mode`指定 Oracle CDC 使用者的启动模式。有效枚举包括: +- `initial` (默认):在第一次启动时对受监视的数据库表执行初始快照,并继续读取最新的归档日志。 +- `latest-offset`:首次启动时,从不对受监视的数据库表执行快照, 连接器仅从归档日志的结尾处开始读取,这意味着连接器只能读取在连接器启动之后的数据更改。 + +例如,可以在 YAML 配置文件中这样指定启动模式: + +```yaml +source: + type: oracle + scan.startup.mode: earliest-offset # Start from earliest offset + scan.startup.mode: latest-offset # Start from latest offset + # ... +``` + +## 数据类型映射 + +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Oracle typeCDC typeNOTE
NUMBER(p,s)DECIMAL(p, s)/BIGINT当s 大于 0 时,使用 DECIMAL(p, s);否则使用 BIGINT。
+ LONG
+
BIGINT
+ DATE + TIMESTAMP [(6)]
+ FLOAT
+ BINARY_FLOAT
+
FLOAT
+ BINARY_DOUBLE
+ DOUBLE +
DOUBLE
+ TIMESTAMP(p) + TIMESTAMP [(p)]
+ TIMESTAMP(p) WITH TIME ZONE + TIMESTAMP_TZ [(p)]
+ TIMESTAMP(p) WITH LOCAL TIME ZONE + TIMESTAMP_LTZ [(p)]
+ INTERVAL YEAR(2) TO MONTH
+ INTERVAL DAY(3) TO SECOND(2) +
BIGINT
+ VARCHAR(n)
+ VARCHAR2(n)
+ NVARCHAR2(n)
+ NCHAR(n)
+ CHAR(n)
+
VARCHAR(n)
+ CLOB
+ BLOB
+ TEXT
+ NCLOB
+ SDO_GEOMETRY
+ XMLTYPE +
STRING目前,对于 Oracle 中的 BLOB 数据类型,仅支持长度不大于 2147483647(2**31-1)的 blob。
+
+{{< top >}} diff --git a/docs/content/docs/connectors/pipeline-connectors/oracle.md b/docs/content/docs/connectors/pipeline-connectors/oracle.md new file mode 100644 index 00000000000..9dd286a731a --- /dev/null +++ b/docs/content/docs/connectors/pipeline-connectors/oracle.md @@ -0,0 +1,291 @@ +--- +title: "ORACLE" +weight: 2 +type: docs +aliases: +- /connectors/pipeline-connectors/oracle +--- + + +# Oracle Connector + +Oracle connector allows reading snapshot data and incremental data from Oracle database and provides end-to-end full-database data synchronization capabilities. +This document describes how to setup the Oracle connector. + + +## Example + +An example of the pipeline for reading data from Oracle and sink to Doris can be defined as follows: + +```yaml +source: + type: oracle + name: Oracle Source + hostname: 127.0.0.1 + port: 1521 + username: debezium + password: dbz + database: ORCLDB + tables: adb.\.*, bdb.user_table_[0-9]+, [app|web].order_\.* + +sink: + type: doris + name: Doris Sink + fenodes: 127.0.0.1:8030 + username: root + password: pass + +pipeline: + name: Oracle to Doris Pipeline + parallelism: 4 +``` + +## Connector Options + +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
OptionRequiredDefaultTypeDescription
hostnamerequired(none)String IP address or hostname of the Oracle database server.
portoptional1521IntegerInteger port number of the Oracle database server.
usernamerequired(none)StringName of the Oracle database to use when connecting to the Oracle database server.
passwordrequired(none)StringPassword to use when connecting to the Oracle database server.
tablesrequired(none)StringTable name of the Oracle database to monitor. The table-name also supports regular expressions to monitor multiple tables that satisfy the regular expressions.
+ It is important to note that the dot (.) is treated as a delimiter for database and table names. + If there is a need to use a dot (.) in a regular expression to match any character, it is necessary to escape the dot with a backslash.
+ eg. db0.\.*, db1.user_table_[0-9]+, db[1-2].[app|web]order_\.*
schema-change.enabledoptionaltrueBooleanWhether to send schema change events, so that downstream sinks can respond to schema changes and achieve table structure synchronization.
scan.incremental.snapshot.chunk.sizeoptional8096IntegerThe chunk size (number of rows) of table snapshot, captured tables are split into multiple chunks when read the snapshot of table.
scan.snapshot.fetch.sizeoptional1024IntegerThe maximum fetch size for per poll when read table snapshot.
scan.startup.modeoptionalinitialStringOptional startup mode for Oracle CDC consumer, valid enumerations are "initial","latest-offset".
debezium.*optional(none)StringPass-through Debezium's properties to Debezium Embedded Engine which is used to capture data changes from Oracle server. + For example: 'debezium.snapshot.mode' = 'never'. + See more about the Debezium's Oracle Connector properties
scan.incremental.close-idle-reader.enabledoptionalfalseBooleanWhether to close idle readers at the end of the snapshot phase.
+ The flink version is required to be greater than or equal to 1.14 when 'execution.checkpointing.checkpoints-after-tasks-finish.enabled' is set to true.
+ If the flink version is greater than or equal to 1.15, the default value of 'execution.checkpointing.checkpoints-after-tasks-finish.enabled' has been changed to true, + so it does not need to be explicitly configured 'execution.checkpointing.checkpoints-after-tasks-finish.enabled' = 'true'
metadata.listoptionalfalseString + List of readable metadata from SourceRecord to be passed to downstream and could be used in transform module, split by `,`. Available readable metadata are: op_ts. +
+
+ +## Startup Reading Position + +The config option `scan.startup.mode` specifies the startup mode for Oracle CDC consumer. The valid enumerations are: + +- `initial` (default): Performs an initial snapshot on the monitored database tables upon first startup, and continue to read the latest archive log. +- `latest-offset`: On first startup, no snapshots of the monitored database tables are performed, and the connector only starts reading from the end of the archive log, which means the connector can only read data changes after the connector starts. + +For example in YAML definition: + +```yaml +source: + type: oracle + scan.startup.mode: earliest-offset # Start from earliest offset + scan.startup.mode: latest-offset # Start from latest offset + # ... +``` + +## Data Type Mapping + +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Oracle typeCDC typeNOTE
NUMBER(p,s)DECIMAL(p, s)/BIGINTWhen s is greater than 0, use DECIMAL (p, s); Otherwise, use BIGINT。
+ LONG
+
BIGINT
+ DATE + TIMESTAMP [(6)]
+ FLOAT
+ BINARY_FLOAT
+
FLOAT
+ BINARY_DOUBLE
+ DOUBLE +
DOUBLE
+ TIMESTAMP(p) + TIMESTAMP [(p)]
+ TIMESTAMP(p) WITH TIME ZONE + TIMESTAMP_TZ [(p)]
+ TIMESTAMP(p) WITH LOCAL TIME ZONE + TIMESTAMP_LTZ [(p)]
+ INTERVAL YEAR(2) TO MONTH
+ INTERVAL DAY(3) TO SECOND(2) +
BIGINT
+ VARCHAR(n)
+ VARCHAR2(n)
+ NVARCHAR2(n)
+ NCHAR(n)
+ CHAR(n)
+
VARCHAR(n)
+ CLOB
+ BLOB
+ TEXT
+ NCLOB
+ SDO_GEOMETRY
+ XMLTYPE +
STRINGCurrently, only blob data types with a length not exceeding 2147483647 (2 * * 31-1) are supported in Oracle.
+
+{{< top >}} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/pom.xml new file mode 100644 index 00000000000..7adc7309697 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/pom.xml @@ -0,0 +1,266 @@ + + + + + + org.apache.flink + flink-cdc-pipeline-connectors + ${revision} + + 4.0.0 + + flink-cdc-pipeline-connector-oracle + + + + + + + io.debezium + debezium-core + ${debezium.version} + + + io.debezium + debezium-ddl-parser + ${debezium.version} + + + debezium-core + io.debezium + + + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${flink.version} + test + + + + org.apache.flink + flink-table-runtime + ${flink.version} + test + + + + org.apache.flink + flink-test-utils + ${flink.version} + test + + + + org.apache.flink + flink-connector-test-utils + ${flink.version} + test + + + + org.apache.flink + flink-core + ${flink.version} + test-jar + test + + + + org.apache.flink + flink-streaming-java + ${flink.version} + test-jar + test + + + + org.apache.flink + flink-table-common + ${flink.version} + test-jar + test + + + + org.apache.flink + flink-tests + ${flink.version} + test-jar + test + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${flink.version} + test-jar + test + + + + + + org.testcontainers + oracle-xe + ${testcontainers.version} + test + + + + com.esri.geometry + esri-geometry-api + ${geometry.version} + + + com.fasterxml.jackson.core + jackson-core + + + + + org.apache.flink + flink-connector-oracle-cdc + ${parent.version} + + + + com.jayway.jsonpath + json-path + ${json-path.version} + test + + + org.apache.flink + flink-connector-test-util + ${parent.version} + test + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.2.4 + + + shade-flink + package + + shade + + + false + + + io.debezium:debezium-api + io.debezium:debezium-embedded + io.debezium:debezium-core + io.debezium:debezium-ddl-parser + io.debezium:debezium-connector-oracle + org.apache.flink:flink-connector-debezium + org.apache.flink:flink-connector-oracle-cdc + org.apache.flink:flink-cdc-base + com.oracle.ojdbc:ojdbc8 + cn.easyproject:orai18n + org.antlr:antlr4-runtime + org.apache.kafka:* + com.fasterxml.*:* + com.google.guava:* + com.esri.geometry:esri-geometry-api + com.zaxxer:HikariCP + + org.apache.flink:flink-shaded-guava + + + + + org.apache.kafka:* + + kafka/kafka-version.properties + LICENSE + + NOTICE + common/** + + + + + + org.apache.kafka + + org.apache.flink.cdc.connectors.shaded.org.apache.kafka + + + + org.antlr + + org.apache.flink.cdc.connectors.shaded.org.antlr + + + + com.fasterxml + + org.apache.flink.cdc.connectors.shaded.com.fasterxml + + + + com.google + + org.apache.flink.cdc.connectors.shaded.com.google + + + + com.esri.geometry + org.apache.flink.cdc.connectors.shaded.com.esri.geometry + + + com.zaxxer + + org.apache.flink.cdc.connectors.shaded.com.zaxxer + + + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + test-jar + + test-jar + + + + + + + + \ No newline at end of file diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/dto/ColumnInfo.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/dto/ColumnInfo.java new file mode 100644 index 00000000000..84c3d540ed2 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/dto/ColumnInfo.java @@ -0,0 +1,69 @@ +/* + * 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.oracle.dto; + +import java.io.Serializable; + +/** dto related to Table schema. */ +public class ColumnInfo implements Serializable { + private String columnName; + private String dataType; + private Integer dataLength; + private Integer dataPrecision; + private Integer dataScale; + + public Integer getDataLength() { + return dataLength; + } + + public void setDataLength(Integer dataLength) { + this.dataLength = dataLength; + } + + public String getColumnName() { + return columnName; + } + + public void setColumnName(String columnName) { + this.columnName = columnName; + } + + public String getDataType() { + return dataType; + } + + public void setDataType(String dataType) { + this.dataType = dataType; + } + + public Integer getDataPrecision() { + return dataPrecision; + } + + public void setDataPrecision(Integer dataPrecision) { + this.dataPrecision = dataPrecision; + } + + public Integer getDataScale() { + return dataScale; + } + + public void setDataScale(Integer dataScale) { + this.dataScale = dataScale; + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/factory/OracleDataSourceFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/factory/OracleDataSourceFactory.java new file mode 100644 index 00000000000..b383d0eae11 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/factory/OracleDataSourceFactory.java @@ -0,0 +1,245 @@ +/* + * 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.oracle.factory; + +import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.cdc.common.configuration.ConfigOption; +import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.factories.DataSourceFactory; +import org.apache.flink.cdc.common.schema.Selectors; +import org.apache.flink.cdc.common.source.DataSource; +import org.apache.flink.cdc.common.utils.StringUtils; +import org.apache.flink.cdc.connectors.oracle.source.OracleDataSource; +import org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions; +import org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfig; +import org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfigFactory; +import org.apache.flink.cdc.connectors.oracle.table.OracleReadableMetaData; +import org.apache.flink.cdc.connectors.oracle.utils.OracleSchemaUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.flink.cdc.connectors.base.utils.ObjectUtils.doubleCompare; +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.METADATA_LIST; +import static org.apache.flink.util.Preconditions.checkState; + +/** A {@link Factory} to create {@link OracleDataSource}. */ +@Internal +public class OracleDataSourceFactory implements DataSourceFactory { + + private static final Logger LOG = LoggerFactory.getLogger(OracleDataSourceFactory.class); + + public static final String IDENTIFIER = "oracle"; + private String[] capturedTables; + + @Override + public DataSource createDataSource(Context context) { + final Configuration config = context.getFactoryConfiguration(); + int fetchSize = config.get(OracleDataSourceOptions.SCAN_SNAPSHOT_FETCH_SIZE); + int splitSize = config.get(OracleDataSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE); + int splitMetaGroupSize = config.get(OracleDataSourceOptions.CHUNK_META_GROUP_SIZE); + + double distributionFactorUpper = + config.get(OracleDataSourceOptions.CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND); + double distributionFactorLower = + config.get(OracleDataSourceOptions.CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND); + int connectMaxRetries = config.get(OracleDataSourceOptions.CONNECT_MAX_RETRIES); + int connectionPoolSize = config.get(OracleDataSourceOptions.CONNECTION_POOL_SIZE); + String tables = config.get(OracleDataSourceOptions.TABLES); + validateIntegerOption( + OracleDataSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE, splitSize, 1); + validateIntegerOption(OracleDataSourceOptions.CHUNK_META_GROUP_SIZE, splitMetaGroupSize, 1); + validateIntegerOption(OracleDataSourceOptions.SCAN_SNAPSHOT_FETCH_SIZE, fetchSize, 1); + validateIntegerOption(OracleDataSourceOptions.CONNECTION_POOL_SIZE, connectionPoolSize, 1); + validateIntegerOption(OracleDataSourceOptions.CONNECT_MAX_RETRIES, connectMaxRetries, 0); + validateDistributionFactorUpper(distributionFactorUpper); + validateDistributionFactorLower(distributionFactorLower); + OracleSourceConfigFactory configFactory = + (OracleSourceConfigFactory) + new OracleSourceConfigFactory() + .hostname(config.get(OracleDataSourceOptions.HOSTNAME)) + .port(config.get(OracleDataSourceOptions.PORT)) + .databaseList( + config.get( + OracleDataSourceOptions + .DATABASE)) // monitor oracledatabase + .tableList( + config.get( + OracleDataSourceOptions + .TABLES)) // monitor productstable + .username(config.get(OracleDataSourceOptions.USERNAME)) + .password(config.get(OracleDataSourceOptions.PASSWORD)) + .includeSchemaChanges(true); + Selectors selectors = new Selectors.SelectorsBuilder().includeTables(tables).build(); + String[] capturedTables = getTableList(configFactory.create(0), selectors); + if (capturedTables.length == 0) { + throw new IllegalArgumentException( + "Cannot find any table by the option 'tables' = " + tables); + } + configFactory.tableList(capturedTables); + configFactory.databaseList(config.get(OracleDataSourceOptions.DATABASE)); + configFactory.schemaList(new String[] {config.get(OracleDataSourceOptions.SCHEMALIST)}); + String metadataList = config.get(METADATA_LIST); + List readableMetadataList = listReadableMetadata(metadataList); + return new OracleDataSource(configFactory, config, capturedTables, readableMetadataList); + } + + public static List listReadableMetadata(String metadataList) { + if (StringUtils.isNullOrWhitespaceOnly(metadataList)) { + return new ArrayList<>(); + } + Set readableMetadataList = + Arrays.stream(metadataList.split(",")) + .map(String::trim) + .collect(Collectors.toSet()); + List foundMetadata = new ArrayList<>(); + for (OracleReadableMetaData metadata : OracleReadableMetaData.values()) { + if (readableMetadataList.contains(metadata.getKey())) { + foundMetadata.add(metadata); + readableMetadataList.remove(metadata.getKey()); + } + } + if (readableMetadataList.isEmpty()) { + return foundMetadata; + } + throw new IllegalArgumentException( + String.format( + "[%s] cannot be found in oracle metadata.", + String.join(", ", readableMetadataList))); + } + + @Override + public Set> requiredOptions() { + Set> options = new HashSet<>(); + options.add(OracleDataSourceOptions.HOSTNAME); + options.add(OracleDataSourceOptions.PORT); + options.add(OracleDataSourceOptions.USERNAME); + options.add(OracleDataSourceOptions.PASSWORD); + options.add(OracleDataSourceOptions.SCHEMALIST); + options.add(OracleDataSourceOptions.DATABASE); + options.add(OracleDataSourceOptions.TABLES); + options.add(OracleDataSourceOptions.METADATA_LIST); + return options; + } + + @Override + public Set> optionalOptions() { + Set> options = new HashSet<>(); + options.add(OracleDataSourceOptions.SERVER_TIME_ZONE); + options.add(OracleDataSourceOptions.SERVER_ID); + options.add(OracleDataSourceOptions.SCAN_STARTUP_MODE); + options.add(OracleDataSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_FILE); + options.add(OracleDataSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_POS); + options.add(OracleDataSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_GTID_SET); + options.add(OracleDataSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_SKIP_EVENTS); + options.add(OracleDataSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_SKIP_ROWS); + options.add(OracleDataSourceOptions.SCAN_STARTUP_TIMESTAMP_MILLIS); + options.add(OracleDataSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE); + options.add(OracleDataSourceOptions.CHUNK_META_GROUP_SIZE); + options.add(OracleDataSourceOptions.SCAN_SNAPSHOT_FETCH_SIZE); + options.add(OracleDataSourceOptions.CONNECT_TIMEOUT); + options.add(OracleDataSourceOptions.CONNECTION_POOL_SIZE); + options.add(OracleDataSourceOptions.CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND); + options.add(OracleDataSourceOptions.CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND); + options.add(OracleDataSourceOptions.CONNECT_MAX_RETRIES); + options.add(OracleDataSourceOptions.SCAN_INCREMENTAL_CLOSE_IDLE_READER_ENABLED); + options.add(OracleDataSourceOptions.HEARTBEAT_INTERVAL); + options.add(OracleDataSourceOptions.SCHEMA_CHANGE_ENABLED); + options.add(OracleDataSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP); + options.add(OracleDataSourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND); + options.add(OracleDataSourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND); + options.add(OracleDataSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_ENABLED); + options.add(OracleDataSourceOptions.SNAPSHOT_LOCKING_MODE); + options.add(OracleDataSourceOptions.HISTORY_CAPTURED_TABLES_DDL_ENABLE); + options.add(OracleDataSourceOptions.LOG_MINING_CONTINUOUS_MINE); + options.add(OracleDataSourceOptions.LOG_MINING_STRATEGY); + options.add(OracleDataSourceOptions.DATABASE_CONNECTION_ADAPTER); + options.add(OracleDataSourceOptions.DATABASE_TABLE_CASE_INSENSITIVE); + return options; + } + + @Override + public String identifier() { + return IDENTIFIER; + } + + private static String[] getTableList(OracleSourceConfig sourceConfig, Selectors selectors) { + return OracleSchemaUtils.listTables(sourceConfig, null).stream() + .filter(selectors::isMatch) + .map(TableId::toString) + .toArray(String[]::new); + } + + /** Checks the value of given integer option is valid. */ + private void validateIntegerOption( + ConfigOption option, int optionValue, int exclusiveMin) { + checkState( + optionValue > exclusiveMin, + String.format( + "The value of option '%s' must larger than %d, but is %d", + option.key(), exclusiveMin, optionValue)); + } + + /** Checks the value of given evenly distribution factor upper bound is valid. */ + private void validateDistributionFactorUpper(double distributionFactorUpper) { + checkState( + doubleCompare(distributionFactorUpper, 1.0d) >= 0, + String.format( + "The value of option '%s' must larger than or equals %s, but is %s", + OracleDataSourceOptions.CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND + .key(), + 1.0d, + distributionFactorUpper)); + } + + /** Checks the value of given evenly distribution factor lower bound is valid. */ + private void validateDistributionFactorLower(double distributionFactorLower) { + checkState( + doubleCompare(distributionFactorLower, 0.0d) >= 0 + && doubleCompare(distributionFactorLower, 1.0d) <= 0, + String.format( + "The value of option '%s' must between %s and %s inclusively, but is %s", + OracleDataSourceOptions.CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND + .key(), + 0.0d, + 1.0d, + distributionFactorLower)); + } + + /** Replaces the default timezone placeholder with session timezone, if applicable. */ + private static ZoneId getServerTimeZone(Configuration config) { + final String serverTimeZone = config.get(OracleDataSourceOptions.SERVER_TIME_ZONE); + if (serverTimeZone != null) { + return ZoneId.of(serverTimeZone); + } else { + LOG.warn( + "{} is not set, which might cause data inconsistencies for time-related fields.", + OracleDataSourceOptions.SERVER_TIME_ZONE.key()); + return ZoneId.systemDefault(); + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleDataSource.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleDataSource.java new file mode 100644 index 00000000000..9468e3a4931 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleDataSource.java @@ -0,0 +1,266 @@ +/* + * 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.oracle.source; + +import org.apache.flink.cdc.common.annotation.VisibleForTesting; +import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.source.DataSource; +import org.apache.flink.cdc.common.source.EventSourceProvider; +import org.apache.flink.cdc.common.source.FlinkSourceFunctionProvider; +import org.apache.flink.cdc.common.source.FlinkSourceProvider; +import org.apache.flink.cdc.common.source.MetadataAccessor; +import org.apache.flink.cdc.connectors.base.options.StartupOptions; +import org.apache.flink.cdc.connectors.base.source.jdbc.JdbcIncrementalSource; +import org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfig; +import org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfigFactory; +import org.apache.flink.cdc.connectors.oracle.source.meta.offset.RedoLogOffsetFactory; +import org.apache.flink.cdc.connectors.oracle.source.reader.OraclePipelineRecordEmitter; +import org.apache.flink.cdc.connectors.oracle.source.reader.OracleSourceReader; +import org.apache.flink.cdc.connectors.oracle.source.reader.OracleTableSourceReader; +import org.apache.flink.cdc.connectors.oracle.table.OracleReadableMetaData; +import org.apache.flink.cdc.debezium.DebeziumSourceFunction; +import org.apache.flink.cdc.debezium.table.DebeziumChangelogMode; +import org.apache.flink.cdc.debezium.table.MetadataConverter; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata; +import org.apache.flink.table.types.DataType; + +import java.time.Duration; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.SCAN_STARTUP_MODE; + +/** + * A {@link DynamicTableSource} that describes how to create a Oracle redo log from a logical + * description. + */ +public class OracleDataSource implements DataSource, SupportsReadingMetadata { + + private static final String SCAN_STARTUP_MODE_VALUE_INITIAL = "initial"; + private static final String SCAN_STARTUP_MODE_VALUE_LATEST = "latest-offset"; + private static final String DEBEZIUM_PROPERTIES_PREFIX = "debezium."; + private final OracleSourceConfig sourceConfig; + private final Configuration config; + private String[] capturedTables; + + // -------------------------------------------------------------------------------------------- + // Mutable attributes + // -------------------------------------------------------------------------------------------- + + /** Data type that describes the final output of the source. */ + protected DataType producedDataType; + + /** Metadata that is appended at the end of a physical source row. */ + protected List metadataKeys; + + private final List readableMetadataList; + + public OracleDataSource( + OracleSourceConfigFactory configFactory, + Configuration config, + String[] capturedTables, + List readableMetadataList) { + this.sourceConfig = configFactory.create(0); + this.config = config; + this.metadataKeys = Collections.emptyList(); + this.capturedTables = capturedTables; + this.readableMetadataList = readableMetadataList; + } + + @Override + public EventSourceProvider getEventSourceProvider() { + String url = config.get(OracleDataSourceOptions.JDBC_URL); + int port = config.get(OracleDataSourceOptions.PORT); + String hostname = config.get(OracleDataSourceOptions.HOSTNAME); + String database = config.get(OracleDataSourceOptions.DATABASE); + String username = config.get(OracleDataSourceOptions.USERNAME); + String password = config.get(OracleDataSourceOptions.PASSWORD); + String schemaName = config.get(OracleDataSourceOptions.SCHEMALIST); + boolean schemaChangeEnabled = config.get(OracleDataSourceOptions.SCHEMA_CHANGE_ENABLED); + + Properties dbzProperties = new Properties(); + dbzProperties.setProperty( + "database.connection.adapter", + config.get(OracleDataSourceOptions.DATABASE_CONNECTION_ADAPTER)); + dbzProperties.setProperty( + "log.mining.strategy", config.get(OracleDataSourceOptions.LOG_MINING_STRATEGY)); + dbzProperties.setProperty( + "snapshot.locking.mode", config.get(OracleDataSourceOptions.SNAPSHOT_LOCKING_MODE)); + dbzProperties.setProperty( + "snapshot.locking.mode", config.get(OracleDataSourceOptions.SNAPSHOT_LOCKING_MODE)); + dbzProperties.setProperty( + "database.history.store.only.captured.tables.ddl", + config.get(OracleDataSourceOptions.HISTORY_CAPTURED_TABLES_DDL_ENABLE)); + dbzProperties.setProperty("include.schema.changes", String.valueOf(schemaChangeEnabled)); + + Map map = + OracleDataSourceOptions.getPropertiesByPrefix(config, DEBEZIUM_PROPERTIES_PREFIX); + map.keySet().stream().forEach(e -> dbzProperties.put(e, map.get(e))); + StartupOptions startupOptions = getStartupOptions(config); + boolean enableParallelRead = + config.get(OracleDataSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_ENABLED); + int splitSize = config.get(OracleDataSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE); + int splitMetaGroupSize = config.get(OracleDataSourceOptions.CHUNK_META_GROUP_SIZE); + int fetchSize = config.get(OracleDataSourceOptions.SCAN_SNAPSHOT_FETCH_SIZE); + Duration connectTimeout = config.get(OracleDataSourceOptions.CONNECT_TIMEOUT); + int connectionPoolSize = config.get(OracleDataSourceOptions.CONNECTION_POOL_SIZE); + int connectMaxRetries = config.get(OracleDataSourceOptions.CONNECT_MAX_RETRIES); + double distributionFactorUpper = + config.get(OracleDataSourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND); + double distributionFactorLower = + config.get(OracleDataSourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND); + boolean closeIdleReaders = + config.get(OracleDataSourceOptions.SCAN_INCREMENTAL_CLOSE_IDLE_READER_ENABLED); + boolean skipSnapshotBackfill = + config.get(OracleDataSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP); + + OracleEventDeserializer deserializer = + new OracleEventDeserializer( + DebeziumChangelogMode.ALL, + config.get(OracleDataSourceOptions.SCHEMA_CHANGE_ENABLED), + readableMetadataList); + + RedoLogOffsetFactory offsetFactory = new RedoLogOffsetFactory(); + if (enableParallelRead) { + JdbcIncrementalSource oracleChangeEventSource = + OracleTableSourceReader.builder() + .hostname(hostname) + .url(url) + .port(port) + .databaseList(database) + .schemaList(schemaName) + .tableList(capturedTables) + .username(username) + .password(password) + .startupOptions(startupOptions) + .deserializer(deserializer) + .debeziumProperties(dbzProperties) + .splitSize(splitSize) + .splitMetaGroupSize(splitMetaGroupSize) + .fetchSize(fetchSize) + .connectTimeout(connectTimeout) + .connectionPoolSize(connectionPoolSize) + .connectMaxRetries(connectMaxRetries) + .distributionFactorUpper(distributionFactorUpper) + .distributionFactorLower(distributionFactorLower) + .closeIdleReaders(closeIdleReaders) + .skipSnapshotBackfill(skipSnapshotBackfill) + .includeSchemaChanges(schemaChangeEnabled) + .recordEmitter( + new OraclePipelineRecordEmitter( + deserializer, true, offsetFactory, sourceConfig)) + .build(); + + return FlinkSourceProvider.of(oracleChangeEventSource); + } else { + + OracleSourceReader.Builder builder = OracleSourceReader.builder(); + if (config.getOptional(OracleDataSourceOptions.JDBC_URL).isPresent()) { + builder = builder.url(config.get(OracleDataSourceOptions.JDBC_URL)); + } + DebeziumSourceFunction sourceFunction = + builder.hostname(config.get(OracleDataSourceOptions.HOSTNAME)) + .port(config.get(OracleDataSourceOptions.PORT)) + .database( + config.get( + OracleDataSourceOptions.DATABASE)) // monitor database + .schemaList( + config.get( + OracleDataSourceOptions.SCHEMALIST)) // monitor schema + .tableList(capturedTables) // monitor + .username(config.get(OracleDataSourceOptions.USERNAME)) + .password(config.get(OracleDataSourceOptions.PASSWORD)) + .deserializer(deserializer) // converts SourceRecord to JSON String + .startupOptions(StartupOptions.initial()) + .debeziumProperties(dbzProperties) + .sourceConfig(sourceConfig) + .build(); + + return FlinkSourceFunctionProvider.of(sourceFunction); + } + } + + private static StartupOptions getStartupOptions(Configuration 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_LATEST: + return StartupOptions.latest(); + + default: + throw new ValidationException( + String.format( + "Invalid value for option '%s'. Supported values are [%s, %s], but was: %s", + SCAN_STARTUP_MODE.key(), + SCAN_STARTUP_MODE_VALUE_INITIAL, + SCAN_STARTUP_MODE_VALUE_LATEST, + modeString)); + } + } + + private MetadataConverter[] getMetadataConverters() { + if (metadataKeys.isEmpty()) { + return new MetadataConverter[0]; + } + + return metadataKeys.stream() + .map( + key -> + Stream.of(OracleReadableMetaData.values()) + .filter(m -> m.getKey().equals(key)) + .findFirst() + .orElseThrow(IllegalStateException::new)) + .map(OracleReadableMetaData::getConverter) + .toArray(MetadataConverter[]::new); + } + + @Override + public MetadataAccessor getMetadataAccessor() { + return new OracleMetadataAccessor(sourceConfig); + } + + @Override + public Map listReadableMetadata() { + return Stream.of(OracleReadableMetaData.values()) + .collect( + Collectors.toMap( + OracleReadableMetaData::getKey, + OracleReadableMetaData::getDataType)); + } + + @Override + public void applyReadableMetadata(List metadataKeys, DataType producedDataType) { + this.metadataKeys = metadataKeys; + this.producedDataType = producedDataType; + } + + @VisibleForTesting + public OracleSourceConfig getSourceConfig() { + return sourceConfig; + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleDataSourceOptions.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleDataSourceOptions.java new file mode 100644 index 00000000000..f156045854a --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleDataSourceOptions.java @@ -0,0 +1,373 @@ +/* + * 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.oracle.source; + +import org.apache.flink.cdc.common.annotation.Experimental; +import org.apache.flink.cdc.common.annotation.PublicEvolving; +import org.apache.flink.cdc.common.configuration.ConfigOption; +import org.apache.flink.cdc.common.configuration.ConfigOptions; +import org.apache.flink.cdc.common.configuration.Configuration; + +import java.time.Duration; +import java.util.HashMap; +import java.util.Map; + +/** Configurations for {@link OracleDataSource}. */ +@PublicEvolving +public class OracleDataSourceOptions { + + public static final ConfigOption JDBC_URL = + ConfigOptions.key("jdbc.url") + .stringType() + .noDefaultValue() + .withDescription("The jdbc url."); + public static final ConfigOption SCHEMALIST = + ConfigOptions.key("schemalist") + .stringType() + .noDefaultValue() + .withDescription("schema list."); + public static final ConfigOption HOSTNAME = + ConfigOptions.key("hostname") + .stringType() + .noDefaultValue() + .withDescription("IP address or hostname of the oracle database server."); + + public static final ConfigOption PORT = + ConfigOptions.key("port") + .intType() + .defaultValue(3306) + .withDescription("Integer port number of the oracle database server."); + + public static final ConfigOption USERNAME = + ConfigOptions.key("username") + .stringType() + .noDefaultValue() + .withDescription( + "Name of the oracle database to use when connecting to the oracle database server."); + + public static final ConfigOption PASSWORD = + ConfigOptions.key("password") + .stringType() + .noDefaultValue() + .withDescription( + "Password to use when connecting to the oracle database server."); + + public static final ConfigOption DATABASE = + ConfigOptions.key("database") + .stringType() + .noDefaultValue() + .withDescription( + "Password to use when connecting to the oracle database server."); + public static final ConfigOption TABLES = + ConfigOptions.key("tables") + .stringType() + .noDefaultValue() + .withDescription( + "Table names of the oracle tables to monitor. Regular expressions are supported. " + + "It is important to note that the dot (.) is treated as a delimiter for database and table names. " + + "If there is a need to use a dot (.) in a regular expression to match any character, " + + "it is necessary to escape the dot with a backslash." + + "eg. db0.\\.*, db1.user_table_[0-9]+, db[1-2].[app|web]_order_\\.*"); + + public static final ConfigOption SERVER_TIME_ZONE = + ConfigOptions.key("server-time-zone") + .stringType() + .noDefaultValue() + .withDescription( + "The session time zone in database server. If not set, then " + + "ZoneId.systemDefault() is used to determine the server time zone."); + + public static final ConfigOption SERVER_ID = + ConfigOptions.key("server-id") + .stringType() + .noDefaultValue() + .withDescription( + "A numeric ID or a numeric ID range of this database client, " + + "The numeric ID syntax is like '5400', the numeric ID range syntax " + + "is like '5400-5408', The numeric ID range syntax is recommended when " + + "'scan.incremental.snapshot.enabled' enabled. Every ID must be unique across all " + + "currently-running database processes in the oracle cluster. This connector" + + " joins the oracle cluster as another server (with this unique ID) " + + "so it can read the binlog. By default, a random number is generated between" + + " 5400 and 6400, though we recommend setting an explicit value."); + + public static final ConfigOption SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE = + ConfigOptions.key("scan.incremental.snapshot.chunk.size") + .intType() + .defaultValue(8096) + .withDescription( + "The chunk size (number of rows) of table snapshot, captured tables are split into multiple chunks when read the snapshot of table."); + + public static final ConfigOption SCAN_SNAPSHOT_FETCH_SIZE = + ConfigOptions.key("scan.snapshot.fetch.size") + .intType() + .defaultValue(1024) + .withDescription( + "The maximum fetch size for per poll when read table snapshot."); + + public static final ConfigOption CONNECT_TIMEOUT = + ConfigOptions.key("connect.timeout") + .durationType() + .defaultValue(Duration.ofSeconds(30)) + .withDescription( + "The maximum time that the connector should wait after trying to connect to the oracle database server before timing out."); + + public static final ConfigOption CONNECTION_POOL_SIZE = + ConfigOptions.key("connection.pool.size") + .intType() + .defaultValue(20) + .withDescription("The connection pool size."); + + public static final ConfigOption CONNECT_MAX_RETRIES = + ConfigOptions.key("connect.max-retries") + .intType() + .defaultValue(3) + .withDescription( + "The max retry times that the connector should retry to build oracle database server connection."); + + public static final ConfigOption SCAN_STARTUP_MODE = + ConfigOptions.key("scan.startup.mode") + .stringType() + .defaultValue("initial") + .withDescription( + "Optional startup mode for oracle CDC consumer, valid enumerations are " + + "\"initial\", \"earliest-offset\", \"latest-offset\", \"timestamp\"\n" + + "or \"specific-offset\""); + + public static final ConfigOption SCAN_STARTUP_SPECIFIC_OFFSET_FILE = + ConfigOptions.key("scan.startup.specific-offset.file") + .stringType() + .noDefaultValue() + .withDescription( + "Optional binlog file name used in case of \"specific-offset\" startup mode"); + + public static final ConfigOption SCAN_STARTUP_SPECIFIC_OFFSET_POS = + ConfigOptions.key("scan.startup.specific-offset.pos") + .longType() + .noDefaultValue() + .withDescription( + "Optional binlog file position used in case of \"specific-offset\" startup mode"); + + public static final ConfigOption SCAN_STARTUP_SPECIFIC_OFFSET_GTID_SET = + ConfigOptions.key("scan.startup.specific-offset.gtid-set") + .stringType() + .noDefaultValue() + .withDescription( + "Optional GTID set used in case of \"specific-offset\" startup mode"); + + public static final ConfigOption SCAN_STARTUP_SPECIFIC_OFFSET_SKIP_EVENTS = + ConfigOptions.key("scan.startup.specific-offset.skip-events") + .longType() + .noDefaultValue() + .withDescription( + "Optional number of events to skip after the specific starting offset"); + + public static final ConfigOption SCAN_STARTUP_SPECIFIC_OFFSET_SKIP_ROWS = + ConfigOptions.key("scan.startup.specific-offset.skip-rows") + .longType() + .noDefaultValue() + .withDescription("Optional number of rows to skip after the specific offset"); + + public static final ConfigOption SCAN_STARTUP_TIMESTAMP_MILLIS = + ConfigOptions.key("scan.startup.timestamp-millis") + .longType() + .noDefaultValue() + .withDescription( + "Optional timestamp used in case of \"timestamp\" startup mode"); + + public static final ConfigOption HEARTBEAT_INTERVAL = + ConfigOptions.key("heartbeat.interval") + .durationType() + .defaultValue(Duration.ofSeconds(30)) + .withDescription( + "Optional interval of sending heartbeat event for tracing the latest available binlog offsets"); + + // ---------------------------------------------------------------------------- + // experimental options, won't add them to documentation + // ---------------------------------------------------------------------------- + @Experimental + public static final ConfigOption CHUNK_META_GROUP_SIZE = + ConfigOptions.key("chunk-meta.group.size") + .intType() + .defaultValue(1000) + .withDescription( + "The group size of chunk meta, if the meta size exceeds the group size, the meta will be divided into multiple groups."); + + @Experimental + public static final ConfigOption CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND = + ConfigOptions.key("chunk-key.even-distribution.factor.upper-bound") + .doubleType() + .defaultValue(1000.0d) + .withFallbackKeys("split-key.even-distribution.factor.upper-bound") + .withDescription( + "The upper bound of chunk key distribution factor. The distribution factor is used to determine whether the" + + " table is evenly distribution or not." + + " The table chunks would use evenly calculation optimization when the data distribution is even," + + " and the query oracle for splitting would happen when it is uneven." + + " The distribution factor could be calculated by (MAX(id) - MIN(id) + 1) / rowCount."); + + @Experimental + public static final ConfigOption CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND = + ConfigOptions.key("chunk-key.even-distribution.factor.lower-bound") + .doubleType() + .defaultValue(0.05d) + .withFallbackKeys("split-key.even-distribution.factor.lower-bound") + .withDescription( + "The lower bound of chunk key distribution factor. The distribution factor is used to determine whether the" + + " table is evenly distribution or not." + + " The table chunks would use evenly calculation optimization when the data distribution is even," + + " and the query oracle for splitting would happen when it is uneven." + + " The distribution factor could be calculated by (MAX(id) - MIN(id) + 1) / rowCount."); + + @Experimental + public static final ConfigOption SCAN_INCREMENTAL_CLOSE_IDLE_READER_ENABLED = + ConfigOptions.key("scan.incremental.close-idle-reader.enabled") + .booleanType() + .defaultValue(false) + .withDescription( + "Whether to close idle readers at the end of the snapshot phase. This feature depends on " + + "FLIP-147: Support Checkpoints After Tasks Finished. The flink version is required to be " + + "greater than or equal to 1.14 when enabling this feature."); + + @Experimental + public static final ConfigOption SCHEMA_CHANGE_ENABLED = + ConfigOptions.key("schema-change.enabled") + .booleanType() + .defaultValue(true) + .withDescription( + "Whether send schema change events, by default is true. If set to false, the schema changes will not be sent."); + + @Experimental + public static final ConfigOption DATABASE_TABLE_CASE_INSENSITIVE = + ConfigOptions.key("debezium.database.tablename.case.insensitive") + .stringType() + .defaultValue("false") + .withDescription( + "The case sensitivity of database and table names usually depends on the type of database and its configuration."); + + @Experimental + public static final ConfigOption DATABASE_CONNECTION_ADAPTER = + ConfigOptions.key("debezium.database.connection.adapter") + .stringType() + .defaultValue("logminer") + .withDescription("Database connection adapter."); + + @Experimental + public static final ConfigOption LOG_MINING_STRATEGY = + ConfigOptions.key("debezium.log.mining.strategy") + .stringType() + .defaultValue("online_catalog") + .withDescription("A strategy in log data analysis or mining."); + + @Experimental + public static final ConfigOption LOG_MINING_CONTINUOUS_MINE = + ConfigOptions.key("debezium.log.mining.continuous.mine") + .stringType() + .defaultValue("true") + .withDescription( + "A continuous or ongoing mining process in the field of log mining."); + + @Experimental + public static final ConfigOption SNAPSHOT_LOCKING_MODE = + ConfigOptions.key("debezium.snapshot.locking.mode") + .stringType() + .defaultValue("none") + .withDescription( + "Controls whether and for how long the connector holds a table lock. Table locks prevent certain types of changes table operations from occurring while the connector performs a snapshot. "); + + @Experimental + public static final ConfigOption HISTORY_CAPTURED_TABLES_DDL_ENABLE = + ConfigOptions.key("debezium.database.history.store.only.captured.tables.ddl") + .stringType() + .defaultValue("true") + .withDescription( + "Used to specify whether the connector records schema structures from the schema or all tables in the database, or only from the tables specified for capture. "); + + public static final ConfigOption SCAN_INCREMENTAL_SNAPSHOT_ENABLED = + ConfigOptions.key("scan.incremental.snapshot.enabled") + .booleanType() + .defaultValue(true) + .withDescription( + "Incremental snapshot is a new mechanism to read snapshot of a table. " + + "Compared to the old snapshot mechanism, the incremental snapshot has many advantages, including:\n" + + "(1) source can be parallel during snapshot reading, \n" + + "(2) source can perform checkpoints in the chunk granularity during snapshot reading, \n" + + "(3) source doesn't need to acquire global read lock (FLUSH TABLES WITH READ LOCK) before snapshot reading.\n" + + "For Oracle, if you would like the source run in parallel, each parallel reader should have an unique server id, " + + "so the 'server-id' must be a range like '5400-6400', and the range must be larger than the parallelism."); + + public static final ConfigOption SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND = + ConfigOptions.key("chunk-key.even-distribution.factor.upper-bound") + .doubleType() + .defaultValue(1000.0d) + .withFallbackKeys("split-key.even-distribution.factor.upper-bound") + .withDescription( + "The upper bound of chunk key distribution factor. The distribution factor is used to determine whether the" + + " table is evenly distribution or not." + + " The table chunks would use evenly calculation optimization when the data distribution is even," + + " and the query for splitting would happen when it is uneven." + + " The distribution factor could be calculated by (MAX(id) - MIN(id) + 1) / rowCount."); + + public static final ConfigOption SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND = + ConfigOptions.key("chunk-key.even-distribution.factor.lower-bound") + .doubleType() + .defaultValue(0.05d) + .withFallbackKeys("split-key.even-distribution.factor.lower-bound") + .withDescription( + "The lower bound of chunk key distribution factor. The distribution factor is used to determine whether the" + + " table is evenly distribution or not." + + " The table chunks would use evenly calculation optimization when the data distribution is even," + + " and the query for splitting would happen when it is uneven." + + " The distribution factor could be calculated by (MAX(id) - MIN(id) + 1) / rowCount."); + + @Experimental + public static final ConfigOption SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP = + ConfigOptions.key("scan.incremental.snapshot.backfill.skip") + .booleanType() + .defaultValue(false) + .withDescription( + "Whether to skip backfill in snapshot reading phase. If backfill is skipped, changes on captured tables during snapshot phase will be consumed later in binlog reading phase instead of being merged into the snapshot.WARNING: Skipping backfill might lead to data inconsistency because some binlog events happened within the snapshot phase might be replayed (only at-least-once semantic is promised). For example updating an already updated value in snapshot, or deleting an already deleted entry in snapshot. These replayed binlog events should be handled specially."); + + public static final ConfigOption BIG_COLUMN_CHARACTER_SET = + ConfigOptions.key("big.column.characterset") + .stringType() + .noDefaultValue() + .withDescription( + "Character set encoding of large field data types at source side binary. "); + + @Experimental + public static final ConfigOption METADATA_LIST = + ConfigOptions.key("metadata.list") + .stringType() + .noDefaultValue() + .withDescription( + "List of readable metadata from SourceRecord to be passed to downstream, split by `,`. " + + "Available readable metadata are: op_ts."); + + public static Map getPropertiesByPrefix( + Configuration tableOptions, String prefix) { + final Map props = new HashMap<>(); + for (Map.Entry entry : tableOptions.toMap().entrySet()) { + if (entry.getKey().startsWith(prefix)) { + String subKey = entry.getKey().substring(prefix.length()); + props.put(subKey, entry.getValue()); + } + } + return props; + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleEventDeserializer.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleEventDeserializer.java new file mode 100644 index 00000000000..318f7425306 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleEventDeserializer.java @@ -0,0 +1,175 @@ +/* + * 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.oracle.source; + +import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.cdc.common.data.binary.BinaryStringData; +import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.connectors.oracle.source.parser.OracleAntlrDdlParser; +import org.apache.flink.cdc.connectors.oracle.table.OracleReadableMetaData; +import org.apache.flink.cdc.debezium.event.DebeziumEventDeserializationSchema; +import org.apache.flink.cdc.debezium.table.DebeziumChangelogMode; +import org.apache.flink.table.data.TimestampData; + +import com.esri.core.geometry.ogc.OGCGeometry; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.debezium.data.Envelope; +import io.debezium.data.geometry.Geometry; +import io.debezium.data.geometry.Point; +import io.debezium.relational.Tables; +import io.debezium.relational.history.HistoryRecord; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; + +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.apache.flink.cdc.connectors.base.utils.SourceRecordUtils.getHistoryRecord; + +/** Event deserializer for {@link OracleDataSource}. */ +@Internal +public class OracleEventDeserializer extends DebeziumEventDeserializationSchema { + + private static final long serialVersionUID = 1L; + + public static final String SCHEMA_CHANGE_EVENT_KEY_NAME = + "io.debezium.connector.oracle.SchemaChangeKey"; + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private final boolean includeSchemaChanges; + + private transient Tables tables; + + private transient OracleAntlrDdlParser customParser; + + List readableMetadataList; + + public OracleEventDeserializer( + DebeziumChangelogMode changelogMode, + boolean includeSchemaChanges, + List readableMetadataList) { + super(new OracleSchemaDataTypeInference(), changelogMode); + this.includeSchemaChanges = includeSchemaChanges; + this.readableMetadataList = readableMetadataList; + } + + @Override + protected List deserializeSchemaChangeRecord(SourceRecord record) { + if (includeSchemaChanges) { + try { + HistoryRecord historyRecord = getHistoryRecord(record); + + String databaseName = + historyRecord.document().getString(HistoryRecord.Fields.DATABASE_NAME); + String schemaName = + historyRecord.document().getString(HistoryRecord.Fields.SCHEMA_NAME); + if (customParser == null) { + customParser = new OracleAntlrDdlParser(databaseName, schemaName); + tables = new Tables(); + } + String ddl = + historyRecord.document().getString(HistoryRecord.Fields.DDL_STATEMENTS); + customParser.setCurrentDatabase(databaseName); + customParser.parse(ddl, tables); + return customParser.getAndClearParsedEvents(); + } catch (Exception e) { + throw new IllegalStateException("Failed to parse the schema change : " + record, e); + } + } + return Collections.emptyList(); + } + + @Override + protected boolean isDataChangeRecord(SourceRecord record) { + Schema valueSchema = record.valueSchema(); + Struct value = (Struct) record.value(); + return value != null + && valueSchema != null + && valueSchema.field(Envelope.FieldName.OPERATION) != null + && value.getString(Envelope.FieldName.OPERATION) != null; + } + + @Override + protected boolean isSchemaChangeRecord(SourceRecord record) { + Schema keySchema = record.keySchema(); + return keySchema != null && SCHEMA_CHANGE_EVENT_KEY_NAME.equalsIgnoreCase(keySchema.name()); + } + + @Override + protected TableId getTableId(SourceRecord record) { + String[] parts = record.topic().split("\\."); + return TableId.tableId(parts[1], parts[2]); + } + + @Override + protected Map getMetadata(SourceRecord record) { + Map map = new HashMap<>(); + readableMetadataList.forEach( + (oracleReadableMetaData -> { + Object metadata = oracleReadableMetaData.getConverter().read(record); + if (oracleReadableMetaData.equals(OracleReadableMetaData.OP_TS)) { + map.put( + oracleReadableMetaData.getKey(), + String.valueOf(((TimestampData) metadata).getMillisecond())); + } else { + map.put(oracleReadableMetaData.getKey(), String.valueOf(metadata)); + } + })); + return map; + } + + @Override + protected Object convertToString(Object dbzObj, Schema schema) { + // the Geometry datatype in oracle will be converted to + // a String with Json format + if (Point.LOGICAL_NAME.equals(schema.name()) + || Geometry.LOGICAL_NAME.equals(schema.name())) { + try { + Struct geometryStruct = (Struct) dbzObj; + byte[] wkb = geometryStruct.getBytes("wkb"); + String geoJson = OGCGeometry.fromBinary(ByteBuffer.wrap(wkb)).asGeoJson(); + JsonNode originGeoNode = OBJECT_MAPPER.readTree(geoJson); + Optional srid = Optional.ofNullable(geometryStruct.getInt32("srid")); + Map geometryInfo = new HashMap<>(); + String geometryType = originGeoNode.get("type").asText(); + geometryInfo.put("type", geometryType); + if (geometryType.equals("GeometryCollection")) { + geometryInfo.put("geometries", originGeoNode.get("geometries")); + } else { + geometryInfo.put("coordinates", originGeoNode.get("coordinates")); + } + geometryInfo.put("srid", srid.orElse(0)); + return BinaryStringData.fromString( + OBJECT_MAPPER.writer().writeValueAsString(geometryInfo)); + } catch (Exception e) { + throw new IllegalArgumentException( + String.format("Failed to convert %s to geometry JSON.", dbzObj), e); + } + } else { + return BinaryStringData.fromString(dbzObj.toString()); + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleMetadataAccessor.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleMetadataAccessor.java new file mode 100644 index 00000000000..b68708f03a3 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleMetadataAccessor.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.connectors.oracle.source; + +import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.common.source.MetadataAccessor; +import org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfig; +import org.apache.flink.cdc.connectors.oracle.utils.OracleSchemaUtils; + +import io.debezium.connector.oracle.OraclePartition; + +import javax.annotation.Nullable; + +import java.util.List; + +/** {@link MetadataAccessor} for {@link OracleDataSource}. */ +@Internal +public class OracleMetadataAccessor implements MetadataAccessor { + + private final OracleSourceConfig sourceConfig; + + private final OraclePartition partition; + + public OracleMetadataAccessor(OracleSourceConfig sourceConfig) { + this.sourceConfig = sourceConfig; + this.partition = new OraclePartition(sourceConfig.getDbzConnectorConfig().getLogicalName()); + } + + /** + * Always throw {@link UnsupportedOperationException} because oracle does not support namespace. + */ + @Override + public List listNamespaces() { + throw new UnsupportedOperationException("List namespace is not supported by oracle."); + } + + /** + * List all database from oracle. + * + * @param namespace This parameter is ignored because oracle does not support namespace. + * @return The list of database + */ + @Override + public List listSchemas(@Nullable String namespace) { + return OracleSchemaUtils.listDatabases(sourceConfig); + } + + /** + * List tables from oracle. + * + * @param namespace This parameter is ignored because oracle does not support namespace. + * @param dbName The database to list tables from. If null, list tables from all databases. + * @return The list of {@link TableId}s. + */ + @Override + public List listTables(@Nullable String namespace, @Nullable String dbName) { + return OracleSchemaUtils.listTables(sourceConfig, dbName); + } + + /** + * Get the {@link Schema} of the given table. + * + * @param tableId The {@link TableId} of the given table. + * @return The {@link Schema} of the table. + */ + @Override + public Schema getTableSchema(TableId tableId) { + Schema schema = OracleSchemaUtils.getTableSchema(tableId, sourceConfig); + return schema; + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleSchemaDataTypeInference.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleSchemaDataTypeInference.java new file mode 100644 index 00000000000..4379e37986c --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleSchemaDataTypeInference.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.connectors.oracle.source; + +import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.common.types.DataTypes; +import org.apache.flink.cdc.debezium.event.DebeziumSchemaDataTypeInference; + +import io.debezium.data.geometry.Geometry; +import io.debezium.data.geometry.Point; +import org.apache.kafka.connect.data.Schema; + +/** {@link DataType} inference for oracle debezium {@link Schema}. */ +@Internal +public class OracleSchemaDataTypeInference extends DebeziumSchemaDataTypeInference { + + private static final long serialVersionUID = 1L; + + protected DataType inferStruct(Object value, Schema schema) { + // the Geometry datatype in oracle will be converted to + // a String with Json format + if (Point.LOGICAL_NAME.equals(schema.name()) + || Geometry.LOGICAL_NAME.equals(schema.name())) { + return DataTypes.STRING(); + } else { + return super.inferStruct(value, schema); + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/parser/BaseParserListener.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/parser/BaseParserListener.java new file mode 100644 index 00000000000..efd4d304356 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/parser/BaseParserListener.java @@ -0,0 +1,96 @@ +/* + * 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.oracle.source.parser; + +import io.debezium.ddl.parser.oracle.generated.PlSqlParser; +import io.debezium.ddl.parser.oracle.generated.PlSqlParserBaseListener; + +/** This class contains common methods for all listeners. */ +public class BaseParserListener extends PlSqlParserBaseListener { + + public String getTableName(final PlSqlParser.Tableview_nameContext tableviewName) { + final String tableName; + if (tableviewName.id_expression() != null) { + tableName = tableviewName.id_expression().getText(); + } else { + tableName = tableviewName.identifier().id_expression().getText(); + } + return getTableOrColumnName(tableName); + } + + public String getTableName(final PlSqlParser.Column_nameContext ctx) { + final String tableName; + if (ctx.id_expression() != null && ctx.id_expression().size() > 1) { + tableName = getTableOrColumnName(ctx.id_expression(0).getText()); + } else { + tableName = getTableOrColumnName(ctx.identifier().id_expression().getText()); + } + return tableName; + } + + public String getColumnName(final PlSqlParser.Column_nameContext ctx) { + final String columnName; + if (ctx.id_expression() != null && ctx.id_expression().size() > 0) { + columnName = + getTableOrColumnName( + ctx.id_expression(ctx.id_expression().size() - 1).getText()); + } else { + columnName = getTableOrColumnName(ctx.identifier().id_expression().getText()); + } + return columnName; + } + + public String getColumnName(final PlSqlParser.Old_column_nameContext ctx) { + return getTableOrColumnName(ctx.getText()); + } + + public String getColumnName(final PlSqlParser.New_column_nameContext ctx) { + return getTableOrColumnName(ctx.getText()); + } + + /** + * Resolves a table or column name from the provided string. + * + *

Oracle table and column names are inherently stored in upper-case; however, if the objects + * are created using double-quotes, the case of the object name is retained. Therefore when + * needing to parse a table or column name, this method will adhere to those rules and will + * always return the name in upper-case unless the provided name is double-quoted in which the + * returned value will have the double-quotes removed and case retained. + * + * @param name table or column name + * @return parsed table or column name from the supplied name argument + */ + private static String getTableOrColumnName(String name) { + return removeQuotes(name, true); + } + + /** + * Removes leading and trailing double quote characters from the provided string. + * + * @param text value to have double quotes removed + * @param upperCaseIfNotQuoted control if returned string is upper-cased if not quoted + * @return string that has had quotes removed + */ + @SuppressWarnings("SameParameterValue") + private static String removeQuotes(String text, boolean upperCaseIfNotQuoted) { + if (text != null && text.length() > 2 && text.startsWith("\"") && text.endsWith("\"")) { + return text.substring(1, text.length() - 1); + } + return upperCaseIfNotQuoted ? text.toUpperCase() : text; + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/parser/ColumnDefinitionParserListener.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/parser/ColumnDefinitionParserListener.java new file mode 100644 index 00000000000..00452d87766 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/parser/ColumnDefinitionParserListener.java @@ -0,0 +1,312 @@ +/* + * 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.oracle.source.parser; + +import io.debezium.antlr.DataTypeResolver; +import io.debezium.connector.oracle.antlr.OracleDdlParser; +import io.debezium.ddl.parser.oracle.generated.PlSqlParser; +import io.debezium.relational.Column; +import io.debezium.relational.ColumnEditor; +import io.debezium.relational.TableEditor; +import oracle.jdbc.OracleTypes; +import org.antlr.v4.runtime.tree.ParseTreeListener; + +import java.sql.Types; +import java.util.List; + +/** Parser listener that parses column definitions of Oracle DDL statements. */ +public class ColumnDefinitionParserListener extends BaseParserListener { + + private final OracleDdlParser parser; + private final DataTypeResolver dataTypeResolver; + private final TableEditor tableEditor; + private final List listeners; + private ColumnEditor columnEditor; + + public ColumnDefinitionParserListener( + final TableEditor tableEditor, + final ColumnEditor columnEditor, + OracleDdlParser parser, + List listeners) { + this.tableEditor = tableEditor; + this.columnEditor = columnEditor; + this.parser = parser; + this.dataTypeResolver = parser.dataTypeResolver(); + this.listeners = listeners; + } + + public void setColumnEditor(ColumnEditor columnEditor) { + this.columnEditor = columnEditor; + } + + Column getColumn() { + return columnEditor.create(); + } + + @Override + public void enterColumn_definition(PlSqlParser.Column_definitionContext ctx) { + resolveColumnDataType(ctx); + if (ctx.DEFAULT() != null) { + columnEditor.defaultValueExpression(ctx.column_default_value().getText()); + } + super.enterColumn_definition(ctx); + } + + @Override + public void enterPrimary_key_clause(PlSqlParser.Primary_key_clauseContext ctx) { + // this rule will be parsed only if no primary key is set in a table + // otherwise the statement can't be executed due to multiple primary key error + columnEditor.optional(false); + tableEditor.addColumn(columnEditor.create()); + tableEditor.setPrimaryKeyNames(columnEditor.name()); + super.enterPrimary_key_clause(ctx); + } + + @Override + public void enterModify_col_properties(PlSqlParser.Modify_col_propertiesContext ctx) { + resolveColumnDataType(ctx); + if (ctx.DEFAULT() != null) { + columnEditor.defaultValueExpression(ctx.column_default_value().getText()); + } + super.enterModify_col_properties(ctx); + } + + // todo use dataTypeResolver instead + private void resolveColumnDataType(PlSqlParser.Column_definitionContext ctx) { + columnEditor.name(getColumnName(ctx.column_name())); + + boolean hasNotNullConstraint = + ctx.inline_constraint().stream().anyMatch(c -> c.NOT() != null); + columnEditor.optional(!hasNotNullConstraint); + + if (ctx.datatype() == null) { + if (ctx.type_name() != null + && "MDSYS.SDO_GEOMETRY" + .equalsIgnoreCase(ctx.type_name().getText().replace("\"", ""))) { + columnEditor.jdbcType(Types.STRUCT).type("MDSYS.SDO_GEOMETRY"); + } + } else { + resolveColumnDataType(ctx.datatype()); + } + } + + private void resolveColumnDataType(PlSqlParser.Modify_col_propertiesContext ctx) { + columnEditor.name(getColumnName(ctx.column_name())); + + resolveColumnDataType(ctx.datatype()); + + boolean hasNullConstraint = + ctx.inline_constraint().stream().anyMatch(c -> c.NULL_() != null); + boolean hasNotNullConstraint = + ctx.inline_constraint().stream().anyMatch(c -> c.NOT() != null); + if (hasNotNullConstraint && columnEditor.isOptional()) { + columnEditor.optional(false); + } else if (hasNullConstraint && !columnEditor.isOptional()) { + columnEditor.optional(true); + } + } + + private void resolveColumnDataType(PlSqlParser.DatatypeContext ctx) { + // If the context is null, there is nothing this method can resolve and it is safe to return + if (ctx == null) { + return; + } + + if (ctx.native_datatype_element() != null) { + PlSqlParser.Precision_partContext precisionPart = ctx.precision_part(); + if (ctx.native_datatype_element().INT() != null + || ctx.native_datatype_element().INTEGER() != null + || ctx.native_datatype_element().SMALLINT() != null + || ctx.native_datatype_element().NUMERIC() != null + || ctx.native_datatype_element().DECIMAL() != null) { + // NUMERIC and DECIMAL types have by default zero scale + columnEditor.jdbcType(Types.NUMERIC).type("NUMBER"); + + if (precisionPart == null) { + columnEditor.length(38).scale(0); + } else { + setPrecision(precisionPart, columnEditor); + setScale(precisionPart, columnEditor); + } + } else if (ctx.native_datatype_element().DATE() != null) { + // JDBC driver reports type as timestamp but name DATE + columnEditor.jdbcType(Types.TIMESTAMP).type("DATE"); + } else if (ctx.native_datatype_element().TIMESTAMP() != null) { + if (ctx.WITH() != null && ctx.TIME() != null && ctx.ZONE() != null) { + if (ctx.LOCAL() != null) { + columnEditor + .jdbcType(OracleTypes.TIMESTAMPLTZ) + .type("TIMESTAMP WITH LOCAL TIME ZONE"); + } else { + columnEditor + .jdbcType(OracleTypes.TIMESTAMPTZ) + .type("TIMESTAMP WITH TIME ZONE"); + } + } else { + columnEditor.jdbcType(Types.TIMESTAMP).type("TIMESTAMP"); + } + + if (precisionPart == null) { + columnEditor.length(6); + } else { + setPrecision(precisionPart, columnEditor); + } + } + // VARCHAR is the same as VARCHAR2 in Oracle + else if (ctx.native_datatype_element().VARCHAR2() != null + || ctx.native_datatype_element().VARCHAR() != null) { + columnEditor.jdbcType(Types.VARCHAR).type("VARCHAR2"); + + if (precisionPart == null) { + columnEditor.length(getVarCharDefaultLength()); + } else { + setPrecision(precisionPart, columnEditor); + } + } else if (ctx.native_datatype_element().NVARCHAR2() != null) { + columnEditor.jdbcType(Types.NVARCHAR).type("NVARCHAR2"); + + if (precisionPart == null) { + columnEditor.length(getVarCharDefaultLength()); + } else { + setPrecision(precisionPart, columnEditor); + } + } else if (ctx.native_datatype_element().CHAR() != null) { + columnEditor.jdbcType(Types.CHAR).type("CHAR").length(1); + + if (precisionPart != null) { + setPrecision(precisionPart, columnEditor); + } + } else if (ctx.native_datatype_element().NCHAR() != null) { + columnEditor.jdbcType(Types.NCHAR).type("NCHAR").length(1); + + if (precisionPart != null) { + setPrecision(precisionPart, columnEditor); + } + } else if (ctx.native_datatype_element().BINARY_FLOAT() != null) { + columnEditor.jdbcType(OracleTypes.BINARY_FLOAT).type("BINARY_FLOAT"); + } else if (ctx.native_datatype_element().BINARY_DOUBLE() != null) { + columnEditor.jdbcType(OracleTypes.BINARY_DOUBLE).type("BINARY_DOUBLE"); + } + // PRECISION keyword is mandatory + else if (ctx.native_datatype_element().FLOAT() != null + || (ctx.native_datatype_element().DOUBLE() != null + && ctx.native_datatype_element().PRECISION() != null)) { + columnEditor.jdbcType(Types.FLOAT).type("FLOAT").length(126); + + // TODO float's precision is about bits not decimal digits; should be ok for now to + // over-size + if (precisionPart != null) { + setPrecision(precisionPart, columnEditor); + } + } else if (ctx.native_datatype_element().REAL() != null) { + columnEditor + .jdbcType(Types.FLOAT) + .type("FLOAT") + // TODO float's precision is about bits not decimal digits; should be ok for + // now to over-size + .length(63); + } else if (ctx.native_datatype_element().NUMBER() != null) { + columnEditor.jdbcType(Types.NUMERIC).type("NUMBER"); + + if (precisionPart == null) { + columnEditor.length(38); + } else { + if (precisionPart.ASTERISK() != null) { + // when asterisk is used, explicitly set precision to 38 + columnEditor.length(38); + } else { + setPrecision(precisionPart, columnEditor); + } + setScale(precisionPart, columnEditor); + } + } else if (ctx.native_datatype_element().BLOB() != null) { + columnEditor.jdbcType(Types.BLOB).type("BLOB"); + } else if (ctx.native_datatype_element().CLOB() != null) { + columnEditor.jdbcType(Types.CLOB).type("CLOB"); + } else if (ctx.native_datatype_element().NCLOB() != null) { + columnEditor.jdbcType(Types.NCLOB).type("NCLOB"); + } else if (ctx.native_datatype_element().RAW() != null) { + columnEditor.jdbcType(OracleTypes.RAW).type("RAW"); + + setPrecision(precisionPart, columnEditor); + } else if (ctx.native_datatype_element().SDO_GEOMETRY() != null) { + // Allows the registration of new SDO_GEOMETRY columns via an CREATE/ALTER TABLE + // This is the same registration of the column that is resolved during JDBC metadata + // inspection. + columnEditor.jdbcType(OracleTypes.OTHER).type("SDO_GEOMETRY").length(1); + } else if (ctx.native_datatype_element().ROWID() != null) { + columnEditor.jdbcType(Types.VARCHAR).type("ROWID"); + } else { + columnEditor + .jdbcType(OracleTypes.OTHER) + .type(ctx.native_datatype_element().getText()); + } + } else if (ctx.INTERVAL() != null + && ctx.YEAR() != null + && ctx.TO() != null + && ctx.MONTH() != null) { + columnEditor.jdbcType(OracleTypes.INTERVALYM).type("INTERVAL YEAR TO MONTH").length(2); + if (!ctx.expression().isEmpty()) { + columnEditor.length(Integer.valueOf((ctx.expression(0).getText()))); + } + } else if (ctx.INTERVAL() != null + && ctx.DAY() != null + && ctx.TO() != null + && ctx.SECOND() != null) { + columnEditor + .jdbcType(OracleTypes.INTERVALDS) + .type("INTERVAL DAY TO SECOND") + .length(2) + .scale(6); + for (final PlSqlParser.ExpressionContext e : ctx.expression()) { + if (e.getSourceInterval().startsAfter(ctx.TO().getSourceInterval())) { + columnEditor.scale(Integer.valueOf(e.getText())); + } else { + columnEditor.length(Integer.valueOf(e.getText())); + } + } + if (!ctx.expression().isEmpty()) { + columnEditor.length(Integer.valueOf((ctx.expression(0).getText()))); + } + } else { + columnEditor.jdbcType(OracleTypes.OTHER).type(ctx.getText()); + } + } + + private int getVarCharDefaultLength() { + // TODO replace with value from select name, value from v$parameter where + // name='max_string_size'; + return 4000; + } + + private void setPrecision( + PlSqlParser.Precision_partContext precisionPart, ColumnEditor columnEditor) { + columnEditor.length(Integer.valueOf(precisionPart.numeric(0).getText())); + } + + private void setScale( + PlSqlParser.Precision_partContext precisionPart, ColumnEditor columnEditor) { + if (precisionPart.numeric().size() > 1) { + columnEditor.scale(Integer.valueOf(precisionPart.numeric(1).getText())); + } else if (precisionPart.numeric_negative() != null) { + columnEditor.scale(Integer.valueOf(precisionPart.numeric_negative().getText())); + } else { + columnEditor.scale(0); + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/parser/OracleAlterTableParserListener.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/parser/OracleAlterTableParserListener.java new file mode 100644 index 00000000000..14886a0c5be --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/parser/OracleAlterTableParserListener.java @@ -0,0 +1,410 @@ +/* + * 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.oracle.source.parser; + +import org.apache.flink.cdc.common.event.AddColumnEvent; +import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; +import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.RenameColumnEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.types.DataType; + +import io.debezium.connector.oracle.antlr.OracleDdlParser; +import io.debezium.ddl.parser.oracle.generated.PlSqlParser; +import io.debezium.relational.Column; +import io.debezium.relational.ColumnEditor; +import io.debezium.relational.Table; +import io.debezium.relational.TableEditor; +import io.debezium.relational.TableId; +import io.debezium.text.ParsingException; +import org.antlr.v4.runtime.tree.ParseTreeListener; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +import static io.debezium.antlr.AntlrDdlParser.getText; +import static org.apache.flink.cdc.connectors.oracle.utils.OracleTypeUtils.fromDbzColumn; + +/** Parser listener that is parsing Oracle ALTER TABLE statements. */ +public class OracleAlterTableParserListener extends BaseParserListener { + + private static final Logger LOGGER = + LoggerFactory.getLogger( + io.debezium.connector.oracle.antlr.listener.AlterTableParserListener.class); + + private static final int STARTING_INDEX = 1; + private TableEditor tableEditor; + private String catalogName; + private String schemaName; + private TableId previousTableId; + private OracleDdlParser parser; + private final List listeners; + private ColumnDefinitionParserListener columnDefinitionParserListener; + private List columnEditors; + private int parsingColumnIndex = STARTING_INDEX; + private final LinkedList changes; + + /** + * Package visible Constructor. + * + * @param catalogName Represents database name. If null, points to the current database. + * @param schemaName Schema/user name. If null, points to the current schema. + * @param parser Oracle Antlr parser. + * @param listeners registered listeners. + */ + public OracleAlterTableParserListener( + final String catalogName, + final String schemaName, + final OracleDdlParser parser, + final List listeners, + LinkedList changes) { + this.catalogName = catalogName; + this.schemaName = schemaName; + this.parser = parser; + this.listeners = listeners; + this.changes = changes; + } + + @Override + public void enterAlter_table(PlSqlParser.Alter_tableContext ctx) { + TableId tableId = new TableId(catalogName, schemaName, getTableName(ctx.tableview_name())); + tableEditor = Table.editor().tableId(tableId); + if (tableEditor == null) { + throw new ParsingException( + null, + "Trying to alter table " + + tableId.toString() + + ", which does not exist. Query: " + + getText(ctx)); + } + super.enterAlter_table(ctx); + } + + @Override + public void exitAlter_table(PlSqlParser.Alter_tableContext ctx) { + parser.runIfNotNull( + () -> { + listeners.remove(columnDefinitionParserListener); + parser.databaseTables().overwriteTable(tableEditor.create()); + parser.signalAlterTable( + tableEditor.tableId(), previousTableId, ctx.getParent()); + }, + tableEditor); + super.exitAlter_table(ctx); + } + + @Override + public void enterAlter_table_properties(PlSqlParser.Alter_table_propertiesContext ctx) { + parser.runIfNotNull( + () -> { + if (ctx.RENAME() != null && ctx.TO() != null) { + previousTableId = tableEditor.tableId(); + String tableName = getTableName(ctx.tableview_name()); + final TableId newTableId = + new TableId( + tableEditor.tableId().catalog(), + tableEditor.tableId().schema(), + tableName); + if (parser.getTableFilter().isIncluded(previousTableId) + && !parser.getTableFilter().isIncluded(newTableId)) { + LOGGER.warn( + "Renaming included table {} to non-included table {}, this can lead to schema inconsistency", + previousTableId, + newTableId); + } else if (!parser.getTableFilter().isIncluded(previousTableId) + && parser.getTableFilter().isIncluded(newTableId)) { + LOGGER.warn( + "Renaming non-included table {} to included table {}, this can lead to schema inconsistency", + previousTableId, + newTableId); + } + parser.databaseTables().overwriteTable(tableEditor.create()); + parser.databaseTables().renameTable(tableEditor.tableId(), newTableId); + tableEditor = parser.databaseTables().editTable(newTableId); + } + }, + tableEditor); + super.exitAlter_table_properties(ctx); + } + + @Override + public void enterAdd_column_clause(PlSqlParser.Add_column_clauseContext ctx) { + parser.runIfNotNull( + () -> { + List columns = ctx.column_definition(); + columnEditors = new ArrayList<>(columns.size()); + List addedColumns = new ArrayList<>(); + for (PlSqlParser.Column_definitionContext column : columns) { + String columnName = getColumnName(column.column_name()); + String type = column.datatype().children.get(0).getText(); + int length = 0; + int precision = 0; + int scale = 0; + if (column.datatype().children.size() == 2) { + String typeDescribStr = column.datatype().children.get(1).getText(); + if (typeDescribStr.contains("(")) { + typeDescribStr = typeDescribStr.replace("(", "").replace(")", ""); + if (column.datatype().children.get(1).getText().contains(",")) { + length = Integer.parseInt(typeDescribStr.split(",")[0]); + scale = Integer.parseInt(typeDescribStr.split(",")[1]); + } else { + length = Integer.parseInt(typeDescribStr); + } + } + } + ColumnEditor editor = + Column.editor() + .name(columnName) + .type(type) + .length(length) + .scale(scale); + columnEditors.add(editor); + addedColumns.add( + new AddColumnEvent.ColumnWithPosition( + toCdcColumn(editor.create()))); + } + + changes.add( + new AddColumnEvent(toCdcTableId(tableEditor.tableId()), addedColumns)); + columnDefinitionParserListener = + new ColumnDefinitionParserListener( + tableEditor, columnEditors.get(0), parser, listeners); + listeners.add(columnDefinitionParserListener); + }, + tableEditor); + super.enterAdd_column_clause(ctx); + } + + @Override + public void enterModify_column_clauses(PlSqlParser.Modify_column_clausesContext ctx) { + parser.runIfNotNull( + () -> { + List columns = + ctx.modify_col_properties(); + columnEditors = new ArrayList<>(columns.size()); + for (PlSqlParser.Modify_col_propertiesContext column : columns) { + String columnName = getColumnName(column.column_name()); + String lengthStr = column.datatype().children.get(1).getChild(1).getText(); + int length; + Column existingColumn; + if (lengthStr != null) { + length = Integer.parseInt(lengthStr); + existingColumn = + Column.editor() + .name(columnName) + .type(column.datatype().children.get(0).getText()) + .length(length) + .create(); + } else { + existingColumn = + Column.editor() + .name(columnName) + .type(column.datatype().children.get(0).getText()) + .create(); + } + if (existingColumn != null) { + ColumnEditor columnEditor = existingColumn.edit(); + columnEditors.add(columnEditor); + Map typeMapping = new HashMap<>(); + typeMapping.put(columnName, fromDbzColumn(columnEditor.create())); + changes.add( + new AlterColumnTypeEvent( + toCdcTableId(tableEditor.tableId()), typeMapping)); + } else { + throw new ParsingException( + null, + "trying to change column " + + columnName + + " in " + + tableEditor.tableId().toString() + + " table, which does not exist. Query: " + + getText(ctx)); + } + } + }, + tableEditor); + super.enterModify_column_clauses(ctx); + } + + @Override + public void exitAdd_column_clause(PlSqlParser.Add_column_clauseContext ctx) { + parser.runIfNotNull( + () -> { + columnEditors.forEach( + columnEditor -> tableEditor.addColumn(columnEditor.create())); + listeners.remove(columnDefinitionParserListener); + columnDefinitionParserListener = null; + }, + tableEditor, + columnEditors); + super.exitAdd_column_clause(ctx); + } + + @Override + public void exitModify_column_clauses(PlSqlParser.Modify_column_clausesContext ctx) { + parser.runIfNotNull( + () -> { + columnEditors.forEach( + columnEditor -> tableEditor.addColumn(columnEditor.create())); + listeners.remove(columnDefinitionParserListener); + columnDefinitionParserListener = null; + }, + tableEditor, + columnEditors); + super.exitModify_column_clauses(ctx); + } + + @Override + public void exitColumn_definition(PlSqlParser.Column_definitionContext ctx) { + parser.runIfNotNull( + () -> { + if (columnEditors != null) { + // column editor list is not null when a multiple columns are parsed in one + // statement + if (columnEditors.size() > parsingColumnIndex) { + // assign next column editor to parse another column definition + columnDefinitionParserListener.setColumnEditor( + columnEditors.get(parsingColumnIndex++)); + } else { + // all columns parsed + // reset global variables for next parsed statement + columnEditors.forEach( + columnEditor -> tableEditor.addColumn(columnEditor.create())); + columnEditors = null; + parsingColumnIndex = STARTING_INDEX; + } + } + }, + tableEditor, + columnEditors); + super.exitColumn_definition(ctx); + } + + @Override + public void exitModify_col_properties(PlSqlParser.Modify_col_propertiesContext ctx) { + parser.runIfNotNull( + () -> { + if (columnEditors != null) { + // column editor list is not null when multiple columns are paresd in one + // statement + if (columnEditors.size() > parsingColumnIndex) { + // assign next column editor to parse another column definition + columnDefinitionParserListener.setColumnEditor( + columnEditors.get(parsingColumnIndex++)); + } else { + // all columns parsed + // reset global variables for next parsed statement + columnEditors.forEach( + columnEditor -> tableEditor.addColumn(columnEditor.create())); + columnEditors = null; + parsingColumnIndex = STARTING_INDEX; + } + } + }, + tableEditor, + columnEditors); + super.exitModify_col_properties(ctx); + } + + @Override + public void enterDrop_column_clause(PlSqlParser.Drop_column_clauseContext ctx) { + parser.runIfNotNull( + () -> { + List columnNameContexts = ctx.column_name(); + columnEditors = new ArrayList<>(columnNameContexts.size()); + for (PlSqlParser.Column_nameContext columnNameContext : columnNameContexts) { + String columnName = getColumnName(columnNameContext); + tableEditor.removeColumn(columnName); + changes.add( + new DropColumnEvent( + toCdcTableId(tableEditor.tableId()), + Arrays.asList(columnName))); + } + }, + tableEditor); + super.enterDrop_column_clause(ctx); + } + + @Override + public void exitRename_column_clause(PlSqlParser.Rename_column_clauseContext ctx) { + parser.runIfNotNull( + () -> { + String oldColumnName = getColumnName(ctx.old_column_name()); + String newColumnName = getColumnName(ctx.new_column_name()); + if (newColumnName != null && !oldColumnName.equalsIgnoreCase(newColumnName)) { + Map renameMap = new HashMap<>(); + renameMap.put(oldColumnName, newColumnName); + changes.add( + new RenameColumnEvent( + toCdcTableId(tableEditor.tableId()), renameMap)); + } + }, + tableEditor); + super.exitRename_column_clause(ctx); + } + + @Override + public void enterConstraint_clauses(PlSqlParser.Constraint_clausesContext ctx) { + parser.runIfNotNull( + () -> { + if (ctx.ADD() != null) { + // ALTER TABLE ADD PRIMARY KEY + List primaryKeyColumns = new ArrayList<>(); + for (PlSqlParser.Out_of_line_constraintContext constraint : + ctx.out_of_line_constraint()) { + if (constraint.PRIMARY() != null && constraint.KEY() != null) { + for (PlSqlParser.Column_nameContext columnNameContext : + constraint.column_name()) { + primaryKeyColumns.add(getColumnName(columnNameContext)); + } + } + } + if (!primaryKeyColumns.isEmpty()) { + tableEditor.setPrimaryKeyNames(primaryKeyColumns); + } + } else if (ctx.MODIFY() != null && ctx.PRIMARY() != null && ctx.KEY() != null) { + // ALTER TABLE MODIFY PRIMARY KEY columns + List primaryKeyColumns = new ArrayList<>(); + for (PlSqlParser.Column_nameContext columnNameContext : ctx.column_name()) { + primaryKeyColumns.add(getColumnName(columnNameContext)); + } + if (!primaryKeyColumns.isEmpty()) { + tableEditor.setPrimaryKeyNames(primaryKeyColumns); + } + } + }, + tableEditor); + super.enterConstraint_clauses(ctx); + } + + private org.apache.flink.cdc.common.schema.Column toCdcColumn(Column dbzColumn) { + return org.apache.flink.cdc.common.schema.Column.physicalColumn( + dbzColumn.name(), fromDbzColumn(dbzColumn), dbzColumn.comment()); + } + + private org.apache.flink.cdc.common.event.TableId toCdcTableId(TableId dbzTableId) { + return org.apache.flink.cdc.common.event.TableId.tableId( + dbzTableId.schema(), dbzTableId.table()); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/parser/OracleAntlrDdlParser.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/parser/OracleAntlrDdlParser.java new file mode 100644 index 00000000000..ec51b72851e --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/parser/OracleAntlrDdlParser.java @@ -0,0 +1,55 @@ +/* + * 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.oracle.source.parser; + +import org.apache.flink.cdc.common.event.SchemaChangeEvent; + +import io.debezium.antlr.AntlrDdlParserListener; +import io.debezium.connector.oracle.antlr.OracleDdlParser; + +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; + +/** A ddl parser that will use custom listener. */ +public class OracleAntlrDdlParser extends OracleDdlParser { + + private final LinkedList parsedEvents; + + private final String databaseName; + private final String schemaName; + + public OracleAntlrDdlParser(String databaseName, String schemaName) { + super(); + this.parsedEvents = new LinkedList<>(); + this.databaseName = databaseName; + this.schemaName = schemaName; + } + + @Override + protected AntlrDdlParserListener createParseTreeWalkerListener() { + return new OracleAntlrDdlParserListener( + this.databaseName, this.schemaName, this, parsedEvents); + } + + public List getAndClearParsedEvents() { + List result = new ArrayList<>(parsedEvents); + parsedEvents.clear(); + return result; + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/parser/OracleAntlrDdlParserListener.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/parser/OracleAntlrDdlParserListener.java new file mode 100644 index 00000000000..0445f923506 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/parser/OracleAntlrDdlParserListener.java @@ -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. + */ + +package org.apache.flink.cdc.connectors.oracle.source.parser; + +import org.apache.flink.cdc.common.event.SchemaChangeEvent; + +import io.debezium.antlr.AntlrDdlParserListener; +import io.debezium.antlr.ProxyParseTreeListenerUtil; +import io.debezium.connector.oracle.antlr.OracleDdlParser; +import io.debezium.ddl.parser.oracle.generated.PlSqlParserBaseListener; +import io.debezium.text.ParsingException; +import org.antlr.v4.runtime.ParserRuleContext; +import org.antlr.v4.runtime.tree.ParseTreeListener; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; + +/** + * This class is Oracle main DDL parser listener class. It instantiates supported listeners, walks + * listeners through every parsing rule and collects parsing exceptions. + */ +public class OracleAntlrDdlParserListener extends PlSqlParserBaseListener + implements AntlrDdlParserListener { + + private final List listeners = new CopyOnWriteArrayList<>(); + private final Collection errors = new ArrayList<>(); + + public OracleAntlrDdlParserListener( + final String catalogName, + final String schemaName, + final OracleDdlParser parser, + LinkedList parsedEvents) { + listeners.add( + new OracleAlterTableParserListener( + catalogName, schemaName, parser, listeners, parsedEvents)); + } + + @Override + public Collection getErrors() { + return errors; + } + + @Override + public void enterEveryRule(ParserRuleContext ctx) { + ProxyParseTreeListenerUtil.delegateEnterRule(ctx, listeners, errors); + } + + @Override + public void exitEveryRule(ParserRuleContext ctx) { + ProxyParseTreeListenerUtil.delegateExitRule(ctx, listeners, errors); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/reader/OracleDebeziumSourceFunction.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/reader/OracleDebeziumSourceFunction.java new file mode 100644 index 00000000000..00c95abc451 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/reader/OracleDebeziumSourceFunction.java @@ -0,0 +1,124 @@ +/* + * 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.oracle.source.reader; + +import org.apache.flink.cdc.common.annotation.PublicEvolving; +import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfig; +import org.apache.flink.cdc.connectors.oracle.utils.DebeziumUtils; +import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.DebeziumSourceFunction; +import org.apache.flink.cdc.debezium.Validator; +import org.apache.flink.cdc.debezium.internal.DebeziumOffset; + +import io.debezium.engine.DebeziumEngine; +import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.TableId; + +import javax.annotation.Nullable; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.Locale; +import java.util.Properties; +import java.util.Set; + +import static org.apache.flink.cdc.connectors.oracle.utils.OracleSchemaUtils.getSchema; + +/** + * The {@link OracleDebeziumSourceFunction} is a streaming data source that pulls captured change + * data from databases into Flink. + * + *

There are two workers during the runtime. One worker periodically pulls records from the + * database and pushes the records into the {@link Handover}. The other worker consumes the records + * from the {@link Handover} and convert the records to the data in Flink style. The reason why + * don't use one workers is because debezium has different behaviours in snapshot phase and + * streaming phase. + * + *

Here we use the {@link Handover} as the buffer to submit data from the producer to the + * consumer. Because the two threads don't communicate to each other directly, the error reporting + * also relies on {@link Handover}. When the engine gets errors, the engine uses the {@link + * DebeziumEngine.CompletionCallback} to report errors to the {@link Handover} and wakes up the + * consumer to check the error. However, the source function just closes the engine and wakes up the + * producer if the error is from the Flink side. + * + *

If the execution is canceled or finish(only snapshot phase), the exit logic is as same as the + * logic in the error reporting. + * + *

The source function participates in checkpointing and guarantees that no data is lost during a + * failure, and that the computation processes elements "exactly once". + * + *

Note: currently, the source function can't run in multiple parallel instances. + * + *

Please refer to Debezium's documentation for the available configuration properties: + * https://debezium.io/documentation/reference/1.9/development/engine.html#engine-properties + */ +@PublicEvolving +public class OracleDebeziumSourceFunction extends DebeziumSourceFunction { + private Set alreadySendCreateTableTables; + private String[] tableList; + private OracleSourceConfig sourceConfig; + + public OracleDebeziumSourceFunction( + DebeziumDeserializationSchema deserializer, + Properties properties, + @Nullable DebeziumOffset specificOffset, + Validator validator, + String[] tableList, + OracleSourceConfig sourceConfig) { + super(deserializer, properties, specificOffset, validator); + this.tableList = tableList; + this.sourceConfig = sourceConfig; + alreadySendCreateTableTables = new HashSet<>(); + } + + @Override + public void run(SourceContext sourceContext) throws Exception { + + Arrays.stream(tableList) + .sequential() + .forEach( + e -> { + TableId tableId = TableId.parse(e); + if (!alreadySendCreateTableTables.contains(tableId)) { + try (JdbcConnection jdbc = + DebeziumUtils.createOracleConnection(sourceConfig)) { + sendCreateTableEvent( + jdbc, tableId, (SourceContext) sourceContext); + alreadySendCreateTableTables.add(tableId); + } catch (Exception ex) { + ex.printStackTrace(); + } + } + }); + super.run(sourceContext); + } + + private void sendCreateTableEvent( + JdbcConnection jdbc, TableId tableId, SourceContext sourceContext) { + Schema schema = getSchema(jdbc, tableId); + sourceContext.collect( + new CreateTableEvent( + org.apache.flink.cdc.common.event.TableId.tableId( + tableId.catalog().toUpperCase(Locale.ROOT), + tableId.table().toUpperCase(Locale.ROOT)), + schema)); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/reader/OraclePipelineRecordEmitter.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/reader/OraclePipelineRecordEmitter.java new file mode 100644 index 00000000000..3fdfb1aa6fe --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/reader/OraclePipelineRecordEmitter.java @@ -0,0 +1,91 @@ +/* + * 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.oracle.source.reader; + +import org.apache.flink.api.connector.source.SourceOutput; +import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.connectors.base.source.meta.offset.OffsetFactory; +import org.apache.flink.cdc.connectors.base.source.meta.split.SourceSplitState; +import org.apache.flink.cdc.connectors.base.source.reader.IncrementalSourceRecordEmitter; +import org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfig; +import org.apache.flink.cdc.connectors.oracle.utils.DebeziumUtils; +import org.apache.flink.cdc.connectors.oracle.utils.OracleSchemaUtils; +import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; + +import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.TableId; +import org.apache.kafka.connect.source.SourceRecord; + +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; + +/** + * A builder to build a SourceFunction which can read snapshot and continue to consume binlog for + * PostgreSQL. + */ +public class OraclePipelineRecordEmitter extends IncrementalSourceRecordEmitter { + private static final long serialVersionUID = 1L; + private List tableList; + private OracleSourceConfig sourceConfig; + private List createTableEventCache = null; + private boolean alreadySendCreateTableForBinlogSplit = false; + + public OraclePipelineRecordEmitter( + DebeziumDeserializationSchema debeziumDeserializationSchema, + boolean includeSchemaChanges, + OffsetFactory offsetFactory, + OracleSourceConfig sourceConfig) { + super(debeziumDeserializationSchema, null, includeSchemaChanges, offsetFactory); + this.tableList = sourceConfig.getTableList(); + this.sourceConfig = sourceConfig; + this.createTableEventCache = new ArrayList<>(); + try (JdbcConnection jdbc = DebeziumUtils.createOracleConnection(sourceConfig)) { + + List capturedTableIds = new ArrayList<>(); + for (String table : tableList) { + TableId capturedTableId = TableId.parse(table.toUpperCase(Locale.ROOT)); + capturedTableIds.add(capturedTableId); + } + for (TableId tableId : capturedTableIds) { + Schema schema = OracleSchemaUtils.getSchema(jdbc, tableId); + createTableEventCache.add( + new CreateTableEvent( + org.apache.flink.cdc.common.event.TableId.tableId( + tableId.catalog(), tableId.table()), + schema)); + } + } catch (SQLException e) { + throw new RuntimeException("Cannot start emitter to fetch table schema.", e); + } + } + + @Override + protected void processElement( + SourceRecord element, SourceOutput output, SourceSplitState splitState) + throws Exception { + if (!alreadySendCreateTableForBinlogSplit) { + createTableEventCache.forEach(output::collect); + alreadySendCreateTableForBinlogSplit = true; + } + super.processElement(element, output, splitState); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/reader/OracleSourceReader.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/reader/OracleSourceReader.java new file mode 100644 index 00000000000..11666a064c7 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/reader/OracleSourceReader.java @@ -0,0 +1,210 @@ +/* + * 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.oracle.source.reader; + +import org.apache.flink.cdc.connectors.base.options.StartupOptions; +import org.apache.flink.cdc.connectors.oracle.OracleValidator; +import org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfig; +import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.DebeziumSourceFunction; + +import io.debezium.connector.oracle.OracleConnector; + +import javax.annotation.Nullable; + +import java.util.Properties; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A builder to build a SourceFunction which can read snapshot and continue to consume log miner. + */ +public class OracleSourceReader { + + private static final String DATABASE_SERVER_NAME = "oracle_logminer"; + + public static Builder builder() { + return new Builder<>(); + } + + /** Builder class of {@link OracleSourceReader}. */ + public static class Builder { + + private Integer port = 1521; // default 1521 port + private String hostname; + private String database; + private String username; + private String password; + private String url; + private String[] tableList; + private String[] schemaList; + private Properties dbzProperties; + private StartupOptions startupOptions = StartupOptions.initial(); + private DebeziumDeserializationSchema deserializer; + + private OracleSourceConfig sourceConfig; + + public Builder hostname(@Nullable String hostname) { + this.hostname = hostname; + return this; + } + + /** Integer port number of the Oracle database server. */ + public Builder port(@Nullable Integer port) { + this.port = port; + return this; + } + + /** Url to use when connecting to the Oracle database server. */ + public Builder url(@Nullable String url) { + this.url = url; + return this; + } + + /** + * An optional list of regular expressions that match database names to be monitored; any + * database name not included in the whitelist will be excluded from monitoring. By default + * all databases will be monitored. + */ + public Builder database(String database) { + this.database = database; + return this; + } + + public Builder sourceConfig(OracleSourceConfig sourceConfig) { + this.sourceConfig = sourceConfig; + return this; + } + + /** + * An optional list of regular expressions that match fully-qualified table identifiers for + * tables to be monitored; any table not included in the list will be excluded from + * monitoring. Each identifier is of the form schemaName.tableName. By default the connector + * will monitor every non-system table in each monitored database. + */ + public Builder tableList(String... tableList) { + this.tableList = tableList; + return this; + } + + /** + * An optional list of regular expressions that match schema names to be monitored; any + * schema name not included in the whitelist will be excluded from monitoring. By default + * all non-system schemas will be monitored. + */ + public Builder schemaList(String... schemaList) { + this.schemaList = schemaList; + return this; + } + + /** Name of the Oracle database to use when connecting to the Oracle database server. */ + public Builder username(String username) { + this.username = username; + return this; + } + + /** Password to use when connecting to the Oracle database server. */ + public Builder password(String password) { + this.password = password; + return this; + } + + /** The Debezium Oracle connector properties. For example, "snapshot.mode". */ + public Builder debeziumProperties(Properties properties) { + this.dbzProperties = properties; + return this; + } + + /** + * The deserializer used to convert from consumed {@link + * org.apache.kafka.connect.source.SourceRecord}. + */ + public Builder deserializer(DebeziumDeserializationSchema deserializer) { + this.deserializer = deserializer; + return this; + } + + /** Specifies the startup options. */ + public Builder startupOptions(StartupOptions startupOptions) { + this.startupOptions = startupOptions; + return this; + } + + public DebeziumSourceFunction build() { + Properties props = new Properties(); + props.setProperty("connector.class", OracleConnector.class.getCanonicalName()); + // Logical name that identifies and provides a namespace for the particular Oracle + // database server being + // monitored. The logical name should be unique across all other connectors, since it is + // used as a prefix + // for all Kafka topic names emanating from this connector. Only alphanumeric characters + // and + // underscores should be used. + props.setProperty("database.server.name", DATABASE_SERVER_NAME); + props.setProperty("database.user", checkNotNull(username)); + props.setProperty("database.password", checkNotNull(password)); + + if (url != null) { + props.setProperty("database.url", url); + } + if (hostname != null) { + props.setProperty("database.hostname", hostname); + } + if (port != null) { + props.setProperty("database.port", String.valueOf(port)); + } + props.setProperty("database.history.skip.unparseable.ddl", String.valueOf(true)); + props.setProperty("database.dbname", checkNotNull(database)); + if (schemaList != null) { + props.setProperty("schema.include.list", String.join(",", schemaList)); + } + if (tableList != null) { + props.setProperty("table.include.list", String.join(",", tableList)); + } + // we need this in order not to lose any transaction during snapshot to streaming switch + props.setProperty("internal.log.mining.transaction.snapshot.boundary.mode", "all"); + + switch (startupOptions.startupMode) { + case INITIAL: + props.setProperty("snapshot.mode", "initial"); + break; + + case LATEST_OFFSET: + props.setProperty("snapshot.mode", "schema_only"); + break; + + default: + throw new UnsupportedOperationException(); + } + + if (dbzProperties != null) { + props.putAll(dbzProperties); + } + + if (url == null) { + checkNotNull(hostname, "hostname is required when url is not configured"); + props.setProperty("database.hostname", hostname); + checkNotNull(port, "port is required when url is not configured"); + props.setProperty("database.port", String.valueOf(port)); + } + + return new OracleDebeziumSourceFunction( + deserializer, props, null, new OracleValidator(props), tableList, sourceConfig); + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/reader/OracleTableSourceReader.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/reader/OracleTableSourceReader.java new file mode 100644 index 00000000000..0a4b523e30a --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/reader/OracleTableSourceReader.java @@ -0,0 +1,49 @@ +/* + * 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.oracle.source.reader; + +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.cdc.connectors.base.config.JdbcSourceConfig; +import org.apache.flink.cdc.connectors.base.source.reader.IncrementalSourceReader; +import org.apache.flink.cdc.connectors.oracle.source.OracleDialect; +import org.apache.flink.cdc.connectors.oracle.source.OracleSourceBuilder; +import org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfigFactory; +import org.apache.flink.cdc.connectors.oracle.source.meta.offset.RedoLogOffsetFactory; +import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; + +/** + * The basic source of Incremental Snapshot framework for JDBC datasource, it is based on FLIP-27 + * and Watermark Signal Algorithm which supports parallel reading snapshot of table and then + * continue to capture data change by streaming reading. + */ +public class OracleTableSourceReader extends OracleSourceBuilder.OracleIncrementalSource { + + public OracleTableSourceReader( + OracleSourceConfigFactory configFactory, + DebeziumDeserializationSchema deserializationSchema, + RedoLogOffsetFactory offsetFactory, + OracleDialect dataSourceDialect) { + super(configFactory, deserializationSchema, offsetFactory, dataSourceDialect); + } + + @Override + public IncrementalSourceReader createReader( + SourceReaderContext readerContext) throws Exception { + return super.createReader(readerContext); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/utils/DebeziumUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/utils/DebeziumUtils.java new file mode 100644 index 00000000000..49837e2733a --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/utils/DebeziumUtils.java @@ -0,0 +1,63 @@ +/* + * 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.oracle.utils; + +import org.apache.flink.cdc.connectors.oracle.source.OracleDialect; +import org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfig; +import org.apache.flink.util.FlinkRuntimeException; + +import io.debezium.jdbc.JdbcConnection; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.SQLException; +import java.util.HashMap; +import java.util.Map; + +/** Utilities related to Debezium. */ +public class DebeziumUtils { + private static final Logger LOG = LoggerFactory.getLogger(DebeziumUtils.class); + + /** Creates a new {@link JdbcConnection}, but not open the connection. */ + public static JdbcConnection createOracleConnection(OracleSourceConfig sourceConfig) { + OracleDialect dialect = new OracleDialect(); + return dialect.openJdbcConnection(sourceConfig); + } + + private static Map querySystemVariables( + JdbcConnection connection, String statement) { + final Map variables = new HashMap<>(); + try { + connection.query( + statement, + rs -> { + while (rs.next()) { + String varName = rs.getString(1); + String value = rs.getString(2); + if (varName != null && value != null) { + variables.put(varName, value); + } + } + }); + } catch (SQLException e) { + throw new FlinkRuntimeException("Error reading oracle variables: " + e.getMessage(), e); + } + + return variables; + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/utils/OracleSchemaUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/utils/OracleSchemaUtils.java new file mode 100644 index 00000000000..e9097e8e881 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/utils/OracleSchemaUtils.java @@ -0,0 +1,232 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.connectors.oracle.utils; + +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.schema.Column; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.connectors.oracle.dto.ColumnInfo; +import org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfig; +import org.apache.flink.cdc.connectors.oracle.source.utils.OracleSchema; + +import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.ColumnEditor; +import io.debezium.relational.Table; +import io.debezium.relational.history.TableChanges; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Locale; +import java.util.stream.Collectors; + +/** Utilities for converting from debezium {@link Table} types to {@link Schema}. */ +public class OracleSchemaUtils { + + private static final Logger LOG = LoggerFactory.getLogger(OracleSchemaUtils.class); + + public static List listTables( + OracleSourceConfig sourceConfig, @Nullable String dbName) { + try (JdbcConnection jdbc = DebeziumUtils.createOracleConnection(sourceConfig)) { + List databases = + dbName != null ? Collections.singletonList(dbName) : listDatabases(jdbc); + + List tableIds = new ArrayList<>(); + for (String database : databases) { + tableIds.addAll(listTables(jdbc, database)); + } + return tableIds; + } catch (SQLException e) { + throw new RuntimeException("Error to list databases: " + e.getMessage(), e); + } + } + + public static List listDatabases(OracleSourceConfig sourceConfig) { + try (JdbcConnection jdbc = DebeziumUtils.createOracleConnection(sourceConfig)) { + return listDatabases(jdbc); + } catch (SQLException e) { + throw new RuntimeException("Error to list databases: " + e.getMessage(), e); + } + } + + public static List listDatabases(JdbcConnection jdbc) throws SQLException { + // READ DATABASE NAMES + LOG.info("Read list of available schemas"); + final List schemaNames = new ArrayList<>(); + jdbc.query( + "SELECT username FROM all_users", + rs -> { + while (rs.next()) { + schemaNames.add(rs.getString(1)); + } + }); + LOG.info("\t list of available databases are: {}", schemaNames); + return schemaNames; + } + + public static List listTables(JdbcConnection jdbc, String schemaName) + throws SQLException { + // ---------------- + // READ TABLE NAMES + // ---------------- + // Get the list of table IDs for each schema. + LOG.info("Read list of available tables in {}", schemaName); + final List tableIds = new ArrayList<>(); + jdbc.query( + "SELECT table_name FROM all_tables WHERE owner ='" + schemaName.toUpperCase() + "'", + statementFactory -> { + Statement statement = statementFactory.createStatement(); + return statement; + }, + rs -> { + while (rs.next()) { + tableIds.add( + TableId.tableId( + schemaName.toLowerCase(Locale.ROOT), + rs.getString(1).toLowerCase(Locale.ROOT))); + } + }); + LOG.info("\t list of available tables are: {}", tableIds); + return tableIds; + } + + public static Schema getTableSchema(TableId tableId, OracleSourceConfig sourceConfig) { + try { + // fetch table schemas + JdbcConnection jdbc = DebeziumUtils.createOracleConnection(sourceConfig); + OracleSchema oracleSchema = new OracleSchema(); + TableChanges.TableChange tableSchema = + oracleSchema.getTableSchema(jdbc, toDbzTableId(tableId)); + return toSchema(tableSchema.getTable()); + } catch (Exception e) { + throw new RuntimeException("Error to get table schema: " + e.getMessage(), e); + } + } + + public static Schema toSchema(Table table) { + List columns = + table.columns().stream() + .map(OracleSchemaUtils::toColumn) + .collect(Collectors.toList()); + + return Schema.newBuilder() + .setColumns(columns) + .primaryKey(table.primaryKeyColumnNames()) + .comment(table.comment()) + .build(); + } + + public static Column toColumn(io.debezium.relational.Column column) { + return Column.physicalColumn( + column.name(), OracleTypeUtils.fromDbzColumn(column), column.comment()); + } + + public static io.debezium.relational.TableId toDbzTableId(TableId tableId) { + return new io.debezium.relational.TableId( + tableId.getSchemaName(), null, tableId.getTableName()); + } + + private OracleSchemaUtils() {} + + private static io.debezium.relational.Column columnInfotoColumn(ColumnInfo columnInfo) { + Integer length = + columnInfo.getDataPrecision() == 0 || columnInfo.getDataPrecision() == null + ? columnInfo.getDataLength() + : columnInfo.getDataPrecision(); + ColumnEditor editor = + io.debezium.relational.Column.editor() + .name(columnInfo.getColumnName()) + .type(columnInfo.getDataType()) + .length(length) + .scale(columnInfo.getDataScale()); + return editor.create(); + } + + public static Schema getSchema(JdbcConnection jdbc, io.debezium.relational.TableId tableId) { + List columns = showCreateTable(jdbc, tableId); + List pks = getTablePks(jdbc, tableId); + List list = new ArrayList<>(); + for (ColumnInfo columnInfo : columns) { + DataType dataType = null; + dataType = OracleTypeUtils.fromDbzColumn(columnInfotoColumn(columnInfo)); + org.apache.flink.cdc.common.schema.Column column = + org.apache.flink.cdc.common.schema.Column.metadataColumn( + columnInfo.getColumnName().toLowerCase(Locale.ROOT), dataType); + list.add(column); + } + return Schema.newBuilder().setColumns(list).primaryKey(pks).build(); + } + + public static List showCreateTable( + JdbcConnection jdbc, io.debezium.relational.TableId tableId) { + List list = new ArrayList<>(); + final String showCreateTableQuery = + String.format( + "select COLUMN_NAME,DATA_TYPE,DATA_LENGTH,DATA_PRECISION,DATA_SCALE from all_tab_columns where Table_Name='%s' and OWNER='%s' order by COLUMN_ID", + tableId.table().toUpperCase(), tableId.catalog().toUpperCase()); + try { + return jdbc.queryAndMap( + showCreateTableQuery, + rs -> { + while (rs.next()) { + ColumnInfo columnInfo = new ColumnInfo(); + columnInfo.setColumnName(rs.getString(1)); + columnInfo.setDataType(rs.getString(2)); + columnInfo.setDataLength(rs.getInt(3)); + columnInfo.setDataPrecision(rs.getInt(4)); + columnInfo.setDataScale(rs.getInt(5)); + list.add(columnInfo); + } + return list; + }); + } catch (SQLException e) { + throw new RuntimeException( + String.format("Failed to show create table for %s", tableId), e); + } + } + + public static List getTablePks( + JdbcConnection jdbc, io.debezium.relational.TableId tableId) { + List list = new ArrayList<>(); + final String showCreateTableQuery = + String.format( + "SELECT COLUMN_NAME FROM all_constraints cons, all_cons_columns cols WHERE cols.table_name = '%s' and cols.OWNER='%s' AND cons.constraint_type = 'P' AND cons.constraint_name = cols.constraint_name AND cons.owner = cols.owner ORDER BY cols.table_name, cols.position ", + tableId.table().toUpperCase(), tableId.catalog().toUpperCase()); + try { + return jdbc.queryAndMap( + showCreateTableQuery, + rs -> { + while (rs.next()) { + String columnName = null; + columnName = rs.getString(1); + list.add(columnName.toLowerCase(Locale.ROOT)); + } + return list; + }); + } catch (SQLException e) { + throw new RuntimeException(String.format("Failed to get table pks for %s", tableId), e); + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/utils/OracleTypeUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/utils/OracleTypeUtils.java new file mode 100644 index 00000000000..ffaec926e09 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/utils/OracleTypeUtils.java @@ -0,0 +1,133 @@ +/* + * 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.oracle.utils; + +import org.apache.flink.cdc.common.types.BigIntType; +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.common.types.DataTypes; +import org.apache.flink.cdc.common.types.DecimalType; +import org.apache.flink.cdc.common.types.DoubleType; +import org.apache.flink.cdc.common.types.FloatType; +import org.apache.flink.cdc.common.types.LocalZonedTimestampType; +import org.apache.flink.cdc.common.types.TimestampType; +import org.apache.flink.cdc.common.types.VarCharType; +import org.apache.flink.cdc.common.types.ZonedTimestampType; + +import io.debezium.relational.Column; + +/** Utilities for converting from oracle types to {@link DataType}s. */ +public class OracleTypeUtils { + + /** Returns a corresponding Flink data type from a debezium {@link Column}. */ + public static DataType fromDbzColumn(Column column) { + DataType dataType = convertFromColumn(column); + if (column.isOptional()) { + return dataType; + } else { + return dataType.notNull(); + } + } + + /** + * Returns a corresponding Flink data type from a debezium {@link Column} with nullable always + * be true. + */ + private static DataType convertFromColumn(Column column) { + String type = column.typeName(); + DataType dataType; + switch (type.toUpperCase()) { + case "VARCHAR2": + case "VARCHAR": + case "NVARCHAR2": + case "CHAR": + case "NCHAR": + dataType = new VarCharType(column.length()); + break; + case "BLOB": + case "CLOB": + case "TEXT": + dataType = DataTypes.STRING(); + break; + case "NUMBER": + dataType = + column.length() == 0 || column.scale().get() <= 0 + ? new BigIntType() + : new DecimalType(column.length(), column.scale().get()); + break; + case "LONG": + dataType = new BigIntType(); + break; + case "DATE": + dataType = new TimestampType(6); + break; + case "FLOAT": + case "BINARY_FLOAT": + dataType = new FloatType(); + break; + case "BINARY_DOUBLE": + case "DOUBLE": + dataType = new DoubleType(); + break; + case "TIMESTAMP(1)": + dataType = new TimestampType(1); + break; + case "TIMESTAMP(2)": + dataType = new TimestampType(2); + break; + case "TIMESTAMP(3)": + dataType = new TimestampType(3); + break; + case "TIMESTAMP(4)": + dataType = new TimestampType(4); + break; + case "TIMESTAMP(6)": + dataType = new TimestampType(6); + break; + case "TIMESTAMP(9)": + dataType = new TimestampType(9); + break; + case "TIMESTAMP(9) WITH TIME ZONE": + dataType = new ZonedTimestampType(9); + break; + case "TIMESTAMP(6) WITH TIME ZONE": + dataType = new ZonedTimestampType(6); + break; + case "TIMESTAMP(3) WITH TIME ZONE": + dataType = new ZonedTimestampType(3); + break; + case "TIMESTAMP(13) WITH TIME ZONE": + dataType = new ZonedTimestampType(13); + break; + case "TIMESTAMP(6) WITH LOCAL TIME ZONE": + dataType = new LocalZonedTimestampType(6); + break; + case "INTERVAL YEAR(2) TO MONTH": + case "INTERVAL DAY(3) TO SECOND(2)": + dataType = new BigIntType(); + break; + case "XMLTYPE": + dataType = new VarCharType(); + break; + default: + throw new RuntimeException("Unsupported data type:" + type); + } + return dataType; + } + + private OracleTypeUtils() {} +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/resources/META-INF/services/org.apache.flink.cdc.common.factories.Factory b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/resources/META-INF/services/org.apache.flink.cdc.common.factories.Factory new file mode 100644 index 00000000000..abe2a9f920a --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/resources/META-INF/services/org.apache.flink.cdc.common.factories.Factory @@ -0,0 +1,15 @@ +# Copyright 2023 Ververica Inc. +# +# Licensed 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. + +org.apache.flink.cdc.connectors.oracle.factory.OracleDataSourceFactory diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleDataSourceFactoryTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleDataSourceFactoryTest.java new file mode 100644 index 00000000000..4dabfdbca74 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleDataSourceFactoryTest.java @@ -0,0 +1,121 @@ +/* + * 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.oracle.source; + +import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.factories.Factory; +import org.apache.flink.cdc.connectors.oracle.factory.OracleDataSourceFactory; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.testcontainers.lifecycle.Startables; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.stream.Stream; + +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.DATABASE; +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.HOSTNAME; +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.PASSWORD; +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.PORT; +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.SCHEMALIST; +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.TABLES; +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.USERNAME; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Unit tests for {@link OracleDataSourceFactory}. */ +public class OracleDataSourceFactoryTest extends OracleSourceTestBase { + + @BeforeAll + public static void beforeClass() { + LOG.info("Starting oracle19c containers..."); + Startables.deepStart(Stream.of(ORACLE_CONTAINER)).join(); + LOG.info("Container oracle19c is started."); + } + + @AfterAll + public static void afterClass() { + LOG.info("Stopping oracle19c containers..."); + ORACLE_CONTAINER.stop(); + LOG.info("Container oracle19c is stopped."); + } + + @Test + public void testCreateSource() { + Map options = new HashMap<>(); + options.put(HOSTNAME.key(), ORACLE_CONTAINER.getHost()); + options.put(PORT.key(), String.valueOf(ORACLE_CONTAINER.getOraclePort())); + options.put(USERNAME.key(), CONNECTOR_USER); + options.put(PASSWORD.key(), CONNECTOR_PWD); + options.put(TABLES.key(), "debezium.products,debezium.category"); + options.put(DATABASE.key(), ORACLE_CONTAINER.getDatabaseName()); + options.put(SCHEMALIST.key(), "DEBEZIUM"); + Factory.Context context = new MockContext(Configuration.fromMap(options)); + + OracleDataSourceFactory factory = new OracleDataSourceFactory(); + OracleDataSource dataSource = (OracleDataSource) factory.createDataSource(context); + assertThat(dataSource.getSourceConfig().getTableList()) + .isEqualTo(Arrays.asList("debezium.products", "debezium.category")); + } + + @Test + public void testNoMatchedTable() { + Map options = new HashMap<>(); + options.put(HOSTNAME.key(), ORACLE_CONTAINER.getHost()); + options.put(PORT.key(), String.valueOf(ORACLE_CONTAINER.getOraclePort())); + options.put(USERNAME.key(), CONNECTOR_USER); + options.put(PASSWORD.key(), CONNECTOR_PWD); + options.put(SCHEMALIST.key(), "DEBEZIUM"); + options.put(DATABASE.key(), ORACLE_CONTAINER.getDatabaseName()); + String tables = "DEBEZIUM.TEST"; + options.put(TABLES.key(), tables); + Factory.Context context = new MockContext(Configuration.fromMap(options)); + + OracleDataSourceFactory factory = new OracleDataSourceFactory(); + assertThatThrownBy(() -> factory.createDataSource(context)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot find any table by the option 'tables' = " + tables); + } + + class MockContext implements Factory.Context { + + Configuration factoryConfiguration; + + public MockContext(Configuration factoryConfiguration) { + this.factoryConfiguration = factoryConfiguration; + } + + @Override + public Configuration getFactoryConfiguration() { + return factoryConfiguration; + } + + @Override + public Configuration getPipelineConfiguration() { + return null; + } + + @Override + public ClassLoader getClassLoader() { + return this.getClassLoader(); + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleFullTypesITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleFullTypesITCase.java new file mode 100644 index 00000000000..ffbac8785b7 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleFullTypesITCase.java @@ -0,0 +1,338 @@ +/* + * 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.oracle.source; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.data.DecimalData; +import org.apache.flink.cdc.common.data.LocalZonedTimestampData; +import org.apache.flink.cdc.common.data.RecordData; +import org.apache.flink.cdc.common.data.TimestampData; +import org.apache.flink.cdc.common.data.binary.BinaryStringData; +import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.cdc.common.event.DataChangeEvent; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.factories.Factory; +import org.apache.flink.cdc.common.source.FlinkSourceProvider; +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.common.types.DataTypes; +import org.apache.flink.cdc.common.types.RowType; +import org.apache.flink.cdc.connectors.oracle.factory.OracleDataSourceFactory; +import org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfigFactory; +import org.apache.flink.cdc.connectors.oracle.table.OracleReadableMetaData; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.CloseableIterator; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.testcontainers.lifecycle.Startables; + +import java.math.BigDecimal; +import java.sql.Connection; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Stream; + +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.DATABASE; +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.HOSTNAME; +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.METADATA_LIST; +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.PASSWORD; +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.PORT; +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.SCHEMALIST; +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.TABLES; +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.USERNAME; +import static org.assertj.core.api.Assertions.assertThat; + +/** IT case for Oracle event source. */ +public class OracleFullTypesITCase extends OracleSourceTestBase { + + private final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(); + + @BeforeAll + public static void beforeClass() { + LOG.info("Starting oracle19c containers..."); + Startables.deepStart(Stream.of(ORACLE_CONTAINER)).join(); + LOG.info("Container oracle19c is started."); + } + + @AfterAll + public static void afterClass() { + LOG.info("Stopping oracle19c containers..."); + ORACLE_CONTAINER.stop(); + LOG.info("Container oracle19c is stopped."); + } + + @BeforeEach + public void before() throws Exception { + env.setParallelism(DEFAULT_PARALLELISM); + env.enableCheckpointing(200); + env.setRestartStrategy(RestartStrategies.noRestart()); + Connection conn = getJdbcConnectionAsDBA(); + conn.createStatement().execute("GRANT ANALYZE ANY TO " + CONNECTOR_USER); + } + + @Test + public void testOracleCommonDataTypes() throws Throwable { + testCommonDataTypes(); + } + + private void testCommonDataTypes() throws Exception { + createAndInitialize("column_type_test.sql"); + CloseableIterator iterator = + env.fromSource( + getFlinkSourceProvider(new String[] {"FULL_TYPES"}).getSource(), + WatermarkStrategy.noWatermarks(), + "Event-Source") + .executeAndCollect(); + + Object[] expectedSnapshot = + new Object[] { + DecimalData.fromBigDecimal(new BigDecimal("1"), 9, 0), + BinaryStringData.fromString("vc2"), + BinaryStringData.fromString("vc2"), + BinaryStringData.fromString("nvc2"), + BinaryStringData.fromString("c "), + BinaryStringData.fromString("nc "), + new Float(1.1), + new Double(2.22), + DecimalData.fromBigDecimal(new BigDecimal(3.33), 10, 2), + DecimalData.fromBigDecimal(new BigDecimal(8.888), 10, 3), + DecimalData.fromBigDecimal(new BigDecimal(4.4444), 10, 6), + DecimalData.fromBigDecimal(new BigDecimal(5.555), 10, 3), + DecimalData.fromBigDecimal(new BigDecimal(6.66), 10, 2), + DecimalData.fromBigDecimal(new BigDecimal(1234.567891), 10, 6), + DecimalData.fromBigDecimal(new BigDecimal(1234.567891), 10, 6), + DecimalData.fromBigDecimal(new BigDecimal(77.323), 10, 3), + DecimalData.fromBigDecimal(new BigDecimal(1), 10, 0), + DecimalData.fromBigDecimal(new BigDecimal(22), 10, 0), + DecimalData.fromBigDecimal(new BigDecimal(333), 10, 0), + DecimalData.fromBigDecimal(new BigDecimal(4444), 10, 0), + DecimalData.fromBigDecimal(new BigDecimal(5555), 10, 0), + DecimalData.fromBigDecimal(new BigDecimal(1), 10, 0), + DecimalData.fromBigDecimal(new BigDecimal(99), 10, 0), + DecimalData.fromBigDecimal(new BigDecimal(9999), 10, 0), + DecimalData.fromBigDecimal(new BigDecimal(999999999), 38, 0), + DecimalData.fromBigDecimal(new BigDecimal(999999999999999999L), 38, 0), + new Long(90), + new Long(9900), + new Long(999999990), + new Long(999999999999999900L), + BinaryStringData.fromString("9.99999999999999999999999999999999999E+37"), + TimestampData.fromLocalDateTime( + LocalDateTime.parse( + "2022-10-30 00:00:00", + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"))), + TimestampData.fromLocalDateTime( + LocalDateTime.parse( + "2022-10-30 12:34:56.00789", + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSSSS"))), + TimestampData.fromLocalDateTime( + LocalDateTime.parse( + "2022-10-30 12:34:56.13", + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SS"))), + TimestampData.fromLocalDateTime( + LocalDateTime.parse( + "2022-10-30 12:34:56.1255", + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSSS"))), + TimestampData.fromLocalDateTime( + LocalDateTime.parse( + "2022-10-30 12:34:56.125457", + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSSSSS"))), + TimestampData.fromLocalDateTime( + LocalDateTime.parse( + "2022-10-30 12:34:56.00789", + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSSSS"))), + LocalZonedTimestampData.fromInstant( + LocalDateTime.parse( + "2022-10-30 01:34:56.00789", + DateTimeFormatter.ofPattern( + "yyyy-MM-dd HH:mm:ss.SSSSS")) + .atZone(ZoneId.of("Asia/Shanghai")) + .toInstant()), + new Long(-110451600000000L), + new Long(-93784560000L), + BinaryStringData.fromString( + "\n test xmlType\n\n") + }; + // skip CreateTableEvent + List snapshotResults = fetchResultsAndCreateTableEvent(iterator, 1).f0; + RecordData snapshotRecord = ((DataChangeEvent) snapshotResults.get(0)).after(); + Object[] objects = recordFields(snapshotRecord, COMMON_TYPES); + assertThat(objects).isEqualTo(expectedSnapshot); + } + + private FlinkSourceProvider getFlinkSourceProvider(String[] captureTables) { + String[] captureTableIds = + Arrays.stream(captureTables) + .map(tableName -> "DEBEZIUM." + tableName) + .toArray(String[]::new); + + Map options = new HashMap<>(); + options.put(HOSTNAME.key(), ORACLE_CONTAINER.getHost()); + options.put(PORT.key(), String.valueOf(ORACLE_CONTAINER.getOraclePort())); + options.put(USERNAME.key(), CONNECTOR_USER); + options.put(PASSWORD.key(), CONNECTOR_PWD); + options.put(TABLES.key(), "DEBEZIUM\\.FULL_TYPES"); + options.put(DATABASE.key(), ORACLE_CONTAINER.getDatabaseName()); + options.put(SCHEMALIST.key(), "DEBEZIUM"); + options.put(METADATA_LIST.key(), "op_ts"); + Factory.Context context = new MockContext(Configuration.fromMap(options)); + + OracleSourceConfigFactory configFactory = new OracleSourceConfigFactory(); + configFactory.username(CONNECTOR_USER); + configFactory.password(CONNECTOR_PWD); + configFactory.port(ORACLE_CONTAINER.getOraclePort()); + configFactory.databaseList(ORACLE_CONTAINER.getDatabaseName()); + configFactory.schemaList("DEBEZIUM"); + configFactory.hostname(ORACLE_CONTAINER.getHost()); + configFactory.tableList("DEBEZIUM\\.FULL_TYPES"); + List readableMetadataList = + OracleDataSourceFactory.listReadableMetadata("op_ts"); + return (FlinkSourceProvider) + new OracleDataSource( + configFactory, + context.getFactoryConfiguration(), + captureTableIds, + readableMetadataList) + .getEventSourceProvider(); + } + + private static final RowType COMMON_TYPES = + RowType.of( + DataTypes.DECIMAL(9, 0).notNull(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.FLOAT(), + DataTypes.DOUBLE(), + DataTypes.DECIMAL(10, 2), + DataTypes.DECIMAL(10, 3), + DataTypes.DECIMAL(10, 6), + DataTypes.DECIMAL(10, 3), + DataTypes.DECIMAL(10, 2), + DataTypes.DECIMAL(10, 6), + DataTypes.DECIMAL(10, 6), + DataTypes.DECIMAL(10, 3), + DataTypes.DECIMAL(10, 0), + DataTypes.DECIMAL(10, 0), + DataTypes.DECIMAL(10, 0), + DataTypes.DECIMAL(38, 0), + DataTypes.DECIMAL(38, 0), + DataTypes.DECIMAL(1, 0), + DataTypes.DECIMAL(2, 0), + DataTypes.DECIMAL(4, 0), + DataTypes.DECIMAL(9, 0), + DataTypes.DECIMAL(18, 0), + DataTypes.BIGINT(), + DataTypes.BIGINT(), + DataTypes.BIGINT(), + DataTypes.BIGINT(), + DataTypes.STRING(), + DataTypes.TIMESTAMP(), + DataTypes.TIMESTAMP(6), + DataTypes.TIMESTAMP(2), + DataTypes.TIMESTAMP(4), + DataTypes.TIMESTAMP(6), + DataTypes.TIMESTAMP(6), + DataTypes.TIMESTAMP_LTZ(6), + DataTypes.BIGINT(), + DataTypes.BIGINT(), + DataTypes.STRING()); + + public static List fetchResults(Iterator iter, int size) { + List result = new ArrayList<>(size); + while (size > 0 && iter.hasNext()) { + T event = iter.next(); + result.add(event); + size--; + } + return result; + } + + public static Tuple2, List> fetchResultsAndCreateTableEvent( + Iterator iter, int size) { + List result = new ArrayList<>(size); + List createTableEvents = new ArrayList<>(); + while (size > 0 && iter.hasNext()) { + T event = iter.next(); + if (event instanceof CreateTableEvent) { + createTableEvents.add((CreateTableEvent) event); + } else { + result.add(event); + size--; + } + } + return Tuple2.of(result, createTableEvents); + } + + public static Object[] recordFields(RecordData record, RowType rowType) { + int fieldNum = record.getArity(); + List fieldTypes = rowType.getChildren(); + Object[] fields = new Object[fieldNum]; + for (int i = 0; i < fieldNum; i++) { + if (record.isNullAt(i)) { + fields[i] = null; + } else { + DataType type = fieldTypes.get(i); + RecordData.FieldGetter fieldGetter; + fieldGetter = RecordData.createFieldGetter(type, i); + Object o = fieldGetter.getFieldOrNull(record); + fields[i] = o; + } + } + return fields; + } + + class MockContext implements Factory.Context { + + Configuration factoryConfiguration; + + public MockContext(Configuration factoryConfiguration) { + this.factoryConfiguration = factoryConfiguration; + } + + @Override + public Configuration getFactoryConfiguration() { + return factoryConfiguration; + } + + @Override + public Configuration getPipelineConfiguration() { + return null; + } + + @Override + public ClassLoader getClassLoader() { + return this.getClassLoader(); + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleMetadataAccessorITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleMetadataAccessorITCase.java new file mode 100644 index 00000000000..c963e5bafae --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleMetadataAccessorITCase.java @@ -0,0 +1,224 @@ +/* + * 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.oracle.source; + +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.common.types.DataTypes; +import org.apache.flink.cdc.common.types.RowType; +import org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfig; +import org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfigFactory; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.testcontainers.lifecycle.Startables; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** IT cases for {@link OracleMetadataAccessor}. */ +public class OracleMetadataAccessorITCase extends OracleSourceTestBase { + + private final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(); + + @BeforeAll + public static void beforeClass() { + LOG.info("Starting Oracle containers..."); + Startables.deepStart(Stream.of(ORACLE_CONTAINER)).join(); + LOG.info("Container Oracle is started."); + } + + @AfterAll + public static void afterClass() { + LOG.info("Stopping Oracle containers..."); + ORACLE_CONTAINER.stop(); + LOG.info("Container Oracle is stopped."); + } + + @BeforeEach + public void before() { + env.setParallelism(DEFAULT_PARALLELISM); + env.enableCheckpointing(200); + env.setRestartStrategy(RestartStrategies.noRestart()); + } + + @Test + public void testOracleAccessDatabaseAndTable() throws Exception { + testAccessDatabaseAndTable(); + } + + @Test + public void testOracleAccessCommonTypesSchema() throws Exception { + testAccessCommonTypesSchema(); + } + + private void testAccessDatabaseAndTable() throws Exception { + createAndInitialize("column_type_test.sql"); + String[] tables = new String[] {"full_types"}; + OracleMetadataAccessor metadataAccessor = getMetadataAccessor(tables); + + assertThatThrownBy(metadataAccessor::listNamespaces) + .isInstanceOf(UnsupportedOperationException.class); + + List schemas = metadataAccessor.listSchemas(null); + assertThat(schemas).contains("DEBEZIUM"); + + List actualTables = metadataAccessor.listTables(null, "DEBEZIUM"); + List expectedTables = + Arrays.stream(tables) + .map(table -> TableId.tableId("debezium", table)) + .collect(Collectors.toList()); + assertThat(actualTables).containsExactlyInAnyOrderElementsOf(expectedTables); + } + + private void testAccessCommonTypesSchema() throws Exception { + createAndInitialize("column_type_test.sql"); + String[] tables = new String[] {"FULL_TYPES"}; + OracleMetadataAccessor metadataAccessor = getMetadataAccessor(tables); + + Schema actualSchema = + metadataAccessor.getTableSchema( + TableId.tableId(ORACLE_CONTAINER.getDatabaseName(), "FULL_TYPES")); + Schema expectedSchema = + Schema.newBuilder() + .primaryKey("ID") + .fromRowDataType( + RowType.of( + new DataType[] { + DataTypes.BIGINT().notNull(), + DataTypes.VARCHAR(1000), + DataTypes.VARCHAR(1000), + DataTypes.VARCHAR(1000), + DataTypes.VARCHAR(3), + DataTypes.VARCHAR(3), + DataTypes.FLOAT(), + DataTypes.DOUBLE(), + DataTypes.FLOAT(), + DataTypes.FLOAT(), + DataTypes.DECIMAL(10, 6), + DataTypes.FLOAT(), + DataTypes.FLOAT(), + DataTypes.DECIMAL(10, 6), + DataTypes.DECIMAL(10, 6), + DataTypes.BIGINT(), + DataTypes.BIGINT(), + DataTypes.BIGINT(), + DataTypes.BIGINT(), + DataTypes.BIGINT(), + DataTypes.BIGINT(), + DataTypes.BIGINT(), + DataTypes.BIGINT(), + DataTypes.BIGINT(), + DataTypes.BIGINT(), + DataTypes.BIGINT(), + DataTypes.BIGINT(), + DataTypes.BIGINT(), + DataTypes.BIGINT(), + DataTypes.BIGINT(), + DataTypes.BIGINT(), + DataTypes.TIMESTAMP(6), + DataTypes.TIMESTAMP(6), + DataTypes.TIMESTAMP(2), + DataTypes.TIMESTAMP(4), + DataTypes.TIMESTAMP(6), + DataTypes.TIMESTAMP_TZ(6), + DataTypes.TIMESTAMP_LTZ(6), + DataTypes.BIGINT(), + DataTypes.BIGINT(), + DataTypes.VARCHAR(1) + }, + new String[] { + "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_TSTZ", + "VAL_TSLTZ", + "VAL_INT_YTM", + "VAL_INT_DTS", + "T15VARCHAR" + })) + .build(); + assertThat(actualSchema).isEqualTo(expectedSchema); + } + + private OracleMetadataAccessor getMetadataAccessor(String[] tables) { + OracleSourceConfig sourceConfig = getConfig(tables); + return new OracleMetadataAccessor(sourceConfig); + } + + private OracleSourceConfig getConfig(String[] captureTables) { + String[] captureTableIds = + Arrays.stream(captureTables) + .map(tableName -> ORACLE_CONTAINER.getDatabaseName() + "." + tableName) + .toArray(String[]::new); + OracleSourceConfigFactory configFactory = new OracleSourceConfigFactory(); + configFactory.username(CONNECTOR_USER); + configFactory.password(CONNECTOR_PWD); + configFactory.port(ORACLE_CONTAINER.getOraclePort()); + configFactory.databaseList(ORACLE_CONTAINER.getDatabaseName()); + configFactory.schemaList("DEBEZIUM"); + configFactory.hostname(ORACLE_CONTAINER.getHost()); + configFactory.tableList("DEBEZIUM\\.FULL_TYPES"); + + return configFactory.create(0); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OraclePipelineITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OraclePipelineITCase.java new file mode 100644 index 00000000000..97db1c948a7 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OraclePipelineITCase.java @@ -0,0 +1,910 @@ +/* + * 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.oracle.source; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.data.RecordData; +import org.apache.flink.cdc.common.data.binary.BinaryStringData; +import org.apache.flink.cdc.common.event.AddColumnEvent; +import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; +import org.apache.flink.cdc.common.event.ChangeEvent; +import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.cdc.common.event.DataChangeEvent; +import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.event.RenameColumnEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.factories.Factory; +import org.apache.flink.cdc.common.schema.Column; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.common.source.FlinkSourceProvider; +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.common.types.DataTypes; +import org.apache.flink.cdc.common.types.RowType; +import org.apache.flink.cdc.common.utils.SchemaUtils; +import org.apache.flink.cdc.connectors.base.options.StartupOptions; +import org.apache.flink.cdc.connectors.oracle.factory.OracleDataSourceFactory; +import org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfigFactory; +import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator; +import org.apache.flink.cdc.runtime.typeutils.EventTypeInfo; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.planner.factories.TestValuesTableFactory; +import org.apache.flink.util.CloseableIterator; + +import org.jetbrains.annotations.NotNull; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.MethodOrderer; +import org.junit.jupiter.api.Order; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestMethodOrder; +import org.testcontainers.lifecycle.Startables; + +import java.sql.Connection; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.DATABASE; +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.HOSTNAME; +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.METADATA_LIST; +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.PASSWORD; +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.PORT; +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.SCAN_STARTUP_MODE; +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.SCHEMALIST; +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.TABLES; +import static org.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.USERNAME; +import static org.assertj.core.api.Assertions.assertThat; + +/** IT tests for {@link OracleDataSource}. */ +@TestMethodOrder(MethodOrderer.OrderAnnotation.class) +public class OraclePipelineITCase extends OracleSourceTestBase { + + private StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + @BeforeAll + public static void startContainers() { + LOG.info("Starting containers..."); + Startables.deepStart(Stream.of(ORACLE_CONTAINER)).join(); + LOG.info("Containers are started."); + } + + @AfterAll + public static void stopContainers() { + LOG.info("Stopping containers..."); + ORACLE_CONTAINER.stop(); + LOG.info("Containers are stopped."); + } + + @BeforeEach + public void before() throws Exception { + TestValuesTableFactory.clearAllData(); + env.setParallelism(1); + env.setRestartStrategy(RestartStrategies.noRestart()); + Connection conn = getJdbcConnectionAsDBA(); + conn.createStatement().execute("GRANT ANALYZE ANY TO " + CONNECTOR_USER); + } + + @Test + @Order(1) + public void testParseAlterStatement() throws Exception { + Map options = new HashMap<>(); + options.put(HOSTNAME.key(), ORACLE_CONTAINER.getHost()); + options.put(PORT.key(), String.valueOf(ORACLE_CONTAINER.getOraclePort())); + options.put(USERNAME.key(), CONNECTOR_USER); + options.put(PASSWORD.key(), CONNECTOR_PWD); + options.put(TABLES.key(), "debezium.products"); + options.put(DATABASE.key(), ORACLE_CONTAINER.getDatabaseName()); + options.put(SCHEMALIST.key(), "DEBEZIUM"); + options.put(SCAN_STARTUP_MODE.key(), "initial"); + Factory.Context context = new MockContext(Configuration.fromMap(options)); + OracleSourceConfigFactory configFactory = new OracleSourceConfigFactory(); + configFactory.username(CONNECTOR_USER); + configFactory.password(CONNECTOR_PWD); + configFactory.port(ORACLE_CONTAINER.getOraclePort()); + configFactory.databaseList(ORACLE_CONTAINER.getDatabaseName()); + configFactory.schemaList("DEBEZIUM"); + configFactory.hostname(ORACLE_CONTAINER.getHost()); + configFactory.tableList("DEBEZIUM.PRODUCTS"); + configFactory.startupOptions(StartupOptions.initial()); + FlinkSourceProvider sourceProvider = + (FlinkSourceProvider) + new OracleDataSource( + configFactory, + context.getFactoryConfiguration(), + new String[] {"DEBEZIUM.PRODUCTS"}, + new ArrayList<>()) + .getEventSourceProvider(); + CloseableIterator events = + env.fromSource( + sourceProvider.getSource(), + WatermarkStrategy.noWatermarks(), + OracleDataSourceFactory.IDENTIFIER, + new EventTypeInfo()) + .executeAndCollect(); + Thread.sleep(5_000); + + TableId tableId = TableId.tableId("DEBEZIUM", "PRODUCTS"); + List expected = new ArrayList<>(); + expected.add(getProductsCreateTableEvent(tableId)); + List expectedSnapshot = + getSnapshotExpected(tableId).stream() + .map( + event -> { + DataChangeEvent dataChangeEvent = (DataChangeEvent) event; + return DataChangeEvent.insertEvent( + dataChangeEvent.tableId(), dataChangeEvent.after()); + }) + .collect(Collectors.toList()); + expected.addAll(expectedSnapshot); + try (Connection connection = getJdbcConnection(); + Statement statement = connection.createStatement()) { + expected.addAll(executeAlterAndProvideExpected(tableId, statement)); + + statement.execute( + String.format("ALTER TABLE %s.PRODUCTS ADD COLS1 VARCHAR(45)", "DEBEZIUM")); + statement.execute( + String.format("ALTER TABLE %s.PRODUCTS ADD COLS2 VARCHAR(55)", "DEBEZIUM")); + expected.add( + new AddColumnEvent( + tableId, + Arrays.asList( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn( + "COLS1", DataTypes.VARCHAR(45)))))); + expected.add( + new AddColumnEvent( + tableId, + Arrays.asList( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn( + "COLS2", DataTypes.VARCHAR(55)))))); + statement.execute( + String.format("ALTER TABLE %s.PRODUCTS ADD COLS3 VARCHAR(45)", "DEBEZIUM")); + statement.execute( + String.format("ALTER TABLE %s.PRODUCTS ADD COLS4 VARCHAR(55)", "DEBEZIUM")); + expected.add( + new AddColumnEvent( + tableId, + Arrays.asList( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn( + "COLS3", DataTypes.VARCHAR(45)))))); + expected.add( + new AddColumnEvent( + tableId, + Arrays.asList( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn( + "COLS4", DataTypes.VARCHAR(55)))))); + + statement.execute( + String.format("ALTER TABLE %s.PRODUCTS ADD COLS5 LONG NULL", "DEBEZIUM")); + expected.add( + new AddColumnEvent( + tableId, + Collections.singletonList( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn("COLS5", DataTypes.BIGINT()))))); + statement.execute( + String.format("ALTER TABLE %s.PRODUCTS DROP COLUMN COLS1", "DEBEZIUM")); + expected.add(new DropColumnEvent(tableId, Collections.singletonList("COLS1"))); + statement.execute( + String.format("ALTER TABLE %s.PRODUCTS DROP COLUMN COLS2", "DEBEZIUM")); + expected.add(new DropColumnEvent(tableId, Collections.singletonList("COLS2"))); + statement.execute( + String.format("ALTER TABLE %s.PRODUCTS DROP COLUMN COLS3", "DEBEZIUM")); + expected.add(new DropColumnEvent(tableId, Collections.singletonList("COLS3"))); + statement.execute( + String.format("ALTER TABLE %s.PRODUCTS DROP COLUMN COLS4", "DEBEZIUM")); + expected.add(new DropColumnEvent(tableId, Collections.singletonList("COLS4"))); + statement.execute( + String.format("ALTER TABLE %s.PRODUCTS DROP COLUMN COLS5", "DEBEZIUM")); + expected.add(new DropColumnEvent(tableId, Collections.singletonList("COLS5"))); + } + List actual = fetchResults(events, expected.size()); + List actualStr = + actual.stream().map(e -> e.toString()).collect(Collectors.toList()); + List expectedStr = + expected.stream().map(e -> e.toString()).collect(Collectors.toList()); + assertThat(actualStr) + .containsExactlyInAnyOrder(expectedStr.toArray(new String[expectedStr.size()])); + } + + @Test + @Order(2) + public void testInitialStartupMode() throws Exception { + Map options = new HashMap<>(); + options.put(HOSTNAME.key(), ORACLE_CONTAINER.getHost()); + options.put(PORT.key(), String.valueOf(ORACLE_CONTAINER.getOraclePort())); + options.put(USERNAME.key(), CONNECTOR_USER); + options.put(PASSWORD.key(), CONNECTOR_PWD); + options.put(TABLES.key(), "debezium.products"); + options.put(DATABASE.key(), ORACLE_CONTAINER.getDatabaseName()); + options.put(SCHEMALIST.key(), "DEBEZIUM"); + options.put(SCAN_STARTUP_MODE.key(), "initial"); + Factory.Context context = new MockContext(Configuration.fromMap(options)); + OracleSourceConfigFactory configFactory = new OracleSourceConfigFactory(); + configFactory.username(TEST_USER); + configFactory.password(TEST_PWD); + configFactory.port(ORACLE_CONTAINER.getOraclePort()); + configFactory.databaseList(ORACLE_CONTAINER.getDatabaseName()); + configFactory.schemaList("DEBEZIUM"); + configFactory.hostname(ORACLE_CONTAINER.getHost()); + configFactory.tableList("DEBEZIUM.PRODUCTS"); + configFactory.startupOptions(StartupOptions.initial()); + FlinkSourceProvider sourceProvider = + (FlinkSourceProvider) + new OracleDataSource( + configFactory, + context.getFactoryConfiguration(), + new String[] {"DEBEZIUM.PRODUCTS"}, + new ArrayList<>()) + .getEventSourceProvider(); + CloseableIterator events = + env.fromSource( + sourceProvider.getSource(), + WatermarkStrategy.noWatermarks(), + OracleDataSourceFactory.IDENTIFIER, + new EventTypeInfo()) + .executeAndCollect(); + Thread.sleep(150_000); + + TableId tableId = TableId.tableId("DEBEZIUM", "PRODUCTS"); + CreateTableEvent createTableEvent = getProductsCreateTableEvent(tableId); + + // generate snapshot data + List expectedSnapshot = new ArrayList<>(); + expectedSnapshot.add(createTableEvent); + expectedSnapshot.addAll(getSnapshotExpected(tableId)); + + List expectedBinlog = new ArrayList<>(); + List actual; + try (Connection connection = getJdbcConnection(); + Statement statement = connection.createStatement()) { + expectedBinlog.addAll(executeAlterAndProvideExpected(tableId, statement)); + + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.BIGINT().notNull(), + DataTypes.VARCHAR(255).notNull(), + DataTypes.VARCHAR(512), + DataTypes.FLOAT() + }, + new String[] {"ID", "NAME", "DESCRIPTION", "WEIGHT"}); + BinaryRecordDataGenerator generator = new BinaryRecordDataGenerator(rowType); + // insert more data + statement.execute( + String.format( + "INSERT INTO %s.products VALUES (110,'jack','13V jack',5.5)", + "debezium")); // 110 + expectedBinlog.add( + DataChangeEvent.insertEvent( + tableId, + generator.generate( + new Object[] { + 110L, + BinaryStringData.fromString("jack"), + BinaryStringData.fromString("13V jack"), + 7.7E-44f + }))); + statement.execute( + String.format( + "INSERT INTO %s.products VALUES (111,'football','31o football',6.6f)", + "debezium")); // 111 + expectedBinlog.add( + DataChangeEvent.insertEvent( + tableId, + generator.generate( + new Object[] { + 111L, + BinaryStringData.fromString("football"), + BinaryStringData.fromString("31o football"), + 1.4051863E-36f + }))); + statement.execute( + String.format( + "UPDATE %s.products SET NAME='peter', WEIGHT=6.7 WHERE id=110", + "debezium")); + expectedBinlog.add( + DataChangeEvent.updateEvent( + tableId, + generator.generate( + new Object[] { + 110L, + BinaryStringData.fromString("jack"), + BinaryStringData.fromString("13V jack"), + 7.7E-44f + }), + generator.generate( + new Object[] { + 110L, + BinaryStringData.fromString("peter"), + BinaryStringData.fromString("13V jack"), + 9.4E-44f + }))); + statement.execute(String.format("DELETE FROM %s.products WHERE id = 111", "debezium")); + expectedBinlog.add( + DataChangeEvent.deleteEvent( + tableId, + generator.generate( + new Object[] { + 111L, + BinaryStringData.fromString("football"), + BinaryStringData.fromString("31o football"), + 1.4051863E-36f + }))); + statement.execute(String.format("TRUNCATE TABLE %s.products", "DEBEZIUM")); + + // In this configuration, several subtasks might emit their corresponding + // CreateTableEvent + // to downstream. Since it is not possible to predict how many CreateTableEvents should + // we + // expect, we simply filter them out from expected sets, and assert there's at least + // one. + actual = + fetchResultsExcept( + events, + expectedSnapshot.size() + expectedBinlog.size(), + createTableEvent); + } + Map> fieldGetterMaps = new HashMap<>(); + String tableIdStr = ((CreateTableEvent) actual.get(0)).tableId().toString(); + TableId tableIdTmp = + TableId.tableId( + tableIdStr.split("\\.")[0].toUpperCase(), + tableIdStr.split("\\.")[1].toUpperCase()); + Schema schema = createTableEvent.getSchema(); + fieldGetterMaps.put(tableIdTmp, SchemaUtils.createFieldGetters(schema)); + StringBuilder actualSnapshotStr = + getResultString(actual.subList(0, expectedSnapshot.size()), fieldGetterMaps, false); + StringBuilder expectedSnapshotStr = + getResultString(expectedSnapshot, fieldGetterMaps, false); + assertThat(actualSnapshotStr.toString()).isEqualTo(expectedSnapshotStr.toString()); + StringBuilder actualBinlogStr = + getResultString( + actual.subList(expectedSnapshot.size(), actual.size()), + fieldGetterMaps, + false); + StringBuilder expectedBinlogStr = getResultString(expectedBinlog, fieldGetterMaps, false); + assertThat(actualBinlogStr.toString()).isEqualTo(expectedBinlogStr.toString()); + } + + @NotNull + private StringBuilder getResultString( + List events, + Map> fieldGetterMaps, + boolean metaFlag) { + StringBuilder sb = new StringBuilder(); + List list = new ArrayList<>(); + for (Event event : events) { + if (event instanceof DataChangeEvent) { + list.add( + convertEventToStr( + event, + fieldGetterMaps.get(((ChangeEvent) event).tableId()), + metaFlag)); + } else { + list.add(event.toString()); + } + } + Collections.sort(list); + for (int i = 0; i < list.size(); i++) { + sb.append(list.get(i)); + sb.append("\r\n"); + } + return sb; + } + + private static List fetchResultsExcept(Iterator iter, int size, T sideEvent) { + List result = new ArrayList<>(size); + List sideResults = new ArrayList<>(); + while (size > 0 && iter.hasNext()) { + T event = iter.next(); + if (!event.equals(sideEvent)) { + result.add(event); + size--; + } else { + sideResults.add(sideEvent); + } + } + // Also ensure we've received at least one or many side events. + // assertThat(sideResults).isNotEmpty(); + return result; + } + + public static String convertEventToStr( + Event event, List fieldGetters, boolean metaFlag) { + if (event instanceof SchemaChangeEvent) { + return event.toString(); + } else if (event instanceof DataChangeEvent) { + DataChangeEvent dataChangeEvent = (DataChangeEvent) event; + String eventStr = + "DataChangeEvent{" + + "tableId=" + + dataChangeEvent.tableId() + + ", before=" + + getFields(fieldGetters, dataChangeEvent.before()) + + ", after=" + + getFields(fieldGetters, dataChangeEvent.after()) + + ", op=" + + dataChangeEvent.op() + + ", meta=" + + (metaFlag ? describeMeta() : dataChangeEvent.describeMeta()) + + '}'; + return eventStr; + } + return "Event{}"; + } + + public static String describeMeta() { + StringBuilder stringBuilder = new StringBuilder("("); + Map metaMap = new HashMap(); + metaMap.put("op_ts", "0"); + stringBuilder.append(metaMap); + stringBuilder.append(")"); + return stringBuilder.toString(); + } + + private static List getFields( + List fieldGetters, RecordData recordData) { + List fields = new ArrayList<>(fieldGetters.size()); + if (recordData == null) { + return fields; + } + for (RecordData.FieldGetter fieldGetter : fieldGetters) { + fields.add(fieldGetter.getFieldOrNull(recordData)); + } + return fields; + } + + @Test + @Order(3) + public void testInitialStartupModeWithOpTs() throws Exception { + Map options = new HashMap<>(); + options.put(HOSTNAME.key(), ORACLE_CONTAINER.getHost()); + options.put(PORT.key(), String.valueOf(ORACLE_CONTAINER.getOraclePort())); + options.put(USERNAME.key(), CONNECTOR_USER); + options.put(PASSWORD.key(), CONNECTOR_PWD); + options.put(TABLES.key(), "debezium.products"); + options.put(DATABASE.key(), ORACLE_CONTAINER.getDatabaseName()); + options.put(SCHEMALIST.key(), "DEBEZIUM"); + options.put(METADATA_LIST.key(), "op_ts"); + options.put(SCAN_STARTUP_MODE.key(), "initial"); + Factory.Context context = new MockContext(Configuration.fromMap(options)); + OracleSourceConfigFactory configFactory = new OracleSourceConfigFactory(); + configFactory.username(TEST_USER); + configFactory.password(TEST_PWD); + configFactory.port(ORACLE_CONTAINER.getOraclePort()); + configFactory.databaseList(ORACLE_CONTAINER.getDatabaseName()); + configFactory.schemaList("DEBEZIUM"); + configFactory.hostname(ORACLE_CONTAINER.getHost()); + configFactory.tableList("DEBEZIUM.PRODUCTS"); + configFactory.startupOptions(StartupOptions.initial()); + FlinkSourceProvider sourceProvider = + (FlinkSourceProvider) + new OracleDataSourceFactory() + .createDataSource(context) + .getEventSourceProvider(); + CloseableIterator events = + env.fromSource( + sourceProvider.getSource(), + WatermarkStrategy.noWatermarks(), + OracleDataSourceFactory.IDENTIFIER, + new EventTypeInfo()) + .executeAndCollect(); + Thread.sleep(150_000); + TableId tableId = TableId.tableId("DEBEZIUM", "PRODUCTS"); + CreateTableEvent createTableEvent = + getProductsCreateTableEvent(TableId.tableId("debezium", "products")); + // generate snapshot data + Map meta = new HashMap<>(); + meta.put("op_ts", "0"); + List expectedSnapshot = + getSnapshotExpected(tableId).stream() + .map( + event -> { + DataChangeEvent dataChangeEvent = (DataChangeEvent) event; + return DataChangeEvent.insertEvent( + dataChangeEvent.tableId(), + dataChangeEvent.after(), + meta); + }) + .collect(Collectors.toList()); + String startTime = String.valueOf(System.currentTimeMillis()); + Thread.sleep(1000); + List expectedBinlog = new ArrayList<>(); + try (Connection connection = getJdbcConnection(); + Statement statement = connection.createStatement()) { + expectedBinlog.addAll(executeAlterAndProvideExpected(tableId, statement)); + expectedBinlog.add(createTableEvent); + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.BIGINT().notNull(), + DataTypes.VARCHAR(255).notNull(), + DataTypes.VARCHAR(512), + DataTypes.FLOAT() + }, + new String[] {"ID", "NAME", "DESCRIPTION", "WEIGHT"}); + BinaryRecordDataGenerator generator = new BinaryRecordDataGenerator(rowType); + // insert more data + statement.execute( + String.format( + "INSERT INTO %s.PRODUCTS VALUES (110,'scooter','c-10',5.5)", + "DEBEZIUM")); // 110 + expectedBinlog.add( + DataChangeEvent.insertEvent( + tableId, + generator.generate( + new Object[] { + 110L, + BinaryStringData.fromString("scooter"), + BinaryStringData.fromString("c-10"), + 7.7E-44f + }), + meta)); + statement.execute( + String.format( + "INSERT INTO %s.PRODUCTS VALUES (111,'football','c-11',6.6)", + "DEBEZIUM")); // 111 + expectedBinlog.add( + DataChangeEvent.insertEvent( + tableId, + generator.generate( + new Object[] { + 111L, + BinaryStringData.fromString("football"), + BinaryStringData.fromString("c-11"), + 9.2E-44f + }), + meta)); + statement.execute( + String.format( + "UPDATE %s.PRODUCTS SET NAME='jack', WEIGHT=6.7 WHERE id=110", + "DEBEZIUM")); + expectedBinlog.add( + DataChangeEvent.updateEvent( + tableId, + generator.generate( + new Object[] { + 110L, + BinaryStringData.fromString("scooter"), + BinaryStringData.fromString("c-10"), + 7.7E-44f + }), + generator.generate( + new Object[] { + 110L, + BinaryStringData.fromString("jack"), + BinaryStringData.fromString("c-10"), + 9.4E-44f + }), + meta)); + statement.execute(String.format("DELETE FROM %s.PRODUCTS WHERE id = 111", "DEBEZIUM")); + expectedBinlog.add( + DataChangeEvent.deleteEvent( + tableId, + generator.generate( + new Object[] { + 111L, + BinaryStringData.fromString("football"), + BinaryStringData.fromString("c-11"), + 9.2E-44f + }), + meta)); + } + List actual = fetchResults(events, expectedBinlog.size()); + assertThat(actual.get(0).toString()).isEqualTo(createTableEvent.toString()); + + Map> fieldGetterMaps = new HashMap<>(); + String tableIdStr = ((CreateTableEvent) actual.get(0)).tableId().toString(); + TableId tableId2 = + TableId.tableId( + tableIdStr.split("\\.")[0].toUpperCase(), + tableIdStr.split("\\.")[1].toUpperCase()); + Schema schema = createTableEvent.getSchema(); + fieldGetterMaps.put(tableId2, SchemaUtils.createFieldGetters(schema)); + StringBuilder actualBinlogStr = getResultString(actual, fieldGetterMaps, true); + StringBuilder expectedBinlogStr = getResultString(expectedBinlog, fieldGetterMaps, true); + assertThat(actualBinlogStr.toString()).isEqualTo(expectedBinlogStr.toString()); + List actualBinlolgs = actual; + for (int i = 0; i < expectedBinlog.size(); i++) { + if (actualBinlolgs.get(i) instanceof DataChangeEvent) { + assertThat( + Long.parseLong( + ((DataChangeEvent) actualBinlolgs.get(i)) + .meta() + .get("op_ts"))) + .isGreaterThanOrEqualTo( + Long.parseLong( + ((DataChangeEvent) expectedBinlog.get(i)) + .meta() + .get("op_ts"))); + } + } + } + + private CreateTableEvent getProductsCreateTableEvent(TableId tableId) { + return new CreateTableEvent( + tableId, + Schema.newBuilder() + .physicalColumn("id", DataTypes.BIGINT()) + .physicalColumn("name", DataTypes.VARCHAR(255)) + .physicalColumn("description", DataTypes.VARCHAR(512)) + .physicalColumn("weight", DataTypes.FLOAT()) + .primaryKey(Arrays.asList("id")) + .build()); + } + + private List getSnapshotExpected(TableId tableId) { + + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.BIGINT().notNull(), + DataTypes.VARCHAR(255).notNull(), + DataTypes.VARCHAR(512), + DataTypes.FLOAT() + }, + new String[] {"ID", "NAME", "DESCRIPTION", "WEIGHT"}); + BinaryRecordDataGenerator generator = new BinaryRecordDataGenerator(rowType); + + List snapshotExpected = new ArrayList<>(); + snapshotExpected.add( + DataChangeEvent.insertEvent( + tableId, + generator.generate( + new Object[] { + 101L, + BinaryStringData.fromString("scooter"), + BinaryStringData.fromString("Small 2-wheel scooter"), + 4.4E-43f + }))); + snapshotExpected.add( + DataChangeEvent.insertEvent( + tableId, + generator.generate( + new Object[] { + 102L, + BinaryStringData.fromString("car battery"), + BinaryStringData.fromString("12V car battery"), + 1.14E-43f + }))); + snapshotExpected.add( + DataChangeEvent.insertEvent( + tableId, + generator.generate( + new Object[] { + 103L, + BinaryStringData.fromString("12-pack drill bits"), + BinaryStringData.fromString( + "12-pack of drill bits with sizes ranging from #40 to #3"), + 1.1E-44f + }))); + snapshotExpected.add( + DataChangeEvent.insertEvent( + tableId, + generator.generate( + new Object[] { + 104L, + BinaryStringData.fromString("hammer"), + BinaryStringData.fromString("12oz carpenters hammer"), + 1.05E-43f + }))); + snapshotExpected.add( + DataChangeEvent.insertEvent( + tableId, + generator.generate( + new Object[] { + 105L, + BinaryStringData.fromString("hammer"), + BinaryStringData.fromString("14oz carpenters hammer"), + 1.226E-42f + }))); + snapshotExpected.add( + DataChangeEvent.insertEvent( + tableId, + generator.generate( + new Object[] { + 106L, + BinaryStringData.fromString("hammer"), + BinaryStringData.fromString("16oz carpenters hammer"), + 1.4E-45f + }))); + snapshotExpected.add( + DataChangeEvent.insertEvent( + tableId, + generator.generate( + new Object[] { + 107L, + BinaryStringData.fromString("rocks"), + BinaryStringData.fromString("box of assorted rocks"), + 7.4E-44f + }))); + snapshotExpected.add( + DataChangeEvent.insertEvent( + tableId, + generator.generate( + new Object[] { + 108L, + BinaryStringData.fromString("jacket"), + BinaryStringData.fromString( + "water resistent black wind breaker"), + 1.4E-45f + }))); + snapshotExpected.add( + DataChangeEvent.insertEvent( + tableId, + generator.generate( + new Object[] { + 109L, + BinaryStringData.fromString("spare tire"), + BinaryStringData.fromString("24 inch spare tire"), + 3.11E-43f + }))); + return snapshotExpected; + } + + /** + * * The final schema of table products is as follows. + * + *
+     * CREATE TABLE products (
+     *   id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,
+     *   name VARCHAR(255) NOT NULL DEFAULT 'flink',
+     *   weight FLOAT,
+     *   col1 VARCHAR(45),
+     *   col2 VARCHAR(55)
+     * );
+     * 
+ */ + private List executeAlterAndProvideExpected(TableId tableId, Statement statement) + throws SQLException { + List expected = new ArrayList<>(); + statement.execute( + String.format( + "alter table %s.products modify DESCRIPTION VARCHAR(255) DEFAULT NULL", + "debezium")); + statement.execute( + String.format( + "ALTER TABLE %s.products RENAME COLUMN DESCRIPTION TO DESCPT", "debezium")); + + expected.add( + new AlterColumnTypeEvent( + tableId, Collections.singletonMap("DESCRIPTION", DataTypes.VARCHAR(255)))); + expected.add( + new RenameColumnEvent(tableId, Collections.singletonMap("DESCRIPTION", "DESCPT"))); + + statement.execute( + String.format( + "ALTER TABLE %s.products MODIFY DESCPT VARCHAR(400) DEFAULT NULL", + "debezium")); + statement.execute( + String.format("ALTER TABLE %s.products RENAME COLUMN DESCPT TO DESC2", "debezium")); + expected.add( + new AlterColumnTypeEvent( + tableId, Collections.singletonMap("DESCPT", DataTypes.VARCHAR(400)))); + expected.add(new RenameColumnEvent(tableId, Collections.singletonMap("DESCPT", "DESC2"))); + + statement.execute( + String.format( + "ALTER TABLE %s.products ADD DESC1 VARCHAR(45) DEFAULT NULL", "debezium")); + expected.add( + new AddColumnEvent( + tableId, + Arrays.asList( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn("DESC1", DataTypes.VARCHAR(45)))))); + + statement.execute( + String.format( + "ALTER TABLE %s.products ADD COL1 VARCHAR(45) DEFAULT NULL", "debezium")); + statement.execute( + String.format( + "ALTER TABLE %s.products ADD COL2 VARCHAR(55) DEFAULT NULL", "debezium")); + expected.add( + new AddColumnEvent( + tableId, + Arrays.asList( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn("COL1", DataTypes.VARCHAR(45)))))); + expected.add( + new AddColumnEvent( + tableId, + Arrays.asList( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn("COL2", DataTypes.VARCHAR(55)))))); + + statement.execute(String.format("ALTER TABLE %s.products DROP COLUMN DESC1", "debezium")); + statement.execute( + String.format( + "ALTER TABLE %s.products MODIFY DESC2 VARCHAR(512) DEFAULT NULL", + "debezium")); + expected.add(new DropColumnEvent(tableId, Collections.singletonList("DESC1"))); + expected.add( + new AlterColumnTypeEvent( + tableId, Collections.singletonMap("DESC2", DataTypes.VARCHAR(512)))); + + statement.execute( + String.format( + "ALTER TABLE %s.products RENAME COLUMN DESC2 TO DESCRIPTION", "debezium")); + expected.add( + new RenameColumnEvent(tableId, Collections.singletonMap("DESC2", "DESCRIPTION"))); + + statement.execute(String.format("ALTER TABLE %s.products DROP COLUMN COL1", "debezium")); + expected.add(new DropColumnEvent(tableId, Collections.singletonList("COL1"))); + statement.execute(String.format("ALTER TABLE %s.products DROP COLUMN COL2", "debezium")); + expected.add(new DropColumnEvent(tableId, Collections.singletonList("COL2"))); + + // Should not catch SchemaChangeEvent of tables other than products + statement.execute( + String.format( + "ALTER TABLE %s.category ADD DESC1 VARCHAR(45) DEFAULT NULL", "debezium")); + statement.execute(String.format("ALTER TABLE %s.category DROP COLUMN DESC1", "debezium")); + return expected; + } + + public static List fetchResults(Iterator iter, int size) { + List result = new ArrayList<>(size); + while (size > 0 && iter.hasNext()) { + LOG.info("ljcaaaa---" + size); + T event = iter.next(); + result.add(event); + size--; + } + return result; + } + + class MockContext implements Factory.Context { + + Configuration factoryConfiguration; + + public MockContext(Configuration factoryConfiguration) { + this.factoryConfiguration = factoryConfiguration; + } + + @Override + public Configuration getFactoryConfiguration() { + return factoryConfiguration; + } + + @Override + public Configuration getPipelineConfiguration() { + return null; + } + + @Override + public ClassLoader getClassLoader() { + return this.getClassLoader(); + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceITCase.java new file mode 100644 index 00000000000..9150c806730 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceITCase.java @@ -0,0 +1,741 @@ +/* + * 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.oracle.source; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.cdc.connectors.base.config.JdbcSourceConfig; +import org.apache.flink.cdc.connectors.base.options.StartupOptions; +import org.apache.flink.cdc.connectors.base.source.utils.hooks.SnapshotPhaseHook; +import org.apache.flink.cdc.connectors.base.source.utils.hooks.SnapshotPhaseHooks; +import org.apache.flink.cdc.connectors.oracle.source.utils.OracleConnectionUtils; +import org.apache.flink.cdc.connectors.oracle.testutils.OracleTestUtils; +import org.apache.flink.cdc.connectors.oracle.testutils.TestTable; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; + +import io.debezium.connector.oracle.OracleConnection; +import io.debezium.jdbc.JdbcConfiguration; +import org.apache.commons.lang3.StringUtils; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static java.lang.String.format; +import static org.apache.flink.cdc.connectors.oracle.testutils.OracleTestUtils.triggerFailover; +import static org.apache.flink.table.api.DataTypes.BIGINT; +import static org.apache.flink.table.api.DataTypes.STRING; +import static org.apache.flink.table.catalog.Column.physical; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * IT tests for {@link + * org.apache.flink.cdc.connectors.oracle.source.OracleSourceBuilder.OracleIncrementalSource}. + */ +@Timeout(value = 300, unit = TimeUnit.SECONDS) +public class OracleSourceITCase extends OracleSourceTestBase { + private static final int USE_POST_LOWWATERMARK_HOOK = 1; + private static final int USE_PRE_HIGHWATERMARK_HOOK = 2; + private static final int USE_POST_HIGHWATERMARK_HOOK = 3; + + private static final Logger LOG = LoggerFactory.getLogger(OracleSourceITCase.class); + + @Test + void testReadSingleTableWithSingleParallelism() throws Exception { + testOracleParallelSource( + 1, + OracleTestUtils.FailoverType.NONE, + OracleTestUtils.FailoverPhase.NEVER, + new String[] {"CUSTOMERS"}); + } + + @Test + void testReadSingleTableWithMultipleParallelism() throws Exception { + testOracleParallelSource( + 4, + OracleTestUtils.FailoverType.NONE, + OracleTestUtils.FailoverPhase.NEVER, + new String[] {"CUSTOMERS"}); + } + + // Failover tests + @Test + void testTaskManagerFailoverInSnapshotPhase() throws Exception { + testOracleParallelSource( + OracleTestUtils.FailoverType.TM, + OracleTestUtils.FailoverPhase.SNAPSHOT, + new String[] {"CUSTOMERS"}); + } + + @Test + void testTaskManagerFailoverInRedoLogPhase() throws Exception { + testOracleParallelSource( + OracleTestUtils.FailoverType.TM, + OracleTestUtils.FailoverPhase.REDO_LOG, + new String[] {"CUSTOMERS"}); + } + + @Test + void testJobManagerFailoverInSnapshotPhase() throws Exception { + testOracleParallelSource( + OracleTestUtils.FailoverType.JM, + OracleTestUtils.FailoverPhase.SNAPSHOT, + new String[] {"CUSTOMERS"}); + } + + @Test + void testJobManagerFailoverInRedoLogPhase() throws Exception { + testOracleParallelSource( + OracleTestUtils.FailoverType.JM, + OracleTestUtils.FailoverPhase.REDO_LOG, + new String[] {"CUSTOMERS"}); + } + + @Test + void testTaskManagerFailoverSingleParallelism() throws Exception { + testOracleParallelSource( + 1, + OracleTestUtils.FailoverType.TM, + OracleTestUtils.FailoverPhase.SNAPSHOT, + new String[] {"CUSTOMERS"}); + } + + @Test + void testJobManagerFailoverSingleParallelism() throws Exception { + testOracleParallelSource( + 1, + OracleTestUtils.FailoverType.JM, + OracleTestUtils.FailoverPhase.SNAPSHOT, + new String[] {"CUSTOMERS"}); + } + + @Test + void testReadSingleTableWithSingleParallelismAndSkipBackfill() throws Exception { + testOracleParallelSource( + DEFAULT_PARALLELISM, + OracleTestUtils.FailoverType.TM, + OracleTestUtils.FailoverPhase.SNAPSHOT, + new String[] {"CUSTOMERS"}, + true, + RestartStrategies.fixedDelayRestart(1, 0), + null); + } + + @Test + void testSnapshotOnlyModeWithDMLPostHighWaterMark() throws Exception { + // The data num is 21, set fetchSize = 22 to test the job is bounded. + List records = + testBackfillWhenWritingEvents( + false, 22, USE_POST_HIGHWATERMARK_HOOK, StartupOptions.snapshot()); + List expectedRecords = + Arrays.asList( + "+I[101, user_1, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "+I[103, user_3, Shanghai, 123567891234]", + "+I[109, user_4, Shanghai, 123567891234]", + "+I[110, user_5, Shanghai, 123567891234]", + "+I[111, user_6, Shanghai, 123567891234]", + "+I[118, user_7, Shanghai, 123567891234]", + "+I[121, user_8, Shanghai, 123567891234]", + "+I[123, user_9, Shanghai, 123567891234]", + "+I[1009, user_10, Shanghai, 123567891234]", + "+I[1010, user_11, Shanghai, 123567891234]", + "+I[1011, user_12, Shanghai, 123567891234]", + "+I[1012, user_13, Shanghai, 123567891234]", + "+I[1013, user_14, Shanghai, 123567891234]", + "+I[1014, user_15, Shanghai, 123567891234]", + "+I[1015, user_16, Shanghai, 123567891234]", + "+I[1016, user_17, Shanghai, 123567891234]", + "+I[1017, user_18, Shanghai, 123567891234]", + "+I[1018, user_19, Shanghai, 123567891234]", + "+I[1019, user_20, Shanghai, 123567891234]", + "+I[2000, user_21, Shanghai, 123567891234]"); + assertEqualsInAnyOrder(expectedRecords, records); + } + + @Test + void testSnapshotOnlyModeWithDMLPreHighWaterMark() throws Exception { + // The data num is 21, set fetchSize = 22 to test the job is bounded. + List records = + testBackfillWhenWritingEvents( + false, 22, USE_PRE_HIGHWATERMARK_HOOK, StartupOptions.snapshot()); + List expectedRecords = + Arrays.asList( + "+I[101, user_1, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "+I[103, user_3, Shanghai, 123567891234]", + "+I[109, user_4, Shanghai, 123567891234]", + "+I[110, user_5, Shanghai, 123567891234]", + "+I[111, user_6, Shanghai, 123567891234]", + "+I[118, user_7, Shanghai, 123567891234]", + "+I[121, user_8, Shanghai, 123567891234]", + "+I[123, user_9, Shanghai, 123567891234]", + "+I[1009, user_10, Shanghai, 123567891234]", + "+I[1010, user_11, Shanghai, 123567891234]", + "+I[1011, user_12, Shanghai, 123567891234]", + "+I[1012, user_13, Shanghai, 123567891234]", + "+I[1013, user_14, Shanghai, 123567891234]", + "+I[1014, user_15, Shanghai, 123567891234]", + "+I[1015, user_16, Shanghai, 123567891234]", + "+I[1016, user_17, Shanghai, 123567891234]", + "+I[1017, user_18, Shanghai, 123567891234]", + "+I[1018, user_19, Shanghai, 123567891234]", + "+I[2000, user_21, Pittsburgh, 123567891234]", + "+I[15213, user_15213, Shanghai, 123567891234]"); + // when enable backfill, the wal log between (snapshot, high_watermark) will be + // applied as snapshot image + assertEqualsInAnyOrder(expectedRecords, records); + } + + @Test + void testEnableBackfillWithDMLPreHighWaterMark() throws Exception { + + List records = + testBackfillWhenWritingEvents( + false, 21, USE_PRE_HIGHWATERMARK_HOOK, StartupOptions.initial()); + + List expectedRecords = + Arrays.asList( + "+I[101, user_1, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "+I[103, user_3, Shanghai, 123567891234]", + "+I[109, user_4, Shanghai, 123567891234]", + "+I[110, user_5, Shanghai, 123567891234]", + "+I[111, user_6, Shanghai, 123567891234]", + "+I[118, user_7, Shanghai, 123567891234]", + "+I[121, user_8, Shanghai, 123567891234]", + "+I[123, user_9, Shanghai, 123567891234]", + "+I[1009, user_10, Shanghai, 123567891234]", + "+I[1010, user_11, Shanghai, 123567891234]", + "+I[1011, user_12, Shanghai, 123567891234]", + "+I[1012, user_13, Shanghai, 123567891234]", + "+I[1013, user_14, Shanghai, 123567891234]", + "+I[1014, user_15, Shanghai, 123567891234]", + "+I[1015, user_16, Shanghai, 123567891234]", + "+I[1016, user_17, Shanghai, 123567891234]", + "+I[1017, user_18, Shanghai, 123567891234]", + "+I[1018, user_19, Shanghai, 123567891234]", + "+I[2000, user_21, Pittsburgh, 123567891234]", + "+I[15213, user_15213, Shanghai, 123567891234]"); + // when enable backfill, the wal log between (snapshot, high_watermark) will be + // applied as snapshot image + assertEqualsInAnyOrder(expectedRecords, records); + } + + @Test + void testEnableBackfillWithDMLPostLowWaterMark() throws Exception { + List records = + testBackfillWhenWritingEvents( + false, 21, USE_POST_LOWWATERMARK_HOOK, StartupOptions.initial()); + + List expectedRecords = + Arrays.asList( + "+I[101, user_1, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "+I[103, user_3, Shanghai, 123567891234]", + "+I[109, user_4, Shanghai, 123567891234]", + "+I[110, user_5, Shanghai, 123567891234]", + "+I[111, user_6, Shanghai, 123567891234]", + "+I[118, user_7, Shanghai, 123567891234]", + "+I[121, user_8, Shanghai, 123567891234]", + "+I[123, user_9, Shanghai, 123567891234]", + "+I[1009, user_10, Shanghai, 123567891234]", + "+I[1010, user_11, Shanghai, 123567891234]", + "+I[1011, user_12, Shanghai, 123567891234]", + "+I[1012, user_13, Shanghai, 123567891234]", + "+I[1013, user_14, Shanghai, 123567891234]", + "+I[1014, user_15, Shanghai, 123567891234]", + "+I[1015, user_16, Shanghai, 123567891234]", + "+I[1016, user_17, Shanghai, 123567891234]", + "+I[1017, user_18, Shanghai, 123567891234]", + "+I[1018, user_19, Shanghai, 123567891234]", + "+I[2000, user_21, Pittsburgh, 123567891234]", + "+I[15213, user_15213, Shanghai, 123567891234]"); + // when enable backfill, the wal log between [low_watermark, snapshot) will be applied + // as snapshot image + assertEqualsInAnyOrder(expectedRecords, records); + } + + @Test + void testEnableBackfillWithDMLPostHighWaterMark() throws Exception { + List records = + testBackfillWhenWritingEvents( + false, 21, USE_POST_HIGHWATERMARK_HOOK, StartupOptions.initial()); + + List expectedRecords = + Arrays.asList( + "+I[101, user_1, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "+I[103, user_3, Shanghai, 123567891234]", + "+I[109, user_4, Shanghai, 123567891234]", + "+I[110, user_5, Shanghai, 123567891234]", + "+I[111, user_6, Shanghai, 123567891234]", + "+I[118, user_7, Shanghai, 123567891234]", + "+I[121, user_8, Shanghai, 123567891234]", + "+I[123, user_9, Shanghai, 123567891234]", + "+I[1009, user_10, Shanghai, 123567891234]", + "+I[1010, user_11, Shanghai, 123567891234]", + "+I[1011, user_12, Shanghai, 123567891234]", + "+I[1012, user_13, Shanghai, 123567891234]", + "+I[1013, user_14, Shanghai, 123567891234]", + "+I[1014, user_15, Shanghai, 123567891234]", + "+I[1015, user_16, Shanghai, 123567891234]", + "+I[1016, user_17, Shanghai, 123567891234]", + "+I[1017, user_18, Shanghai, 123567891234]", + "+I[1018, user_19, Shanghai, 123567891234]", + "+I[1019, user_20, Shanghai, 123567891234]", + "+I[2000, user_21, Shanghai, 123567891234]"); + // when enable backfill, the wal log between [low_watermark, snapshot) will be applied + // as snapshot image + assertEqualsInAnyOrder(expectedRecords, records); + } + + @Test + void testSkipBackfillWithDMLPreHighWaterMark() throws Exception { + List records = + testBackfillWhenWritingEvents( + true, 25, USE_PRE_HIGHWATERMARK_HOOK, StartupOptions.initial()); + + List expectedRecords = + Arrays.asList( + "+I[101, user_1, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "+I[103, user_3, Shanghai, 123567891234]", + "+I[109, user_4, Shanghai, 123567891234]", + "+I[110, user_5, Shanghai, 123567891234]", + "+I[111, user_6, Shanghai, 123567891234]", + "+I[118, user_7, Shanghai, 123567891234]", + "+I[121, user_8, Shanghai, 123567891234]", + "+I[123, user_9, Shanghai, 123567891234]", + "+I[1009, user_10, Shanghai, 123567891234]", + "+I[1010, user_11, Shanghai, 123567891234]", + "+I[1011, user_12, Shanghai, 123567891234]", + "+I[1012, user_13, Shanghai, 123567891234]", + "+I[1013, user_14, Shanghai, 123567891234]", + "+I[1014, user_15, Shanghai, 123567891234]", + "+I[1015, user_16, Shanghai, 123567891234]", + "+I[1016, user_17, Shanghai, 123567891234]", + "+I[1017, user_18, Shanghai, 123567891234]", + "+I[1018, user_19, Shanghai, 123567891234]", + "+I[1019, user_20, Shanghai, 123567891234]", + "+I[2000, user_21, Shanghai, 123567891234]", + "+I[15213, user_15213, Shanghai, 123567891234]", + "-U[2000, user_21, Shanghai, 123567891234]", + "+U[2000, user_21, Pittsburgh, 123567891234]", + "-D[1019, user_20, Shanghai, 123567891234]"); + // when skip backfill, the wal log between (snapshot, high_watermark) will be seen as + // stream event. + assertEqualsInAnyOrder(expectedRecords, records); + } + + @Test + void testSkipBackfillWithDMLPostLowWaterMark() throws Exception { + + List records = + testBackfillWhenWritingEvents( + true, 25, USE_POST_LOWWATERMARK_HOOK, StartupOptions.initial()); + + List expectedRecords = + Arrays.asList( + "+I[101, user_1, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "+I[103, user_3, Shanghai, 123567891234]", + "+I[109, user_4, Shanghai, 123567891234]", + "+I[110, user_5, Shanghai, 123567891234]", + "+I[111, user_6, Shanghai, 123567891234]", + "+I[118, user_7, Shanghai, 123567891234]", + "+I[121, user_8, Shanghai, 123567891234]", + "+I[123, user_9, Shanghai, 123567891234]", + "+I[1009, user_10, Shanghai, 123567891234]", + "+I[1010, user_11, Shanghai, 123567891234]", + "+I[1011, user_12, Shanghai, 123567891234]", + "+I[1012, user_13, Shanghai, 123567891234]", + "+I[1013, user_14, Shanghai, 123567891234]", + "+I[1014, user_15, Shanghai, 123567891234]", + "+I[1015, user_16, Shanghai, 123567891234]", + "+I[1016, user_17, Shanghai, 123567891234]", + "+I[1017, user_18, Shanghai, 123567891234]", + "+I[1018, user_19, Shanghai, 123567891234]", + "+I[2000, user_21, Pittsburgh, 123567891234]", + "+I[15213, user_15213, Shanghai, 123567891234]", + "+I[15213, user_15213, Shanghai, 123567891234]", + "-U[2000, user_21, Shanghai, 123567891234]", + "+U[2000, user_21, Pittsburgh, 123567891234]", + "-D[1019, user_20, Shanghai, 123567891234]"); + // when skip backfill, the wal log between (snapshot, high_watermark) will still be + // seen as stream event. This will occur data duplicate. For example, user_20 will be + // deleted twice, and user_15213 will be inserted twice. + assertEqualsInAnyOrder(expectedRecords, records); + } + + @Test + void testTableWithChunkColumnOfNoPrimaryKey() { + String chunkColumn = "NAME"; + try { + testOracleParallelSource( + 1, + OracleTestUtils.FailoverType.NONE, + OracleTestUtils.FailoverPhase.NEVER, + new String[] {"CUSTOMERS"}, + false, + RestartStrategies.noRestart(), + chunkColumn); + } catch (Exception e) { + Assertions.assertThat(e) + .hasStackTraceContaining( + String.format( + "Chunk key column '%s' doesn't exist in the primary key [%s] of the table %s.", + chunkColumn, "id", "customer.dbo.customers")); + } + } + + private List testBackfillWhenWritingEvents( + boolean skipSnapshotBackfill, + int fetchSize, + int hookType, + StartupOptions startupOptions) + throws Exception { + createAndInitialize("customer.sql"); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.enableCheckpointing(200L); + env.setParallelism(1); + + ResolvedSchema customersSchema = + new ResolvedSchema( + Arrays.asList( + physical("ID", BIGINT().notNull()), + physical("NAME", STRING()), + physical("ADDRESS", STRING()), + physical("PHONE_NUMBER", STRING())), + new ArrayList<>(), + UniqueConstraint.primaryKey("pk", Collections.singletonList("ID"))); + TestTable customerTable = + new TestTable(ORACLE_DATABASE, ORACLE_SCHEMA, "CUSTOMERS", customersSchema); + String tableId = customerTable.getTableId(); + + OracleSourceBuilder.OracleIncrementalSource source = + OracleSourceBuilder.OracleIncrementalSource.builder() + .hostname(ORACLE_CONTAINER.getHost()) + .port(ORACLE_CONTAINER.getOraclePort()) + // To analyze table for approximate rowCnt computation, use admin user + // before chunk splitting. + .username(TOP_USER) + .password(TOP_SECRET) + .databaseList(ORACLE_DATABASE) + .schemaList(ORACLE_SCHEMA) + .tableList("DEBEZIUM.CUSTOMERS") + .skipSnapshotBackfill(skipSnapshotBackfill) + .startupOptions(startupOptions) + .deserializer(customerTable.getDeserializer()) + .build(); + + // Do some database operations during hook in snapshot period. + SnapshotPhaseHooks hooks = new SnapshotPhaseHooks(); + String[] statements = + new String[] { + String.format( + "INSERT INTO %s VALUES (15213, 'user_15213', 'Shanghai', '123567891234')", + tableId), + String.format("UPDATE %s SET address='Pittsburgh' WHERE id=2000", tableId), + String.format("DELETE FROM %s WHERE id=1019", tableId) + }; + SnapshotPhaseHook snapshotPhaseHook = + (sourceConfig, split) -> { + // database update operations use TEST_USER rather than CONNECTOR_USER + JdbcConfiguration configuration = + JdbcConfiguration.copy( + ((JdbcSourceConfig) sourceConfig) + .getDbzConnectorConfig() + .getJdbcConfig()) + .withUser("debezium") + .withPassword("dbz") + .build(); + try (OracleConnection oracleConnection = + OracleConnectionUtils.createOracleConnection(configuration)) { + oracleConnection.execute(statements); + } + }; + + switch (hookType) { + case USE_POST_LOWWATERMARK_HOOK: + hooks.setPostLowWatermarkAction(snapshotPhaseHook); + break; + case USE_PRE_HIGHWATERMARK_HOOK: + hooks.setPreHighWatermarkAction(snapshotPhaseHook); + break; + case USE_POST_HIGHWATERMARK_HOOK: + hooks.setPostHighWatermarkAction(snapshotPhaseHook); + break; + } + source.setSnapshotHooks(hooks); + + List records = new ArrayList<>(); + try (CloseableIterator iterator = + env.fromSource(source, WatermarkStrategy.noWatermarks(), "Backfill Skipped Source") + .executeAndCollect()) { + records = fetchRowData(iterator, fetchSize, customerTable::stringify); + env.close(); + } + return records; + } + + private void testOracleParallelSource( + OracleTestUtils.FailoverType failoverType, + OracleTestUtils.FailoverPhase failoverPhase, + String[] captureCustomerTables) + throws Exception { + testOracleParallelSource( + DEFAULT_PARALLELISM, failoverType, failoverPhase, captureCustomerTables); + } + + private void testOracleParallelSource( + int parallelism, + OracleTestUtils.FailoverType failoverType, + OracleTestUtils.FailoverPhase failoverPhase, + String[] captureCustomerTables) + throws Exception { + testOracleParallelSource( + parallelism, + failoverType, + failoverPhase, + captureCustomerTables, + false, + RestartStrategies.fixedDelayRestart(1, 0), + null); + } + + private void testOracleParallelSource( + int parallelism, + OracleTestUtils.FailoverType failoverType, + OracleTestUtils.FailoverPhase failoverPhase, + String[] captureCustomerTables, + boolean skipSnapshotBackfill, + RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration, + String chunkColumn) + throws Exception { + createAndInitialize("customer.sql"); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); + + env.setParallelism(parallelism); + env.enableCheckpointing(200L); + env.setRestartStrategy(restartStrategyConfiguration); + + String sourceDDL = + format( + "CREATE TABLE products (" + + " ID INT NOT NULL," + + " NAME STRING," + + " ADDRESS STRING," + + " PHONE_NUMBER STRING," + + " primary key (ID) not enforced" + + ") WITH (" + + " 'connector' = 'oracle-cdc'," + + " 'hostname' = '%s'," + + " 'port' = '%s'," + + " 'username' = '%s'," + + " 'password' = '%s'," + + " 'database-name' = '%s'," + + " 'schema-name' = '%s'," + + " 'table-name' = '%s'," + + " 'scan.incremental.snapshot.enabled' = 'false'," + + " 'debezium.log.mining.strategy' = 'online_catalog'," + + " 'debezium.database.history.store.only.captured.tables.ddl' = 'true'," + + " 'scan.incremental.snapshot.backfill.skip' = '%s'" + + "%s" + + ")", + ORACLE_CONTAINER.getHost(), + ORACLE_CONTAINER.getOraclePort(), + // To analyze table for approximate rowCnt computation, use admin user + // before chunk splitting. + TOP_USER, + TOP_SECRET, + ORACLE_DATABASE, + ORACLE_SCHEMA, + getTableNameRegex(captureCustomerTables), // (customer|customer_1) + skipSnapshotBackfill, + chunkColumn == null + ? "" + : ",'scan.incremental.snapshot.chunk.key-column'='" + + chunkColumn + + "'"); + + // first step: check the snapshot data + String[] snapshotForSingleTable = + new String[] { + "+I[101, user_1, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "+I[103, user_3, Shanghai, 123567891234]", + "+I[109, user_4, Shanghai, 123567891234]", + "+I[110, user_5, Shanghai, 123567891234]", + "+I[111, user_6, Shanghai, 123567891234]", + "+I[118, user_7, Shanghai, 123567891234]", + "+I[121, user_8, Shanghai, 123567891234]", + "+I[123, user_9, Shanghai, 123567891234]", + "+I[1009, user_10, Shanghai, 123567891234]", + "+I[1010, user_11, Shanghai, 123567891234]", + "+I[1011, user_12, Shanghai, 123567891234]", + "+I[1012, user_13, Shanghai, 123567891234]", + "+I[1013, user_14, Shanghai, 123567891234]", + "+I[1014, user_15, Shanghai, 123567891234]", + "+I[1015, user_16, Shanghai, 123567891234]", + "+I[1016, user_17, Shanghai, 123567891234]", + "+I[1017, user_18, Shanghai, 123567891234]", + "+I[1018, user_19, Shanghai, 123567891234]", + "+I[1019, user_20, Shanghai, 123567891234]", + "+I[2000, user_21, Shanghai, 123567891234]" + }; + tEnv.executeSql(sourceDDL); + TableResult tableResult = tEnv.executeSql("select * from products"); + CloseableIterator iterator = tableResult.collect(); + JobID jobId = tableResult.getJobClient().get().getJobID(); + List expectedSnapshotData = new ArrayList<>(); + for (int i = 0; i < captureCustomerTables.length; i++) { + expectedSnapshotData.addAll(Arrays.asList(snapshotForSingleTable)); + } + + // trigger failover after some snapshot splits read finished + if (failoverPhase == OracleTestUtils.FailoverPhase.SNAPSHOT && iterator.hasNext()) { + triggerFailover( + failoverType, + jobId, + miniClusterResource.get().getMiniCluster(), + () -> sleepMs(100)); + } + + LOG.info("snapshot data start"); + assertEqualsInAnyOrder( + expectedSnapshotData, fetchRows(iterator, expectedSnapshotData.size())); + + // second step: check the redo log data + for (String tableId : captureCustomerTables) { + makeFirstPartRedoLogEvents(ORACLE_SCHEMA + '.' + tableId); + } + if (failoverPhase == OracleTestUtils.FailoverPhase.REDO_LOG) { + triggerFailover( + failoverType, + jobId, + miniClusterResource.get().getMiniCluster(), + () -> sleepMs(200)); + } + for (String tableId : captureCustomerTables) { + makeSecondPartRedoLogEvents(ORACLE_SCHEMA + '.' + tableId); + } + + String[] redoLogForSingleTable = + new String[] { + "-U[103, user_3, Shanghai, 123567891234]", + "+U[103, user_3, Hangzhou, 123567891234]", + "-D[102, user_2, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "-U[103, user_3, Hangzhou, 123567891234]", + "+U[103, user_3, Shanghai, 123567891234]", + "-U[1010, user_11, Shanghai, 123567891234]", + "+U[1010, user_11, Hangzhou, 123567891234]", + "+I[2001, user_22, Shanghai, 123567891234]", + "+I[2002, user_23, Shanghai, 123567891234]", + "+I[2003, user_24, Shanghai, 123567891234]" + }; + List expectedRedoLogData = new ArrayList<>(); + for (int i = 0; i < captureCustomerTables.length; i++) { + expectedRedoLogData.addAll(Arrays.asList(redoLogForSingleTable)); + } + assertEqualsInAnyOrder( + expectedRedoLogData, fetchRows(iterator, expectedRedoLogData.size())); + tableResult.getJobClient().get().cancel().get(); + } + + private void makeFirstPartRedoLogEvents(String tableId) throws Exception { + executeSql("UPDATE " + tableId + " SET address = 'Hangzhou' where id = 103"); + executeSql("DELETE FROM " + tableId + " where id = 102"); + executeSql("INSERT INTO " + tableId + " VALUES(102, 'user_2','Shanghai','123567891234')"); + executeSql("UPDATE " + tableId + " SET address = 'Shanghai' where id = 103"); + } + + private void makeSecondPartRedoLogEvents(String tableId) throws Exception { + executeSql("UPDATE " + tableId + " SET address = 'Hangzhou' where id = 1010"); + executeSql("INSERT INTO " + tableId + " VALUES(2001, 'user_22','Shanghai','123567891234')"); + executeSql("INSERT INTO " + tableId + " VALUES(2002, 'user_23','Shanghai','123567891234')"); + executeSql("INSERT INTO " + tableId + " VALUES(2003, 'user_24','Shanghai','123567891234')"); + } + + private void sleepMs(long millis) { + try { + Thread.sleep(millis); + } catch (InterruptedException ignored) { + } + } + + private static List fetchRowData( + Iterator iter, int size, Function stringifier) { + List rows = new ArrayList<>(size); + while (size > 0 && iter.hasNext()) { + RowData row = iter.next(); + rows.add(row); + size--; + } + return rows.stream().map(stringifier).collect(Collectors.toList()); + } + + private static List fetchRows(Iterator iter, int size) { + List rows = new ArrayList<>(size); + while (size > 0 && iter.hasNext()) { + Row row = iter.next(); + rows.add(row.toString()); + size--; + } + return rows; + } + + private String getTableNameRegex(String[] captureCustomerTables) { + checkState(captureCustomerTables.length > 0); + if (captureCustomerTables.length == 1) { + return captureCustomerTables[0]; + } else { + // pattern that matches multiple tables + return format("(%s)", StringUtils.join(captureCustomerTables, "|")); + } + } + + private void executeSql(String sql) throws Exception { + try (Connection connection = getJdbcConnection(); + Statement statement = connection.createStatement()) { + statement.execute(sql); + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceTestBase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceTestBase.java new file mode 100644 index 00000000000..c9ef34ca6a8 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceTestBase.java @@ -0,0 +1,196 @@ +/* + * 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.oracle.source; + +import org.apache.flink.cdc.connectors.utils.ExternalResourceProxy; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.TestLogger; + +import io.debezium.relational.TableId; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.DockerClientFactory; +import org.testcontainers.containers.OracleContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.DockerImageName; + +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.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** Basic class for testing {@link OracleDataSource}. */ +public class OracleSourceTestBase extends TestLogger { + + protected static final Logger LOG = LoggerFactory.getLogger(OracleSourceTestBase.class); + protected static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)--.*$"); + protected static final int DEFAULT_PARALLELISM = 4; + public static final String ORACLE_DATABASE = "ORCLCDB"; + public static final String ORACLE_SCHEMA = "DEBEZIUM"; + public static final String CONNECTOR_USER = "dbzuser"; + public static final String CONNECTOR_PWD = "dbz"; + public static final String TEST_USER = "debezium"; + public static final String TEST_PWD = "dbz"; + public static final String TOP_USER = "sys as sysdba"; + public static final String TOP_SECRET = "top_secret"; + + public static final OracleContainer ORACLE_CONTAINER = + new OracleContainer( + DockerImageName.parse("goodboy008/oracle-19.3.0-ee") + .withTag( + DockerClientFactory.instance() + .client() + .versionCmd() + .exec() + .getArch() + .equals("amd64") + ? "non-cdb" + : "arm-non-cdb")) + .withUsername(CONNECTOR_USER) + .withPassword(CONNECTOR_PWD) + .withDatabaseName(ORACLE_DATABASE) + .withLogConsumer(new Slf4jLogConsumer(LOG)); + + @RegisterExtension + public final ExternalResourceProxy miniClusterResource = + new ExternalResourceProxy<>( + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .setConfiguration(new Configuration()) + .withHaLeadershipControl() + .build())); + + @BeforeAll + public static void startContainers() { + LOG.info("Starting containers..."); + Startables.deepStart(Stream.of(ORACLE_CONTAINER)).join(); + LOG.info("Containers are started."); + } + + @AfterAll + public static void stopContainers() { + LOG.info("Stopping containers..."); + if (ORACLE_CONTAINER != null) { + ORACLE_CONTAINER.stop(); + } + LOG.info("Containers are stopped."); + } + + public static void assertEqualsInAnyOrder(List expected, List actual) { + Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(expected); + } + + public static Connection getJdbcConnection() throws SQLException { + return DriverManager.getConnection(ORACLE_CONTAINER.getJdbcUrl(), TEST_USER, TEST_PWD); + } + + public static Connection getJdbcConnectionAsDBA() throws SQLException { + return DriverManager.getConnection(ORACLE_CONTAINER.getJdbcUrl(), TOP_USER, TOP_SECRET); + } + + public static void createAndInitialize(String sqlFile) throws Exception { + final String ddlFile = String.format("ddl/%s", sqlFile); + final URL ddlTestFile = OracleSourceITCase.class.getClassLoader().getResource(ddlFile); + Assertions.assertThat(ddlTestFile).withFailMessage("Cannot locate " + ddlFile).isNotNull(); + try (Connection connection = getJdbcConnection(); + Statement statement = connection.createStatement()) { + connection.setAutoCommit(true); + // region Drop all user tables in Debezium schema + listTables(connection) + .forEach( + tableId -> { + try { + statement.execute( + "DROP TABLE " + + String.join( + ".", + tableId.schema(), + tableId.table())); + } catch (SQLException e) { + LOG.warn("drop table error, table:{}", tableId, e); + } + }); + // endregion + + final List statements = + Arrays.stream( + Files.readAllLines(Paths.get(ddlTestFile.toURI())).stream() + .map(String::trim) + .filter(x -> !x.startsWith("--") && !x.isEmpty()) + .map( + x -> { + final Matcher m = + COMMENT_PATTERN.matcher(x); + return m.matches() ? m.group(1) : x; + }) + .collect(Collectors.joining("\n")) + .split(";")) + .collect(Collectors.toList()); + + for (String stmt : statements) { + statement.execute(stmt); + } + } + } + + // ------------------ utils ----------------------- + protected static List listTables(Connection connection) { + + Set tableIdSet = new HashSet<>(); + String queryTablesSql = + "SELECT OWNER ,TABLE_NAME,TABLESPACE_NAME FROM ALL_TABLES \n" + + "WHERE TABLESPACE_NAME IS NOT NULL AND TABLESPACE_NAME NOT IN ('SYSTEM','SYSAUX') " + + "AND NESTED = 'NO' AND TABLE_NAME NOT IN (SELECT PARENT_TABLE_NAME FROM ALL_NESTED_TABLES)"; + try { + ResultSet resultSet = connection.createStatement().executeQuery(queryTablesSql); + while (resultSet.next()) { + String schemaName = resultSet.getString(1); + String tableName = resultSet.getString(2); + TableId tableId = new TableId(ORACLE_DATABASE, schemaName, tableName); + tableIdSet.add(tableId); + } + } catch (SQLException e) { + LOG.warn(" SQL execute error, sql:{}", queryTablesSql, e); + } + return new ArrayList<>(tableIdSet); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/testutils/OracleTestUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/testutils/OracleTestUtils.java new file mode 100644 index 00000000000..b43927712d0 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/testutils/OracleTestUtils.java @@ -0,0 +1,124 @@ +/* + * 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.oracle.testutils; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.highavailability.nonha.embedded.HaLeadershipControl; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.table.planner.factories.TestValuesTableFactory; + +import org.apache.commons.lang3.StringUtils; + +import static java.lang.String.format; +import static org.apache.flink.util.Preconditions.checkState; + +/** Oracle test utilities. */ +public class OracleTestUtils { + + /** The type of failover. */ + public enum FailoverType { + TM, + JM, + NONE + } + + /** The phase of failover. */ + public enum FailoverPhase { + SNAPSHOT, + REDO_LOG, + NEVER + } + + public static void triggerFailover( + FailoverType type, JobID jobId, MiniCluster miniCluster, Runnable afterFailAction) + throws Exception { + switch (type) { + case TM: + restartTaskManager(miniCluster, afterFailAction); + break; + case JM: + triggerJobManagerFailover(jobId, miniCluster, afterFailAction); + break; + case NONE: + break; + default: + throw new IllegalStateException("Unexpected value: " + type); + } + } + + public static void triggerJobManagerFailover( + JobID jobId, MiniCluster miniCluster, Runnable afterFailAction) throws Exception { + final HaLeadershipControl haLeadershipControl = miniCluster.getHaLeadershipControl().get(); + haLeadershipControl.revokeJobMasterLeadership(jobId).get(); + afterFailAction.run(); + haLeadershipControl.grantJobMasterLeadership(jobId).get(); + } + + public static void restartTaskManager(MiniCluster miniCluster, Runnable afterFailAction) + throws Exception { + miniCluster.terminateTaskManager(0).get(); + afterFailAction.run(); + miniCluster.startTaskManager(); + } + + 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.getRawResultsAsStrings(sinkName).size(); + } catch (IllegalArgumentException e) { + // job is not started yet + return 0; + } + } + } + + public static void waitForUpsertSinkSize(String sinkName, int expectedSize) + throws InterruptedException { + while (upsertSinkSize(sinkName) < expectedSize) { + Thread.sleep(100); + } + } + + public static int upsertSinkSize(String sinkName) { + synchronized (TestValuesTableFactory.class) { + try { + return TestValuesTableFactory.getResultsAsStrings(sinkName).size(); + } catch (IllegalArgumentException e) { + // job is not started yet + return 0; + } + } + } + + public static String getTableNameRegex(String[] captureCustomerTables) { + checkState(captureCustomerTables.length > 0); + if (captureCustomerTables.length == 1) { + return captureCustomerTables[0]; + } else { + // pattern that matches multiple tables + return format("(%s)", StringUtils.join(captureCustomerTables, "|")); + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/testutils/RecordsFormatter.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/testutils/RecordsFormatter.java new file mode 100644 index 00000000000..b5573ed2978 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/testutils/RecordsFormatter.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.oracle.testutils; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.cdc.connectors.base.utils.SourceRecordUtils; +import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.table.RowDataDebeziumDeserializeSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.conversion.RowRowConverter; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.flink.types.Row; +import org.apache.flink.util.Collector; + +import org.apache.kafka.connect.source.SourceRecord; + +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +/** Formatter that formats the {@link SourceRecord} to String. */ +public class RecordsFormatter { + + private final DataType dataType; + private final ZoneId zoneId; + + private TypeInformation typeInfo; + private DebeziumDeserializationSchema deserializationSchema; + private SimpleCollector collector; + private RowRowConverter rowRowConverter; + + public RecordsFormatter(DataType dataType) { + this(dataType, ZoneId.of("UTC")); + } + + public RecordsFormatter(DataType dataType, ZoneId zoneId) { + this.dataType = dataType; + this.zoneId = zoneId; + this.typeInfo = + (TypeInformation) TypeConversions.fromDataTypeToLegacyInfo(dataType); + this.deserializationSchema = + RowDataDebeziumDeserializeSchema.newBuilder() + .setPhysicalRowType((RowType) dataType.getLogicalType()) + .setResultTypeInfo(typeInfo) + .build(); + this.collector = new SimpleCollector(); + this.rowRowConverter = RowRowConverter.create(dataType); + rowRowConverter.open(Thread.currentThread().getContextClassLoader()); + } + + public List format(List records) { + records.stream() + // Keep DataChangeEvent only + .filter(SourceRecordUtils::isDataChangeRecord) + .forEach( + r -> { + try { + deserializationSchema.deserialize(r, collector); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + return collector.list.stream() + .map(rowRowConverter::toExternal) + .map(Row::toString) + .collect(Collectors.toList()); + } + + private static class SimpleCollector implements Collector { + + private List list = new ArrayList<>(); + + @Override + public void collect(RowData record) { + list.add(record); + } + + @Override + public void close() { + // do nothing + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/testutils/TestTable.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/testutils/TestTable.java new file mode 100644 index 00000000000..dc39c4cfc8b --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/apache/flink/cdc/connectors/oracle/testutils/TestTable.java @@ -0,0 +1,99 @@ +/* + * 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.oracle.testutils; + +import org.apache.flink.cdc.connectors.oracle.table.OracleDeserializationConverterFactory; +import org.apache.flink.cdc.debezium.table.RowDataDebeziumDeserializeSchema; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.conversion.RowRowConverter; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.RowType; + +import org.apache.kafka.connect.source.SourceRecord; + +import java.util.List; + +/** + * Test utility for creating converter, formatter and deserializer of a table in the test database. + */ +public class TestTable { + + private final String databaseName; + private final String tableName; + + private final String schemaName; + + private final ResolvedSchema schema; + + // Lazily initialized components + private RowRowConverter rowRowConverter; + private RowDataDebeziumDeserializeSchema deserializer; + private RecordsFormatter recordsFormatter; + + public TestTable( + String databaseName, String schemaName, String tableName, ResolvedSchema schema) { + this.databaseName = databaseName; + this.schemaName = schemaName; + this.tableName = tableName; + this.schema = schema; + } + + public String getTableId() { + return String.format("%s.%s", schemaName, tableName); + } + + public RowType getRowType() { + return (RowType) schema.toPhysicalRowDataType().getLogicalType(); + } + + public RowDataDebeziumDeserializeSchema getDeserializer() { + if (deserializer == null) { + deserializer = + RowDataDebeziumDeserializeSchema.newBuilder() + .setPhysicalRowType(getRowType()) + .setResultTypeInfo(InternalTypeInfo.of(getRowType())) + .setUserDefinedConverterFactory( + OracleDeserializationConverterFactory.instance()) + .build(); + } + return deserializer; + } + + public RowRowConverter getRowRowConverter() { + if (rowRowConverter == null) { + rowRowConverter = RowRowConverter.create(schema.toPhysicalRowDataType()); + } + return rowRowConverter; + } + + public RecordsFormatter getRecordsFormatter() { + if (recordsFormatter == null) { + recordsFormatter = new RecordsFormatter(schema.toPhysicalRowDataType()); + } + return recordsFormatter; + } + + public String stringify(RowData rowData) { + return getRowRowConverter().toExternal(rowData).toString(); + } + + public List stringify(List sourceRecord) { + return getRecordsFormatter().format(sourceRecord); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/testcontainers/containers/OracleContainer.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/testcontainers/containers/OracleContainer.java new file mode 100644 index 00000000000..8df09b6f7f1 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/java/org/testcontainers/containers/OracleContainer.java @@ -0,0 +1,239 @@ +/* + * 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.testcontainers.containers; + +import org.apache.commons.lang3.StringUtils; +import org.jetbrains.annotations.NotNull; +import org.testcontainers.containers.wait.strategy.LogMessageWaitStrategy; +import org.testcontainers.utility.DockerImageName; + +import java.time.Duration; +import java.time.temporal.ChronoUnit; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Future; + +/** Copy from testcontainers. */ +public class OracleContainer extends JdbcDatabaseContainer { + + public static final String NAME = "oracle"; + + private static final DockerImageName DEFAULT_IMAGE_NAME = + DockerImageName.parse("goodboy008/oracle-19.3.0-ee"); + + static final String DEFAULT_TAG = "latest"; + + static final String IMAGE = DEFAULT_IMAGE_NAME.getUnversionedPart(); + + static final int ORACLE_PORT = 1521; + + private static final int APEX_HTTP_PORT = 8080; + + private static final int DEFAULT_STARTUP_TIMEOUT_SECONDS = 240; + + private static final int DEFAULT_CONNECT_TIMEOUT_SECONDS = 120; + + // Container defaults + static final String DEFAULT_DATABASE_NAME = "xepdb1"; + + static final String DEFAULT_SID = "ORCLCDB"; + + static final String DEFAULT_SYSTEM_USER = "system"; + + static final String DEFAULT_SYS_USER = "sys"; + + // Test container defaults + static final String APP_USER = "test"; + + static final String APP_USER_PASSWORD = "test"; + + // Restricted user and database names + private static final List ORACLE_SYSTEM_USERS = + Arrays.asList(DEFAULT_SYSTEM_USER, DEFAULT_SYS_USER); + + private String databaseName = DEFAULT_DATABASE_NAME; + + private String username = APP_USER; + + private String password = APP_USER_PASSWORD; + + private boolean usingSid = false; + + /** @deprecated use @link OracleContainer(DockerImageName) instead */ + @Deprecated + public OracleContainer() { + this(DEFAULT_IMAGE_NAME.withTag(DEFAULT_TAG)); + } + + public OracleContainer(String dockerImageName) { + this(DockerImageName.parse(dockerImageName)); + } + + public OracleContainer(final DockerImageName dockerImageName) { + super(dockerImageName); + dockerImageName.assertCompatibleWith(DEFAULT_IMAGE_NAME); + preconfigure(); + } + + public OracleContainer(Future dockerImageName) { + super(dockerImageName); + preconfigure(); + } + + private void preconfigure() { + this.waitStrategy = + new LogMessageWaitStrategy() + .withRegEx(".*DATABASE IS READY TO USE!.*\\s") + .withTimes(1) + .withStartupTimeout( + Duration.of(DEFAULT_STARTUP_TIMEOUT_SECONDS, ChronoUnit.SECONDS)); + + withConnectTimeoutSeconds(DEFAULT_CONNECT_TIMEOUT_SECONDS); + addExposedPorts(ORACLE_PORT, APEX_HTTP_PORT); + } + + @Override + protected void waitUntilContainerStarted() { + getWaitStrategy().waitUntilReady(this); + } + + @NotNull + @Override + public Set getLivenessCheckPortNumbers() { + return Collections.singleton(getMappedPort(ORACLE_PORT)); + } + + @Override + public String getDriverClassName() { + return "oracle.jdbc.driver.OracleDriver"; + } + + @Override + public String getJdbcUrl() { + return isUsingSid() + ? "jdbc:oracle:thin:" + "@" + getHost() + ":" + getOraclePort() + ":" + getSid() + : "jdbc:oracle:thin:" + + "@" + + getHost() + + ":" + + getOraclePort() + + "/" + + getDatabaseName(); + } + + @Override + public String getUsername() { + // An application user is tied to the database, and therefore not authenticated to connect + // to SID. + return isUsingSid() ? DEFAULT_SYSTEM_USER : username; + } + + @Override + public String getPassword() { + return password; + } + + @Override + public String getDatabaseName() { + return databaseName; + } + + protected boolean isUsingSid() { + return usingSid; + } + + @Override + public OracleContainer withUsername(String username) { + if (StringUtils.isEmpty(username)) { + throw new IllegalArgumentException("Username cannot be null or empty"); + } + if (ORACLE_SYSTEM_USERS.contains(username.toLowerCase())) { + throw new IllegalArgumentException("Username cannot be one of " + ORACLE_SYSTEM_USERS); + } + this.username = username; + return self(); + } + + @Override + public OracleContainer withPassword(String password) { + if (StringUtils.isEmpty(password)) { + throw new IllegalArgumentException("Password cannot be null or empty"); + } + this.password = password; + return self(); + } + + @Override + public OracleContainer withDatabaseName(String databaseName) { + if (StringUtils.isEmpty(databaseName)) { + throw new IllegalArgumentException("Database name cannot be null or empty"); + } + + if (DEFAULT_DATABASE_NAME.equals(databaseName.toLowerCase())) { + throw new IllegalArgumentException( + "Database name cannot be set to " + DEFAULT_DATABASE_NAME); + } + + this.databaseName = databaseName; + return self(); + } + + public OracleContainer usingSid() { + this.usingSid = true; + return self(); + } + + @Override + public OracleContainer withUrlParam(String paramName, String paramValue) { + throw new UnsupportedOperationException("The Oracle Database driver does not support this"); + } + + @SuppressWarnings("SameReturnValue") + public String getSid() { + return DEFAULT_SID; + } + + public Integer getOraclePort() { + return getMappedPort(ORACLE_PORT); + } + + @SuppressWarnings("unused") + public Integer getWebPort() { + return getMappedPort(APEX_HTTP_PORT); + } + + @Override + public String getTestQueryString() { + return "SELECT 1 FROM DUAL"; + } + + @Override + protected void configure() { + withEnv("ORACLE_PASSWORD", password); + + // Only set ORACLE_DATABASE if different than the default. + if (databaseName != DEFAULT_DATABASE_NAME) { + withEnv("ORACLE_DATABASE", databaseName); + } + + withEnv("APP_USER", username); + withEnv("APP_USER_PASSWORD", password); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/resources/ddl/column_type_test.sql b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/resources/ddl/column_type_test.sql new file mode 100644 index 00000000000..24124e385c5 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/resources/ddl/column_type_test.sql @@ -0,0 +1,83 @@ +-- Copyright 2023 Ververica Inc. +-- +-- Licensed under the Apache License, Version 2.0 (the "License"); +-- you may not use this file except in compliance with the License. +-- You may obtain a copy of the License at +-- http://www.apache.org/licenses/LICENSE-2.0 +-- Unless required by applicable law or agreed to in writing, +-- software distributed under the License is distributed on an +-- "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +-- KIND, either express or implied. See the License for the +-- specific language governing permissions and limitations +-- under the License. + +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: column_type_test +-- ---------------------------------------------------------------------------------------------------------------- +-- Set session timezone to fixed Asia/Shanghai for checking TIMESTAMP_LTZ type +ALTER SESSION SET TIME_ZONE='Asia/Shanghai'; + +create table DEBEZIUM.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_TSTZ TIMESTAMP(6) WITH TIME ZONE, + VAL_TSLTZ TIMESTAMP(6) WITH LOCAL TIME ZONE, + VAL_INT_YTM INTERVAL YEAR(2) TO MONTH, + VAL_INT_DTS INTERVAL DAY(3) TO SECOND(2), + T15VARCHAR SYS.XMLTYPE, + primary key (ID) +); + +ALTER TABLE DEBEZIUM.FULL_TYPES ADD SUPPLEMENTAL LOG DATA (ALL) COLUMNS; + +INSERT INTO DEBEZIUM.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_TIMESTAMP_TZ('2022-10-30 01:34:56.00789 -11:00', 'yyyy-mm-dd HH24:MI:SS.FF5 TZH:TZM'), + TO_TIMESTAMP_TZ('2022-10-30 01:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'), + INTERVAL '-3-6' YEAR TO MONTH, + INTERVAL '-1 2:3:4.56' DAY TO SECOND, + sys.xmlType.createXML('test xmlType') + +); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/resources/ddl/customer.sql b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/resources/ddl/customer.sql new file mode 100644 index 00000000000..d1b0d8b3903 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/resources/ddl/customer.sql @@ -0,0 +1,80 @@ +-- Copyright 2023 Ververica Inc. +-- +-- Licensed 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: customer +-- ---------------------------------------------------------------------------------------------------------------- + +-- Create and populate our users using a single insert with many rows +CREATE TABLE DEBEZIUM.CUSTOMERS ( + ID INT NOT NULL, + NAME VARCHAR2(255) NOT NULL, + ADDRESS VARCHAR2(1024), + PHONE_NUMBER VARCHAR2(512), + PRIMARY KEY(ID) +); + +ALTER TABLE DEBEZIUM.CUSTOMERS ADD SUPPLEMENTAL LOG DATA (ALL) COLUMNS; + +INSERT INTO DEBEZIUM.CUSTOMERS VALUES (101,'user_1','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS VALUES (102,'user_2','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS VALUES (103,'user_3','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS VALUES (109,'user_4','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS VALUES (110,'user_5','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS VALUES (111,'user_6','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS VALUES (118,'user_7','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS VALUES (121,'user_8','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS VALUES (123,'user_9','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS VALUES (1009,'user_10','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS VALUES (1010,'user_11','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS VALUES (1011,'user_12','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS VALUES (1012,'user_13','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS VALUES (1013,'user_14','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS VALUES (1014,'user_15','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS VALUES (1015,'user_16','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS VALUES (1016,'user_17','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS VALUES (1017,'user_18','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS VALUES (1018,'user_19','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS VALUES (1019,'user_20','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS VALUES (2000,'user_21','Shanghai','123567891234'); + +-- table has same name prefix with 'customers.*' +CREATE TABLE DEBEZIUM.CUSTOMERS_1 ( + ID INT NOT NULL, + NAME VARCHAR2(255) NOT NULL, + ADDRESS VARCHAR2(1024), + PHONE_NUMBER VARCHAR2(512), + PRIMARY KEY(ID) +); + +INSERT INTO DEBEZIUM.CUSTOMERS_1 VALUES (101,'user_1','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS_1 VALUES (102,'user_2','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS_1 VALUES (103,'user_3','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS_1 VALUES (109,'user_4','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS_1 VALUES (110,'user_5','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS_1 VALUES (111,'user_6','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS_1 VALUES (118,'user_7','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS_1 VALUES (121,'user_8','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS_1 VALUES (123,'user_9','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS_1 VALUES (1009,'user_10','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS_1 VALUES (1010,'user_11','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS_1 VALUES (1011,'user_12','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS_1 VALUES (1012,'user_13','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS_1 VALUES (1013,'user_14','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS_1 VALUES (1014,'user_15','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS_1 VALUES (1015,'user_16','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS_1 VALUES (1016,'user_17','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS_1 VALUES (1017,'user_18','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS_1 VALUES (1018,'user_19','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS_1 VALUES (1019,'user_20','Shanghai','123567891234'); +INSERT INTO DEBEZIUM.CUSTOMERS_1 VALUES (2000,'user_21','Shanghai','123567891234'); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/resources/ddl/product.sql b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/resources/ddl/product.sql new file mode 100644 index 00000000000..b955c63738b --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/resources/ddl/product.sql @@ -0,0 +1,53 @@ +-- Copyright 2023 Ververica Inc. +-- +-- Licensed 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: customer +-- ---------------------------------------------------------------------------------------------------------------- + +-- Create and populate products and category tables using a single insert with many rows +CREATE TABLE DEBEZIUM.PRODUCTS ( + ID NUMBER(9, 0) NOT NULL, + NAME VARCHAR(255) NOT NULL, + DESCRIPTION VARCHAR(512), + WEIGHT FLOAT, + PRIMARY KEY(ID) +); +CREATE TABLE DEBEZIUM.CATEGORY ( + ID NUMBER(9, 0) NOT NULL, + CATEGORY_NAME VARCHAR(255), + PRIMARY KEY(ID) +); + +ALTER TABLE DEBEZIUM.PRODUCTS ADD SUPPLEMENTAL LOG DATA (ALL) COLUMNS; +ALTER TABLE DEBEZIUM.CATEGORY ADD SUPPLEMENTAL LOG DATA (ALL) COLUMNS; + + +INSERT INTO DEBEZIUM.PRODUCTS (ID,NAME,DESCRIPTION,WEIGHT) + VALUES (101,'scooter','Small 2-wheel scooter',3.14); +INSERT INTO DEBEZIUM.PRODUCTS (ID,NAME,DESCRIPTION,WEIGHT) + VALUES (102,'car battery','12V car battery',8.1); +INSERT INTO DEBEZIUM.PRODUCTS (ID,NAME,DESCRIPTION,WEIGHT) + VALUES (103,'12-pack drill bits','12-pack of drill bits with sizes ranging from #40 to #3',0.8); +INSERT INTO DEBEZIUM.PRODUCTS (ID,NAME,DESCRIPTION,WEIGHT) + VALUES (104,'hammer','12oz carpenters hammer',0.75); +INSERT INTO DEBEZIUM.PRODUCTS (ID,NAME,DESCRIPTION,WEIGHT) + VALUES (105,'hammer','14oz carpenters hammer',0.875); +INSERT INTO DEBEZIUM.PRODUCTS (ID,NAME,DESCRIPTION,WEIGHT) + VALUES (106,'hammer','16oz carpenters hammer',1.0); +INSERT INTO DEBEZIUM.PRODUCTS (ID,NAME,DESCRIPTION,WEIGHT) + VALUES (107,'rocks','box of assorted rocks',5.3); +INSERT INTO DEBEZIUM.PRODUCTS (ID,NAME,DESCRIPTION,WEIGHT) + VALUES (108,'jacket','water resistent black wind breaker',0.1); +INSERT INTO DEBEZIUM.PRODUCTS (ID,NAME,DESCRIPTION,WEIGHT) + VALUES (109,'spare tire','24 inch spare tire',22.2); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/resources/log4j2-test.properties b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/resources/log4j2-test.properties new file mode 100644 index 00000000000..a9d045e0ef2 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/test/resources/log4j2-test.properties @@ -0,0 +1,26 @@ +################################################################################ +# Copyright 2023 Ververica Inc. +# +# Licensed 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 root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +rootLogger.level=INFO +rootLogger.appenderRef.test.ref = TestLogger + +appender.testlogger.name = TestLogger +appender.testlogger.type = CONSOLE +appender.testlogger.target = SYSTEM_ERR +appender.testlogger.layout.type = PatternLayout +appender.testlogger.layout.pattern = %-4r [%t] %-5p %c - %m%n diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/pom.xml index 73869eec91a..d47495ed9aa 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/pom.xml +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/pom.xml @@ -37,6 +37,7 @@ limitations under the License. flink-cdc-pipeline-connector-paimon flink-cdc-pipeline-connector-elasticsearch flink-cdc-pipeline-connector-oceanbase + flink-cdc-pipeline-connector-oracle flink-cdc-pipeline-connector-maxcompute flink-cdc-pipeline-connector-iceberg flink-cdc-pipeline-connector-fluss diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/IncrementalSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/IncrementalSource.java index dfee377562f..4c7288b9fe6 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/IncrementalSource.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/IncrementalSource.java @@ -75,6 +75,7 @@ public class IncrementalSource protected final OffsetFactory offsetFactory; protected final DebeziumDeserializationSchema deserializationSchema; protected final SourceSplitSerializer sourceSplitSerializer; + private RecordEmitter recordEmitter; // Actions to perform during the snapshot phase. // This field is introduced for testing purpose, for example testing if changes made in the @@ -226,11 +227,18 @@ public TypeInformation getProducedType() { protected RecordEmitter createRecordEmitter( SourceConfig sourceConfig, SourceReaderMetrics sourceReaderMetrics) { - return new IncrementalSourceRecordEmitter<>( - deserializationSchema, - sourceReaderMetrics, - sourceConfig.isIncludeSchemaChanges(), - offsetFactory); + if (recordEmitter != null) { + IncrementalSourceRecordEmitter incrementalSourceRecordEmitter = + (IncrementalSourceRecordEmitter) recordEmitter; + incrementalSourceRecordEmitter.setSourceReaderMetrics(sourceReaderMetrics); + return incrementalSourceRecordEmitter; + } else { + return new IncrementalSourceRecordEmitter<>( + deserializationSchema, + sourceReaderMetrics, + sourceConfig.isIncludeSchemaChanges(), + offsetFactory); + } } /** @@ -242,4 +250,8 @@ protected RecordEmitter createRecordEmitter( public void setSnapshotHooks(SnapshotPhaseHooks snapshotHooks) { this.snapshotHooks = snapshotHooks; } + + public void setRecordEmitter(RecordEmitter recordEmitter) { + this.recordEmitter = recordEmitter; + } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/reader/IncrementalSourceRecordEmitter.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/reader/IncrementalSourceRecordEmitter.java index 25ebb8aaa45..c63d624777b 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/reader/IncrementalSourceRecordEmitter.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/reader/IncrementalSourceRecordEmitter.java @@ -35,6 +35,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.Serializable; import java.util.HashMap; import java.util.Iterator; import java.util.Map; @@ -55,17 +56,17 @@ * emit records rather than emit the records directly. */ public class IncrementalSourceRecordEmitter - implements RecordEmitter { + implements RecordEmitter, Serializable { private static final Logger LOG = LoggerFactory.getLogger(IncrementalSourceRecordEmitter.class); private static final FlinkJsonTableChangeSerializer TABLE_CHANGE_SERIALIZER = new FlinkJsonTableChangeSerializer(); - protected final DebeziumDeserializationSchema debeziumDeserializationSchema; - protected final SourceReaderMetrics sourceReaderMetrics; - protected final boolean includeSchemaChanges; - protected final OutputCollector outputCollector; - protected final OffsetFactory offsetFactory; + protected DebeziumDeserializationSchema debeziumDeserializationSchema; + protected SourceReaderMetrics sourceReaderMetrics; + protected boolean includeSchemaChanges; + protected OutputCollector outputCollector; + protected OffsetFactory offsetFactory; public IncrementalSourceRecordEmitter( DebeziumDeserializationSchema debeziumDeserializationSchema, @@ -79,6 +80,10 @@ public IncrementalSourceRecordEmitter( this.offsetFactory = offsetFactory; } + // public IncrementalSourceRecordEmitter() { + // + // } + @Override public void emitRecord( SourceRecords sourceRecords, SourceOutput output, SourceSplitState splitState) @@ -151,6 +156,10 @@ public Offset getOffsetPosition(Map offset) { return offsetFactory.newOffset(offsetStrMap); } + public void setSourceReaderMetrics(SourceReaderMetrics sourceReaderMetrics) { + this.sourceReaderMetrics = sourceReaderMetrics; + } + protected void emitElement(SourceRecord element, SourceOutput output) throws Exception { sourceReaderMetrics.markRecord(); sourceReaderMetrics.updateRecordCounters(element); @@ -173,7 +182,7 @@ protected void reportMetrics(SourceRecord element) { } /** An adapter between {@link SourceOutput} and {@link Collector}. */ - protected static class OutputCollector implements Collector { + protected static class OutputCollector implements Collector, Serializable { public SourceOutput output; public Long currentMessageTimestamp; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/org/apache/flink/cdc/debezium/event/DebeziumEventDeserializationSchema.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/org/apache/flink/cdc/debezium/event/DebeziumEventDeserializationSchema.java index ad7ec458e53..c6582e26d0b 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/org/apache/flink/cdc/debezium/event/DebeziumEventDeserializationSchema.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/org/apache/flink/cdc/debezium/event/DebeziumEventDeserializationSchema.java @@ -47,6 +47,7 @@ import io.debezium.time.NanoTime; import io.debezium.time.NanoTimestamp; import io.debezium.time.Timestamp; +import io.debezium.time.ZonedTimestamp; import org.apache.kafka.connect.data.Decimal; import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.Schema; @@ -328,7 +329,7 @@ protected Object convertToLocalTimeZoneTimestamp(Object dbzObj, Schema schema) { if (dbzObj instanceof String) { String str = (String) dbzObj; // TIMESTAMP_LTZ type is encoded in string type - Instant instant = Instant.parse(str); + Instant instant = ZonedTimestamp.FORMATTER.parse(str, Instant::from); return LocalZonedTimestampData.fromInstant(instant); } throw new IllegalArgumentException( diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/org/apache/flink/cdc/debezium/event/DebeziumSchemaDataTypeInference.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/org/apache/flink/cdc/debezium/event/DebeziumSchemaDataTypeInference.java index 3afa7128d13..8429c260b11 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/org/apache/flink/cdc/debezium/event/DebeziumSchemaDataTypeInference.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/org/apache/flink/cdc/debezium/event/DebeziumSchemaDataTypeInference.java @@ -180,6 +180,9 @@ protected DataType inferBytes(Object value, Schema schema) { if (precision > DecimalType.MAX_PRECISION) { return DataTypes.STRING(); } + if (scale < 0 || scale > 36) { + return DataTypes.STRING(); + } return DataTypes.DECIMAL(precision, scale); } return DataTypes.BYTES(); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceBuilder.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceBuilder.java index 97bc18c7dda..23bae338402 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceBuilder.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceBuilder.java @@ -20,9 +20,12 @@ import org.apache.flink.cdc.common.annotation.Internal; import org.apache.flink.cdc.connectors.base.options.StartupOptions; import org.apache.flink.cdc.connectors.base.source.jdbc.JdbcIncrementalSource; +import org.apache.flink.cdc.connectors.base.source.meta.split.SourceRecords; +import org.apache.flink.cdc.connectors.base.source.meta.split.SourceSplitState; import org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfigFactory; import org.apache.flink.cdc.connectors.oracle.source.meta.offset.RedoLogOffsetFactory; import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; +import org.apache.flink.connector.base.source.reader.RecordEmitter; import javax.annotation.Nullable; @@ -44,6 +47,7 @@ public class OracleSourceBuilder { private RedoLogOffsetFactory offsetFactory; private OracleDialect dialect; private DebeziumDeserializationSchema deserializer; + private static RecordEmitter recordEmitter; public OracleSourceBuilder hostname(String hostname) { this.configFactory.hostname(hostname); @@ -258,6 +262,13 @@ public OracleSourceBuilder assignUnboundedChunkFirst(boolean assignUnboundedC return this; } + /** Whether the {@link OracleIncrementalSource} should assign the recordEmitter object. */ + public OracleSourceBuilder recordEmitter( + RecordEmitter recordEmitter) { + this.recordEmitter = recordEmitter; + return this; + } + /** * Build the {@link OracleIncrementalSource}. * @@ -279,6 +290,7 @@ public OracleIncrementalSource( RedoLogOffsetFactory offsetFactory, OracleDialect dataSourceDialect) { super(configFactory, deserializationSchema, offsetFactory, dataSourceDialect); + super.setRecordEmitter(recordEmitter); } public static OracleSourceBuilder builder() {