From d9b253d538c337b853d54d072b0fc6f0bbe8fad9 Mon Sep 17 00:00:00 2001 From: dailai Date: Mon, 21 Oct 2024 15:33:01 +0800 Subject: [PATCH] [Feature][Connector-v2] Support schema evolution for Oracle connector --- docs/en/concept/schema-evolution.md | 99 +++- docs/zh/concept/schema-evolution.md | 93 +++- .../event/handler/AlterTableEventHandler.java | 10 +- .../schema/AbstractSchemaChangeResolver.java | 50 +- .../source/parser/SeatunnelDDLParser.java | 67 +++ .../cdc/base/utils/SourceRecordUtils.java | 13 +- ...SeaTunnelRowDebeziumDeserializeSchema.java | 10 +- .../IncrementalSourceStreamFetcherTest.java | 22 +- .../source/MySqlSchemaChangeResolver.java | 55 +- .../CustomAlterTableParserListener.java | 36 +- .../cdc/mysql/utils/MySqlTypeUtils.java | 9 +- .../source/OracleIncrementalSource.java | 3 + .../source/OracleSchemaChangeResolver.java | 51 ++ .../source/parser/BaseParserListener.java | 75 +++ .../CustomAlterTableParserListener.java | 231 +++++++++ .../CustomColumnDefinitionParserListener.java | 281 ++++++++++ .../parser/CustomOracleAntlrDdlParser.java | 59 +++ .../CustomOracleAntlrDdlParserListener.java | 68 +++ .../cdc/oracle/utils/OracleTypeUtils.java | 31 ++ .../source/parser/OracleDdlParserTest.java | 410 +++++++++++++++ .../jdbc/internal/dialect/JdbcDialect.java | 65 ++- .../internal/dialect/mysql/MysqlDialect.java | 5 +- .../oceanbase/OceanBaseMysqlDialect.java | 5 +- .../dialect/oracle/OracleDialect.java | 46 ++ .../jdbc/sink/AbstractJdbcSinkWriter.java | 8 +- .../seatunnel/jdbc/sink/JdbcSinkWriter.java | 23 + .../connector-cdc-oracle-e2e/pom.xml | 26 + .../cdc/oracle/AbstractOracleCDCIT.java | 154 ++++++ .../seatunnel/cdc/oracle/OracleCDCIT.java | 259 ++++------ .../oracle/OracleCDCWithSchemaChangeIT.java | 487 ++++++++++++++++++ .../src/test/resources/ddl/add_columns.sql | 77 +++ .../src/test/resources/ddl/drop_columns.sql | 54 ++ .../src/test/resources/ddl/full_types.sql | 78 +++ .../src/test/resources/ddl/modify_columns.sql | 37 ++ .../src/test/resources/ddl/rename_columns.sql | 37 ++ .../test/resources/docker/server-gtids/my.cnf | 65 +++ .../src/test/resources/docker/setup.sql | 35 ++ ...oraclecdc_to_mysql_with_schema_change.conf | 63 +++ ...raclecdc_to_oracle_with_schema_change.conf | 64 +++ ...racle_with_schema_change_exactly_once.conf | 67 +++ 40 files changed, 3060 insertions(+), 268 deletions(-) create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/parser/SeatunnelDDLParser.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleSchemaChangeResolver.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/BaseParserListener.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/CustomAlterTableParserListener.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/CustomColumnDefinitionParserListener.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/CustomOracleAntlrDdlParser.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/CustomOracleAntlrDdlParserListener.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/OracleDdlParserTest.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/AbstractOracleCDCIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/OracleCDCWithSchemaChangeIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/ddl/add_columns.sql create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/ddl/drop_columns.sql create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/ddl/full_types.sql create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/ddl/modify_columns.sql create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/ddl/rename_columns.sql create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/docker/server-gtids/my.cnf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/docker/setup.sql create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_mysql_with_schema_change.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_with_schema_change.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_with_schema_change_exactly_once.conf diff --git a/docs/en/concept/schema-evolution.md b/docs/en/concept/schema-evolution.md index 067bfc7b1c9..61596e820c7 100644 --- a/docs/en/concept/schema-evolution.md +++ b/docs/en/concept/schema-evolution.md @@ -1,19 +1,25 @@ # Schema evolution Schema Evolution means that the schema of a data table can be changed and the data synchronization task can automatically adapt to the changes of the new table structure without any other operations. -Now we only support the operation about `add column`、`drop column`、`rename column` and `modify column` of the table in CDC source. This feature is only support zeta engine at now. +Now we only support the operation about `add column`、`drop column`、`rename column` and `modify column` of the table in CDC source. This feature is only support zeta engine at now. + ## Supported connectors ### Source [Mysql-CDC](https://github.com/apache/seatunnel/blob/dev/docs/en/connector-v2/source/MySQL-CDC.md) +[Oracle-CDC](https://github.com/apache/seatunnel/blob/dev/docs/en/connector-v2/source/Oracle-CDC.md) ### Sink [Jdbc-Mysql](https://github.com/apache/seatunnel/blob/dev/docs/en/connector-v2/sink/Jdbc.md) +[Jdbc-Oracle](https://github.com/apache/seatunnel/blob/dev/docs/en/connector-v2/sink/Jdbc.md) + +Note: The schema evolution is not support the transform at now. The schema evolution of different types of databases(Oracle-CDC -> Jdbc-Mysql)is currently not supported the default value of the column in ddl. -Note: The schema evolution is not support the transform at now. +When you use the Oracle-CDC,you can not use the username named `SYS` or `SYSTEM` to modify the table schema, otherwise the ddl event will be filtered out which can lead to the schema evolution not working. +Otherwise, If your table name start with `ORA_TEMP_` will also has the same problem. ## Enable schema evolution -Schema evolution is disabled by default in CDC source. You need configure `debezium.include.schema.changes = true` which is only supported in MySQL-CDC to enable it. +Schema evolution is disabled by default in CDC source. You need configure `debezium.include.schema.changes = true` which is only supported in CDC to enable it. ## Examples @@ -56,3 +62,90 @@ sink { } } ``` + +### Oracle-cdc -> Jdbc-Oracle +``` +env { + # You can set engine configuration here + parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Oracle-CDC { + result_table_name = "customers" + username = "dbzuser" + password = "dbz" + database-names = ["ORCLCDB"] + schema-names = ["DEBEZIUM"] + table-names = ["ORCLCDB.DEBEZIUM.FULL_TYPES"] + base-url = "jdbc:oracle:thin:@oracle-host:1521/ORCLCDB" + source.reader.close.timeout = 120000 + connection.pool.size = 1 + debezium { + include.schema.changes = true + } + } +} + +sink { + Jdbc { + source_table_name = "customers" + driver = "oracle.jdbc.driver.OracleDriver" + url = "jdbc:oracle:thin:@oracle-host:1521/ORCLCDB" + user = "dbzuser" + password = "dbz" + generate_sink_sql = true + database = "ORCLCDB" + table = "DEBEZIUM.FULL_TYPES_SINK" + batch_size = 1 + primary_keys = ["ID"] + connection.pool.size = 1 + } +} +``` + +### Oracle-cdc -> Jdbc-Mysql +``` +env { + # You can set engine configuration here + parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Oracle-CDC { + result_table_name = "customers" + username = "dbzuser" + password = "dbz" + database-names = ["ORCLCDB"] + schema-names = ["DEBEZIUM"] + table-names = ["ORCLCDB.DEBEZIUM.FULL_TYPES"] + base-url = "jdbc:oracle:thin:@oracle-host:1521/ORCLCDB" + source.reader.close.timeout = 120000 + connection.pool.size = 1 + debezium { + include.schema.changes = true + } + } +} + +sink { + jdbc { + source_table_name = "customers" + url = "jdbc:mysql://oracle-host:3306/oracle_sink" + driver = "com.mysql.cj.jdbc.Driver" + user = "st_user_sink" + password = "mysqlpw" + generate_sink_sql = true + # You need to configure both database and table + database = oracle_sink + table = oracle_cdc_2_mysql_sink_table + primary_keys = ["ID"] + } +} +``` diff --git a/docs/zh/concept/schema-evolution.md b/docs/zh/concept/schema-evolution.md index 16f0f5dbc81..4521832934c 100644 --- a/docs/zh/concept/schema-evolution.md +++ b/docs/zh/concept/schema-evolution.md @@ -6,12 +6,16 @@ ### 源 [Mysql-CDC](https://github.com/apache/seatunnel/blob/dev/docs/en/connector-v2/source/MySQL-CDC.md) +[Oracle-CDC](https://github.com/apache/seatunnel/blob/dev/docs/en/connector-v2/source/Oracle-CDC.md) ### 目标 [Jdbc-Mysql](https://github.com/apache/seatunnel/blob/dev/docs/zh/connector-v2/sink/Jdbc.md) +[Jdbc-Oracle](https://github.com/apache/seatunnel/blob/dev/docs/en/connector-v2/sink/Jdbc.md) -注意: 目前模式演进不支持transform. +注意: 目前模式演进不支持transform。不同类型数据库(Oracle-CDC -> Jdbc-Mysql)的模式演进目前不支持ddl中列的默认值。 +当你使用Oracle-CDC时,你不能使用用户名`SYS`或`SYSTEM`来修改表结构,否则ddl事件将被过滤,这可能导致模式演进不起作用; +另外,如果你的表名以`ORA_TEMP_`开头,也会有相同的问题。 ## 启用Schema evolution功能 在CDC源连接器中模式演进默认是关闭的。你需要在CDC连接器中配置`debezium.include.schema.changes = true`来启用它。 @@ -57,3 +61,90 @@ sink { } } ``` + +### Oracle-cdc -> Jdbc-Oracle +``` +env { + # You can set engine configuration here + parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Oracle-CDC { + result_table_name = "customers" + username = "dbzuser" + password = "dbz" + database-names = ["ORCLCDB"] + schema-names = ["DEBEZIUM"] + table-names = ["ORCLCDB.DEBEZIUM.FULL_TYPES"] + base-url = "jdbc:oracle:thin:@oracle-host:1521/ORCLCDB" + source.reader.close.timeout = 120000 + connection.pool.size = 1 + debezium { + include.schema.changes = true + } + } +} + +sink { + Jdbc { + source_table_name = "customers" + driver = "oracle.jdbc.driver.OracleDriver" + url = "jdbc:oracle:thin:@oracle-host:1521/ORCLCDB" + user = "dbzuser" + password = "dbz" + generate_sink_sql = true + database = "ORCLCDB" + table = "DEBEZIUM.FULL_TYPES_SINK" + batch_size = 1 + primary_keys = ["ID"] + connection.pool.size = 1 + } +} +``` + +### Oracle-cdc -> Jdbc-Mysql +``` +env { + # You can set engine configuration here + parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Oracle-CDC { + result_table_name = "customers" + username = "dbzuser" + password = "dbz" + database-names = ["ORCLCDB"] + schema-names = ["DEBEZIUM"] + table-names = ["ORCLCDB.DEBEZIUM.FULL_TYPES"] + base-url = "jdbc:oracle:thin:@oracle-host:1521/ORCLCDB" + source.reader.close.timeout = 120000 + connection.pool.size = 1 + debezium { + include.schema.changes = true + } + } +} + +sink { + jdbc { + source_table_name = "customers" + url = "jdbc:mysql://oracle-host:3306/oracle_sink" + driver = "com.mysql.cj.jdbc.Driver" + user = "st_user_sink" + password = "mysqlpw" + generate_sink_sql = true + # You need to configure both database and table + database = oracle_sink + table = oracle_cdc_2_mysql_sink_table + primary_keys = ["ID"] + } +} +``` diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/event/handler/AlterTableEventHandler.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/event/handler/AlterTableEventHandler.java index b0972ec68a0..46ae76fc873 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/event/handler/AlterTableEventHandler.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/event/handler/AlterTableEventHandler.java @@ -154,10 +154,18 @@ private SeaTunnelRowType applyChangeColumn( String oldColumn = changeColumnEvent.getOldColumn(); int oldColumnIndex = dataType.indexOf(oldColumn); + // Rename column of oracle which only has the name of old column and the name of new column, + // so we need to fill the data type which is the same as the old column. + SeaTunnelDataType fieldType = dataType.getFieldType(oldColumnIndex); + Column column = changeColumnEvent.getColumn(); + if (column.getDataType() == null) { + column = column.copy(fieldType); + } + return applyModifyColumn( dataType, oldColumnIndex, - changeColumnEvent.getColumn(), + column, changeColumnEvent.isFirst(), changeColumnEvent.getAfterColumn()); } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/schema/AbstractSchemaChangeResolver.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/schema/AbstractSchemaChangeResolver.java index ac86dd0d2bc..eba4d23ce80 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/schema/AbstractSchemaChangeResolver.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/schema/AbstractSchemaChangeResolver.java @@ -17,6 +17,12 @@ package org.apache.seatunnel.connectors.cdc.base.schema; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.event.AlterTableColumnEvent; +import org.apache.seatunnel.api.table.event.AlterTableColumnsEvent; +import org.apache.seatunnel.api.table.event.SchemaChangeEvent; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; import org.apache.seatunnel.connectors.cdc.base.utils.SourceRecordUtils; @@ -25,17 +31,24 @@ import org.apache.kafka.connect.source.SourceRecord; import com.google.common.collect.Lists; +import io.debezium.relational.Tables; +import io.debezium.relational.ddl.DdlParser; import io.debezium.relational.history.HistoryRecord; +import lombok.Setter; import lombok.extern.slf4j.Slf4j; import java.util.List; +import java.util.Objects; @Slf4j public abstract class AbstractSchemaChangeResolver implements SchemaChangeResolver { protected static final List SUPPORT_DDL = Lists.newArrayList("ALTER TABLE"); - protected JdbcSourceConfig jdbcSourceConfig; + protected final JdbcSourceConfig jdbcSourceConfig; + @Setter protected transient DdlParser ddlParser; + @Setter protected transient Tables tables; + @Setter protected String sourceDialectName; public AbstractSchemaChangeResolver(JdbcSourceConfig jdbcSourceConfig) { this.jdbcSourceConfig = jdbcSourceConfig; @@ -55,4 +68,39 @@ public boolean support(SourceRecord record) { .map(String::toUpperCase) .anyMatch(prefix -> ddl.toUpperCase().contains(prefix)); } + + @Override + public SchemaChangeEvent resolve(SourceRecord record, SeaTunnelDataType dataType) { + TablePath tablePath = SourceRecordUtils.getTablePath(record); + String ddl = SourceRecordUtils.getDdl(record); + if (Objects.isNull(ddlParser)) { + this.ddlParser = createDdlParser(tablePath); + } + if (Objects.isNull(tables)) { + this.tables = new Tables(); + } + ddlParser.setCurrentDatabase(tablePath.getDatabaseName()); + ddlParser.setCurrentSchema(tablePath.getSchemaName()); + // Parse DDL statement using Debezium's Antlr parser + ddlParser.parse(ddl, tables); + List parsedEvents = getAndClearParsedEvents(); + parsedEvents.forEach(e -> e.setSourceDialectName(getSourceDialectName())); + AlterTableColumnsEvent alterTableColumnsEvent = + new AlterTableColumnsEvent( + TableIdentifier.of( + StringUtils.EMPTY, + tablePath.getDatabaseName(), + tablePath.getSchemaName(), + tablePath.getTableName()), + parsedEvents); + alterTableColumnsEvent.setStatement(ddl); + alterTableColumnsEvent.setSourceDialectName(getSourceDialectName()); + return parsedEvents.isEmpty() ? null : alterTableColumnsEvent; + } + + protected abstract DdlParser createDdlParser(TablePath tablePath); + + protected abstract List getAndClearParsedEvents(); + + protected abstract String getSourceDialectName(); } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/parser/SeatunnelDDLParser.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/parser/SeatunnelDDLParser.java new file mode 100644 index 00000000000..355df7fddef --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/parser/SeatunnelDDLParser.java @@ -0,0 +1,67 @@ +/* + * 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.seatunnel.connectors.cdc.base.source.parser; + +import org.apache.seatunnel.api.table.catalog.TableIdentifier; + +import org.apache.commons.lang3.StringUtils; + +import io.debezium.relational.Column; +import io.debezium.relational.TableId; + +public interface SeatunnelDDLParser { + + /** + * @param column The column to convert + * @return The converted column in SeaTunnel format which has full type information + */ + default org.apache.seatunnel.api.table.catalog.Column toSeatunnelColumnWithFullTypeInfo( + Column column) { + org.apache.seatunnel.api.table.catalog.Column seatunnelColumn = toSeatunnelColumn(column); + String sourceColumnType = getSourceColumnTypeWithLengthScale(column); + return seatunnelColumn.reSourceType(sourceColumnType); + } + + /** + * @param column The column to convert + * @return The converted column in SeaTunnel format + */ + org.apache.seatunnel.api.table.catalog.Column toSeatunnelColumn(Column column); + + /** + * @param column The column to convert + * @return The type with length and scale + */ + default String getSourceColumnTypeWithLengthScale(Column column) { + StringBuilder sb = new StringBuilder(column.typeName()); + if (column.length() >= 0) { + sb.append('(').append(column.length()); + if (column.scale().isPresent()) { + sb.append(", ").append(column.scale().get()); + } + + sb.append(')'); + } + return sb.toString(); + } + + default TableIdentifier toTableIdentifier(TableId tableId) { + return new TableIdentifier( + StringUtils.EMPTY, tableId.catalog(), tableId.schema(), tableId.table()); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/utils/SourceRecordUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/utils/SourceRecordUtils.java index 3245273ace2..f7e9577ddba 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/utils/SourceRecordUtils.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/utils/SourceRecordUtils.java @@ -36,6 +36,7 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.util.Arrays; +import java.util.List; import static io.debezium.connector.AbstractSourceInfo.DATABASE_NAME_KEY; import static io.debezium.connector.AbstractSourceInfo.SCHEMA_NAME_KEY; @@ -46,8 +47,12 @@ public class SourceRecordUtils { private SourceRecordUtils() {} - public static final String SCHEMA_CHANGE_EVENT_KEY_NAME = - "io.debezium.connector.mysql.SchemaChangeKey"; + /** Todo: Support more schema change event key name, currently only support MySQL and Oracle. */ + public static final List SUPPORT_SCHEMA_CHANGE_EVENT_KEY_NAME = + Arrays.asList( + "io.debezium.connector.mysql.SchemaChangeKey", + "io.debezium.connector.oracle.SchemaChangeKey"); + public static final String HEARTBEAT_VALUE_SCHEMA_KEY_NAME = "io.debezium.connector.common.Heartbeat"; private static final DocumentReader DOCUMENT_READER = DocumentReader.defaultReader(); @@ -97,7 +102,9 @@ public static Long getFetchTimestamp(SourceRecord record) { public static boolean isSchemaChangeEvent(SourceRecord sourceRecord) { Schema keySchema = sourceRecord.keySchema(); - return keySchema != null && SCHEMA_CHANGE_EVENT_KEY_NAME.equalsIgnoreCase(keySchema.name()); + return keySchema != null + && SUPPORT_SCHEMA_CHANGE_EVENT_KEY_NAME.stream() + .anyMatch(name -> name.equalsIgnoreCase(keySchema.name())); } public static boolean isDataChangeRecord(SourceRecord record) { diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/debezium/row/SeaTunnelRowDebeziumDeserializeSchema.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/debezium/row/SeaTunnelRowDebeziumDeserializeSchema.java index d09e7b77b5c..948e872d480 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/debezium/row/SeaTunnelRowDebeziumDeserializeSchema.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/debezium/row/SeaTunnelRowDebeziumDeserializeSchema.java @@ -117,7 +117,15 @@ public void deserialize(SourceRecord record, Collector collector) private void deserializeSchemaChangeRecord( SourceRecord record, Collector collector) { - SchemaChangeEvent schemaChangeEvent = schemaChangeResolver.resolve(record, resultTypeInfo); + SchemaChangeEvent schemaChangeEvent = null; + try { + if (schemaChangeResolver != null) { + schemaChangeEvent = schemaChangeResolver.resolve(record, resultTypeInfo); + } + } catch (Exception e) { + log.warn("Failed to resolve schemaChangeEvent, just skip.", e); + return; + } if (schemaChangeEvent == null) { log.warn("Unsupported resolve schemaChangeEvent {}, just skip.", record); return; diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcherTest.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcherTest.java index ee8d4d7e5df..23906ae6f47 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcherTest.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcherTest.java @@ -50,6 +50,7 @@ import java.util.concurrent.atomic.AtomicReference; import static io.debezium.config.CommonConnectorConfig.TRANSACTION_TOPIC; +import static io.debezium.connector.AbstractSourceInfo.DEBEZIUM_CONNECTOR_KEY; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -336,7 +337,22 @@ static SourceRecord createSchemaChangeUnknownEvent() { static SourceRecord createSchemaChangeEvent(String topic) { Schema keySchema = - SchemaBuilder.struct().name(SourceRecordUtils.SCHEMA_CHANGE_EVENT_KEY_NAME).build(); + SchemaBuilder.struct().name("io.debezium.connector.mysql.SchemaChangeKey").build(); + Schema valueKeySchema = + SchemaBuilder.struct() + .name("io.debezium.connector.mysql.Source") + .field(DEBEZIUM_CONNECTOR_KEY, Schema.STRING_SCHEMA) + .build(); + Struct valueValues = new Struct(valueKeySchema); + valueValues.put(DEBEZIUM_CONNECTOR_KEY, "mysql"); + + Schema valueSchema = + SchemaBuilder.struct() + .field(Envelope.FieldName.SOURCE, valueKeySchema) + .name("") + .build(); + Struct value = new Struct(valueSchema); + value.put(valueSchema.field(Envelope.FieldName.SOURCE), valueValues); SourceRecord record = new SourceRecord( Collections.emptyMap(), @@ -344,8 +360,8 @@ static SourceRecord createSchemaChangeEvent(String topic) { topic, keySchema, null, - null, - null); + valueSchema, + value); Assertions.assertTrue(SourceRecordUtils.isSchemaChangeEvent(record)); return record; } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MySqlSchemaChangeResolver.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MySqlSchemaChangeResolver.java index 3ea4a0dfcea..7420f91eea7 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MySqlSchemaChangeResolver.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MySqlSchemaChangeResolver.java @@ -17,64 +17,37 @@ package org.apache.seatunnel.connectors.seatunnel.cdc.mysql.source; -import org.apache.seatunnel.api.table.catalog.TableIdentifier; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.event.AlterTableColumnEvent; -import org.apache.seatunnel.api.table.event.AlterTableColumnsEvent; -import org.apache.seatunnel.api.table.event.SchemaChangeEvent; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; import org.apache.seatunnel.connectors.cdc.base.config.SourceConfig; import org.apache.seatunnel.connectors.cdc.base.schema.AbstractSchemaChangeResolver; -import org.apache.seatunnel.connectors.cdc.base.utils.SourceRecordUtils; import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.source.parser.CustomMySqlAntlrDdlParser; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; -import org.apache.commons.lang3.StringUtils; -import org.apache.kafka.connect.source.SourceRecord; - -import io.debezium.relational.Tables; +import io.debezium.relational.ddl.DdlParser; import java.util.List; -import java.util.Objects; public class MySqlSchemaChangeResolver extends AbstractSchemaChangeResolver { - private transient Tables tables; - private transient CustomMySqlAntlrDdlParser customMySqlAntlrDdlParser; public MySqlSchemaChangeResolver(SourceConfig.Factory sourceConfigFactory) { super(sourceConfigFactory.create(0)); } @Override - public SchemaChangeEvent resolve(SourceRecord record, SeaTunnelDataType dataType) { - TablePath tablePath = SourceRecordUtils.getTablePath(record); - String ddl = SourceRecordUtils.getDdl(record); - if (Objects.isNull(customMySqlAntlrDdlParser)) { - this.customMySqlAntlrDdlParser = - new CustomMySqlAntlrDdlParser( - tablePath, this.jdbcSourceConfig.getDbzConnectorConfig()); - } - if (Objects.isNull(tables)) { - this.tables = new Tables(); - } - customMySqlAntlrDdlParser.setCurrentDatabase(tablePath.getDatabaseName()); - customMySqlAntlrDdlParser.setCurrentSchema(tablePath.getSchemaName()); - // Parse DDL statement using Debezium's Antlr parser - customMySqlAntlrDdlParser.parse(ddl, tables); - List parsedEvents = - customMySqlAntlrDdlParser.getAndClearParsedEvents(); - parsedEvents.forEach(e -> e.setSourceDialectName(DatabaseIdentifier.MYSQL)); - AlterTableColumnsEvent alterTableColumnsEvent = - new AlterTableColumnsEvent( - TableIdentifier.of( - StringUtils.EMPTY, - tablePath.getDatabaseName(), - tablePath.getSchemaName(), - tablePath.getTableName()), - parsedEvents); - alterTableColumnsEvent.setStatement(ddl); - alterTableColumnsEvent.setSourceDialectName(DatabaseIdentifier.MYSQL); - return parsedEvents.isEmpty() ? null : alterTableColumnsEvent; + protected DdlParser createDdlParser(TablePath tablePath) { + return new CustomMySqlAntlrDdlParser( + tablePath, this.jdbcSourceConfig.getDbzConnectorConfig()); + } + + @Override + protected List getAndClearParsedEvents() { + return ((CustomMySqlAntlrDdlParser) ddlParser).getAndClearParsedEvents(); + } + + @Override + protected String getSourceDialectName() { + return DatabaseIdentifier.MYSQL; } } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/parser/CustomAlterTableParserListener.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/parser/CustomAlterTableParserListener.java index bf36d7831ee..2a1e9b2762f 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/parser/CustomAlterTableParserListener.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/parser/CustomAlterTableParserListener.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.api.table.event.AlterTableColumnEvent; import org.apache.seatunnel.api.table.event.AlterTableDropColumnEvent; import org.apache.seatunnel.api.table.event.AlterTableModifyColumnEvent; +import org.apache.seatunnel.connectors.cdc.base.source.parser.SeatunnelDDLParser; import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.utils.MySqlTypeUtils; import org.apache.commons.lang3.StringUtils; @@ -40,7 +41,8 @@ import java.util.LinkedList; import java.util.List; -public class CustomAlterTableParserListener extends MySqlParserBaseListener { +public class CustomAlterTableParserListener extends MySqlParserBaseListener + implements SeatunnelDDLParser { private static final int STARTING_INDEX = 1; private final MySqlAntlrDdlParser parser; private final List listeners; @@ -95,9 +97,7 @@ public void exitAlterByAddColumn(MySqlParser.AlterByAddColumnContext ctx) { () -> { Column column = columnDefinitionListener.getColumn(); org.apache.seatunnel.api.table.catalog.Column seatunnelColumn = - toSeatunnelColumn(column); - String sourceColumnType = getSourceColumnType(column); - seatunnelColumn = seatunnelColumn.reSourceType(sourceColumnType); + toSeatunnelColumnWithFullTypeInfo(column); if (ctx.FIRST() != null) { AlterTableAddColumnEvent alterTableAddColumnEvent = AlterTableAddColumnEvent.addFirst(tableIdentifier, seatunnelColumn); @@ -153,9 +153,7 @@ public void exitAlterByModifyColumn(MySqlParser.AlterByModifyColumnContext ctx) () -> { Column column = columnDefinitionListener.getColumn(); org.apache.seatunnel.api.table.catalog.Column seatunnelColumn = - toSeatunnelColumn(column); - String sourceColumnType = getSourceColumnType(column); - seatunnelColumn = seatunnelColumn.reSourceType(sourceColumnType); + toSeatunnelColumnWithFullTypeInfo(column); if (ctx.FIRST() != null) { AlterTableModifyColumnEvent alterTableModifyColumnEvent = AlterTableModifyColumnEvent.modifyFirst( @@ -197,9 +195,7 @@ public void exitAlterByChangeColumn(MySqlParser.AlterByChangeColumnContext ctx) () -> { Column column = columnDefinitionListener.getColumn(); org.apache.seatunnel.api.table.catalog.Column seatunnelColumn = - toSeatunnelColumn(column); - String sourceColumnType = getSourceColumnType(column); - seatunnelColumn = seatunnelColumn.reSourceType(sourceColumnType); + toSeatunnelColumnWithFullTypeInfo(column); String oldColumnName = column.name(); String newColumnName = parser.parseName(ctx.newColumn); seatunnelColumn = seatunnelColumn.rename(newColumnName); @@ -223,24 +219,8 @@ public void enterAlterByDropColumn(MySqlParser.AlterByDropColumnContext ctx) { super.enterAlterByDropColumn(ctx); } - private org.apache.seatunnel.api.table.catalog.Column toSeatunnelColumn(Column column) { + @Override + public org.apache.seatunnel.api.table.catalog.Column toSeatunnelColumn(Column column) { return MySqlTypeUtils.convertToSeaTunnelColumn(column, dbzConnectorConfig); } - - private TableIdentifier toTableIdentifier(TableId tableId) { - return new TableIdentifier("", tableId.catalog(), tableId.schema(), tableId.table()); - } - - private String getSourceColumnType(Column column) { - StringBuilder sb = new StringBuilder(column.typeName()); - if (column.length() >= 0) { - sb.append('(').append(column.length()); - if (column.scale().isPresent()) { - sb.append(", ").append(column.scale().get()); - } - - sb.append(')'); - } - return sb.toString(); - } } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlTypeUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlTypeUtils.java index 22f9514c6f3..fd85258eb3c 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlTypeUtils.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlTypeUtils.java @@ -30,7 +30,6 @@ import io.debezium.relational.RelationalDatabaseConnectorConfig; import lombok.extern.slf4j.Slf4j; -import java.util.Objects; import java.util.Optional; /** Utilities for converting from MySQL types to SeaTunnel types. */ @@ -70,7 +69,6 @@ public static org.apache.seatunnel.api.table.catalog.Column convertToSeaTunnelCo Optional defaultValueExpression = column.defaultValueExpression(); Object defaultValue = defaultValueExpression.orElse(null); if (defaultValueExpression.isPresent() - && Objects.nonNull(defaultValue) && !MysqlDefaultValueUtils.isSpecialDefaultValue(defaultValue)) { defaultValue = mySqlDefaultValueConverter @@ -82,11 +80,14 @@ public static org.apache.seatunnel.api.table.catalog.Column convertToSeaTunnelCo .name(column.name()) .columnType(column.typeName()) .dataType(column.typeName()) - .length((long) column.length()) - .precision((long) column.length()) .scale(column.scale().orElse(0)) .nullable(column.isOptional()) .defaultValue(defaultValue); + + if (column.length() >= 0) { + builder.length((long) column.length()).precision((long) column.length()); + } + switch (column.typeName().toUpperCase()) { case MySqlTypeConverter.MYSQL_CHAR: case MySqlTypeConverter.MYSQL_VARCHAR: diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleIncrementalSource.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleIncrementalSource.java index a1bbd0cb25c..4401a636dc5 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleIncrementalSource.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleIncrementalSource.java @@ -109,11 +109,14 @@ public DebeziumDeserializationSchema createDebeziumDeserializationSchema( SeaTunnelDataType physicalRowType = dataType; String zoneId = config.get(JdbcSourceOptions.SERVER_TIME_ZONE); + return (DebeziumDeserializationSchema) SeaTunnelRowDebeziumDeserializeSchema.builder() .setPhysicalRowType(physicalRowType) .setResultTypeInfo(physicalRowType) .setServerTimeZone(ZoneId.of(zoneId)) + .setSchemaChangeResolver( + new OracleSchemaChangeResolver(createSourceConfigFactory(config))) .build(); } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleSchemaChangeResolver.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleSchemaChangeResolver.java new file mode 100644 index 00000000000..326209eec44 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleSchemaChangeResolver.java @@ -0,0 +1,51 @@ +/* + * 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.seatunnel.connectors.seatunnel.cdc.oracle.source; + +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.event.AlterTableColumnEvent; +import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; +import org.apache.seatunnel.connectors.cdc.base.config.SourceConfig; +import org.apache.seatunnel.connectors.cdc.base.schema.AbstractSchemaChangeResolver; +import org.apache.seatunnel.connectors.seatunnel.cdc.oracle.source.parser.CustomOracleAntlrDdlParser; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; + +import io.debezium.relational.ddl.DdlParser; + +import java.util.List; + +public class OracleSchemaChangeResolver extends AbstractSchemaChangeResolver { + public OracleSchemaChangeResolver(SourceConfig.Factory sourceConfigFactory) { + super(sourceConfigFactory.create(0)); + } + + @Override + protected DdlParser createDdlParser(TablePath tablePath) { + return new CustomOracleAntlrDdlParser(tablePath); + } + + @Override + protected List getAndClearParsedEvents() { + return ((CustomOracleAntlrDdlParser) ddlParser).getAndClearParsedEvents(); + } + + @Override + protected String getSourceDialectName() { + return DatabaseIdentifier.ORACLE; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/BaseParserListener.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/BaseParserListener.java new file mode 100644 index 00000000000..cba04a72625 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/BaseParserListener.java @@ -0,0 +1,75 @@ +/* + * 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.seatunnel.connectors.seatunnel.cdc.oracle.source.parser; + +import io.debezium.ddl.parser.oracle.generated.PlSqlParser; +import io.debezium.ddl.parser.oracle.generated.PlSqlParserBaseListener; + +public class BaseParserListener extends PlSqlParserBaseListener { + + /** + * 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 != null) ? text.toUpperCase() : text; + } + + String getColumnName(final PlSqlParser.Column_nameContext ctx) { + final String columnName; + if (ctx.id_expression() != null && !ctx.id_expression().isEmpty()) { + columnName = + getTableOrColumnName( + ctx.id_expression(ctx.id_expression().size() - 1).getText()); + } else { + columnName = getTableOrColumnName(ctx.identifier().id_expression().getText()); + } + return columnName; + } + + String getColumnName(final PlSqlParser.Old_column_nameContext ctx) { + return getTableOrColumnName(ctx.getText()); + } + + String getColumnName(final PlSqlParser.New_column_nameContext ctx) { + return getTableOrColumnName(ctx.getText()); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/CustomAlterTableParserListener.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/CustomAlterTableParserListener.java new file mode 100644 index 00000000000..7ebc7b49fff --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/CustomAlterTableParserListener.java @@ -0,0 +1,231 @@ +/* + * 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.seatunnel.connectors.seatunnel.cdc.oracle.source.parser; + +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.event.AlterTableAddColumnEvent; +import org.apache.seatunnel.api.table.event.AlterTableChangeColumnEvent; +import org.apache.seatunnel.api.table.event.AlterTableColumnEvent; +import org.apache.seatunnel.api.table.event.AlterTableDropColumnEvent; +import org.apache.seatunnel.api.table.event.AlterTableModifyColumnEvent; +import org.apache.seatunnel.connectors.cdc.base.source.parser.SeatunnelDDLParser; +import org.apache.seatunnel.connectors.seatunnel.cdc.oracle.utils.OracleTypeUtils; + +import org.apache.commons.lang3.StringUtils; + +import org.antlr.v4.runtime.tree.ParseTreeListener; + +import io.debezium.ddl.parser.oracle.generated.PlSqlParser; +import io.debezium.relational.Column; +import io.debezium.relational.ColumnEditor; +import io.debezium.relational.TableId; +import lombok.extern.slf4j.Slf4j; + +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; + +@Slf4j +public class CustomAlterTableParserListener extends BaseParserListener + implements SeatunnelDDLParser { + + private static final int STARTING_INDEX = 0; + private CustomOracleAntlrDdlParser parser; + private final List listeners; + private CustomColumnDefinitionParserListener columnDefinitionListener; + private List columnEditors; + private int parsingColumnIndex = STARTING_INDEX; + + private final LinkedList changes; + private TableIdentifier tableIdentifier; + + public CustomAlterTableParserListener( + CustomOracleAntlrDdlParser parser, + List listeners, + LinkedList changes) { + this.parser = parser; + this.listeners = listeners; + this.changes = changes; + } + + @Override + public void enterAlter_table(PlSqlParser.Alter_tableContext ctx) { + TableId tableId = this.parser.parseQualifiedTableId(); + this.tableIdentifier = toTableIdentifier(tableId); + super.enterAlter_table(ctx); + } + + @Override + public void exitAlter_table(PlSqlParser.Alter_tableContext ctx) { + listeners.remove(columnDefinitionListener); + super.exitAlter_table(ctx); + } + + @Override + public void enterAdd_column_clause(PlSqlParser.Add_column_clauseContext ctx) { + List columns = ctx.column_definition(); + columnEditors = new ArrayList<>(columns.size()); + for (PlSqlParser.Column_definitionContext column : columns) { + String columnName = getColumnName(column.column_name()); + ColumnEditor editor = Column.editor().name(columnName); + columnEditors.add(editor); + } + columnDefinitionListener = new CustomColumnDefinitionParserListener(); + listeners.add(columnDefinitionListener); + super.enterAdd_column_clause(ctx); + } + + @Override + public void exitAdd_column_clause(PlSqlParser.Add_column_clauseContext ctx) { + columnEditors.forEach( + columnEditor -> { + Column column = columnEditor.create(); + org.apache.seatunnel.api.table.catalog.Column seaTunnelColumn = + toSeatunnelColumnWithFullTypeInfo(column); + AlterTableAddColumnEvent addEvent = + AlterTableAddColumnEvent.add(tableIdentifier, seaTunnelColumn); + changes.add(addEvent); + }); + listeners.remove(columnDefinitionListener); + columnDefinitionListener = null; + super.exitAdd_column_clause(ctx); + } + + @Override + public void enterModify_column_clauses(PlSqlParser.Modify_column_clausesContext ctx) { + List columns = ctx.modify_col_properties(); + columnEditors = new ArrayList<>(columns.size()); + for (PlSqlParser.Modify_col_propertiesContext column : columns) { + String columnName = getColumnName(column.column_name()); + ColumnEditor editor = Column.editor().name(columnName); + columnEditors.add(editor); + } + columnDefinitionListener = new CustomColumnDefinitionParserListener(); + listeners.add(columnDefinitionListener); + super.enterModify_column_clauses(ctx); + } + + @Override + public void exitModify_column_clauses(PlSqlParser.Modify_column_clausesContext ctx) { + parser.runIfNotNull( + () -> { + Column column = columnDefinitionListener.getColumn(); + org.apache.seatunnel.api.table.catalog.Column seaTunnelColumn = + toSeatunnelColumnWithFullTypeInfo(column); + AlterTableModifyColumnEvent alterTableModifyColumnEvent = + AlterTableModifyColumnEvent.modify(tableIdentifier, seaTunnelColumn); + changes.add(alterTableModifyColumnEvent); + listeners.remove(columnDefinitionListener); + columnDefinitionListener = null; + super.exitModify_column_clauses(ctx); + }, + columnDefinitionListener); + } + + @Override + public void enterModify_col_properties(PlSqlParser.Modify_col_propertiesContext ctx) { + parser.runIfNotNull( + () -> { + // 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 + columnDefinitionListener.setColumnEditor( + columnEditors.get(parsingColumnIndex++)); + } + }, + columnEditors); + super.enterModify_col_properties(ctx); + } + + @Override + public void exitModify_col_properties(PlSqlParser.Modify_col_propertiesContext ctx) { + parser.runIfNotNull( + () -> { + if (columnEditors.size() == parsingColumnIndex) { + // all columns parsed + // reset global variables for next parsed statement + parsingColumnIndex = STARTING_INDEX; + } + }, + columnEditors); + super.exitModify_col_properties(ctx); + } + + @Override + public void enterColumn_definition(PlSqlParser.Column_definitionContext ctx) { + parser.runIfNotNull( + () -> { + // 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 + columnDefinitionListener.setColumnEditor( + columnEditors.get(parsingColumnIndex++)); + } + }, + columnEditors); + } + + @Override + public void exitColumn_definition(PlSqlParser.Column_definitionContext ctx) { + parser.runIfNotNull( + () -> { + if (columnEditors.size() == parsingColumnIndex) { + // all columns parsed + // reset global variables for next parsed statement + parsingColumnIndex = STARTING_INDEX; + } + }, + columnEditors); + super.exitColumn_definition(ctx); + } + + @Override + public void enterDrop_column_clause(PlSqlParser.Drop_column_clauseContext ctx) { + List columnNameContexts = ctx.column_name(); + columnEditors = new ArrayList<>(columnNameContexts.size()); + for (PlSqlParser.Column_nameContext columnNameContext : columnNameContexts) { + String columnName = getColumnName(columnNameContext); + AlterTableDropColumnEvent alterTableDropColumnEvent = + new AlterTableDropColumnEvent(tableIdentifier, columnName); + changes.add(alterTableDropColumnEvent); + } + super.enterDrop_column_clause(ctx); + } + + @Override + public void enterRename_column_clause(PlSqlParser.Rename_column_clauseContext ctx) { + String oldColumnName = getColumnName(ctx.old_column_name()); + String newColumnName = getColumnName(ctx.new_column_name()); + PhysicalColumn newColumn = PhysicalColumn.builder().name(newColumnName).build(); + AlterTableChangeColumnEvent alterTableChangeColumnEvent = + AlterTableChangeColumnEvent.change(tableIdentifier, oldColumnName, newColumn); + if (StringUtils.isNotBlank(newColumnName) + && !StringUtils.equals(oldColumnName, newColumnName)) { + changes.add(alterTableChangeColumnEvent); + } + super.enterRename_column_clause(ctx); + } + + @Override + public org.apache.seatunnel.api.table.catalog.Column toSeatunnelColumn(Column column) { + return OracleTypeUtils.convertToSeaTunnelColumn(column); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/CustomColumnDefinitionParserListener.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/CustomColumnDefinitionParserListener.java new file mode 100644 index 00000000000..12e02dbd281 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/CustomColumnDefinitionParserListener.java @@ -0,0 +1,281 @@ +/* + * 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.seatunnel.connectors.seatunnel.cdc.oracle.source.parser; + +import io.debezium.ddl.parser.oracle.generated.PlSqlParser; +import io.debezium.relational.Column; +import io.debezium.relational.ColumnEditor; +import lombok.Getter; +import lombok.Setter; +import oracle.jdbc.OracleTypes; + +import java.sql.Types; + +@Getter +@Setter +public class CustomColumnDefinitionParserListener extends BaseParserListener { + private ColumnEditor columnEditor; + + public CustomColumnDefinitionParserListener() {} + + @Override + public void enterColumn_definition(PlSqlParser.Column_definitionContext ctx) { + if (columnEditor != null) { + resolveColumnDataType(ctx); + if (ctx.DEFAULT() != null) { + this.columnEditor.defaultValueExpression(ctx.column_default_value().getText()); + } + } + super.enterColumn_definition(ctx); + } + + @Override + public void enterModify_col_properties(PlSqlParser.Modify_col_propertiesContext ctx) { + if (columnEditor != null) { + 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) { + 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"); + + // 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) { + 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()); + } + } + + public Column getColumn() { + return columnEditor.create(); + } + + 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/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/CustomOracleAntlrDdlParser.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/CustomOracleAntlrDdlParser.java new file mode 100644 index 00000000000..1d0384d7ed4 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/CustomOracleAntlrDdlParser.java @@ -0,0 +1,59 @@ +/* + * 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.seatunnel.connectors.seatunnel.cdc.oracle.source.parser; + +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.event.AlterTableColumnEvent; + +import com.google.common.collect.Lists; +import io.debezium.antlr.AntlrDdlParserListener; +import io.debezium.connector.oracle.antlr.OracleDdlParser; +import io.debezium.relational.TableId; + +import java.util.LinkedList; +import java.util.List; + +/** A ddl parser that will use custom listener. */ +public class CustomOracleAntlrDdlParser extends OracleDdlParser { + + private final LinkedList parsedEvents; + + private final TablePath tablePath; + + public CustomOracleAntlrDdlParser(TablePath tablePath) { + super(); + this.tablePath = tablePath; + this.parsedEvents = new LinkedList<>(); + } + + public TableId parseQualifiedTableId() { + return new TableId( + tablePath.getDatabaseName(), tablePath.getSchemaName(), tablePath.getTableName()); + } + + @Override + protected AntlrDdlParserListener createParseTreeWalkerListener() { + return new CustomOracleAntlrDdlParserListener(this, parsedEvents); + } + + public List getAndClearParsedEvents() { + List result = Lists.newArrayList(parsedEvents); + parsedEvents.clear(); + return result; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/CustomOracleAntlrDdlParserListener.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/CustomOracleAntlrDdlParserListener.java new file mode 100644 index 00000000000..1924861c4b0 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/CustomOracleAntlrDdlParserListener.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.oracle.source.parser; + +import org.apache.seatunnel.api.table.event.AlterTableColumnEvent; + +import org.antlr.v4.runtime.ParserRuleContext; +import org.antlr.v4.runtime.tree.ParseTreeListener; + +import io.debezium.antlr.AntlrDdlParserListener; +import io.debezium.antlr.ProxyParseTreeListenerUtil; +import io.debezium.text.ParsingException; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; + +public class CustomOracleAntlrDdlParserListener extends BaseParserListener + implements AntlrDdlParserListener { + + private final List listeners = new CopyOnWriteArrayList<>(); + private final Collection errors = new ArrayList<>(); + + public CustomOracleAntlrDdlParserListener( + CustomOracleAntlrDdlParser parser, LinkedList parsedEvents) { + // Currently only DDL statements that modify the table structure are supported, so add + // custom listeners to handle these events. + listeners.add(new CustomAlterTableParserListener(parser, listeners, parsedEvents)); + } + + /** + * Returns all caught errors during tree walk. + * + * @return list of Parsing exceptions + */ + @Override + public Collection getErrors() { + return Collections.emptyList(); + } + + @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/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/utils/OracleTypeUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/utils/OracleTypeUtils.java index 91547b17b2e..8147a187bb1 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/utils/OracleTypeUtils.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/utils/OracleTypeUtils.java @@ -23,6 +23,8 @@ import io.debezium.relational.Column; +import java.util.Optional; + /** Utilities for converting from oracle types to SeaTunnel types. */ public class OracleTypeUtils { @@ -40,4 +42,33 @@ public static SeaTunnelDataType convertFromColumn(Column column) { OracleTypeConverter.INSTANCE.convert(typeDefine); return seaTunnelColumn.getDataType(); } + + public static org.apache.seatunnel.api.table.catalog.Column convertToSeaTunnelColumn( + io.debezium.relational.Column column) { + + Optional defaultValueExpression = column.defaultValueExpression(); + Object defaultValue = defaultValueExpression.orElse(null); + + BasicTypeDefine.BasicTypeDefineBuilder builder = + BasicTypeDefine.builder() + .name(column.name()) + .columnType(column.typeName()) + .dataType(column.typeName()) + .scale(column.scale().orElse(0)) + .nullable(column.isOptional()) + .defaultValue(defaultValue); + + // The default value of length in column is -1 if it is not set + if (column.length() >= 0) { + builder.length((long) column.length()).precision((long) column.length()); + } + + // TIMESTAMP or TIMESTAMP WITH TIME ZONE + // This is useful for OracleTypeConverter.convert() + if (column.typeName() != null && column.typeName().toUpperCase().startsWith("TIMESTAMP")) { + builder.scale(column.length()); + } + + return new OracleTypeConverter(false).convert(builder.build()); + } } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/OracleDdlParserTest.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/OracleDdlParserTest.java new file mode 100644 index 00000000000..54986eae603 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/parser/OracleDdlParserTest.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.seatunnel.connectors.seatunnel.cdc.oracle.source.parser; + +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.event.AlterTableAddColumnEvent; +import org.apache.seatunnel.api.table.event.AlterTableChangeColumnEvent; +import org.apache.seatunnel.api.table.event.AlterTableColumnEvent; +import org.apache.seatunnel.api.table.event.AlterTableDropColumnEvent; +import org.apache.seatunnel.api.table.event.AlterTableModifyColumnEvent; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oracle.OracleTypeConverter; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import io.debezium.relational.Tables; + +import java.util.List; + +public class OracleDdlParserTest { + private static final String PDB_NAME = "qyws_empi"; + private static final String SCHEMA_NAME = "QYWS_EMPI"; + private static final String TABLE_NAME = "STUDENTS"; + private static CustomOracleAntlrDdlParser parser; + + @BeforeAll + public static void setUp() { + parser = new CustomOracleAntlrDdlParser(TablePath.of(PDB_NAME, SCHEMA_NAME, TABLE_NAME)); + parser.setCurrentDatabase(PDB_NAME); + parser.setCurrentSchema(SCHEMA_NAME); + } + + @Test + public void testParseDDLForAddColumn() { + String ddl = + "alter table \"" + + SCHEMA_NAME + + "\".\"" + + TABLE_NAME + + "\" add (" + + "\"col21\" varchar2(20), col22 number(19));"; + parser.parse(ddl, new Tables()); + List addEvent1 = parser.getAndClearParsedEvents(); + Assertions.assertEquals(2, addEvent1.size()); + testColumn(addEvent1.get(0), "col21", "varchar2(20)", "STRING", 20 * 4L, null, true, null); + testColumn( + addEvent1.get(1), + "col22".toUpperCase(), + "number(19, 0)", + "Decimal(19, 0)", + 19L, + null, + true, + null); + + ddl = "alter table " + TABLE_NAME + " add (col23 varchar2(20) not null);"; + parser.parse(ddl, new Tables()); + List addEvent2 = parser.getAndClearParsedEvents(); + Assertions.assertEquals(1, addEvent2.size()); + testColumn( + addEvent2.get(0), + "col23".toUpperCase(), + "varchar2(20)", + "STRING", + 20 * 4L, + null, + false, + null); + + ddl = + "alter table " + + TABLE_NAME + + " add (" + + "col1 numeric(4,2),\n" + + "col2 varchar2(255) default 'debezium' not null ,\n" + + "col3 varchar2(255) default sys_context('userenv','host') not null ,\n" + + "col4 nvarchar2(255) not null,\n" + + "col5 char(4),\n" + + "col6 nchar(4),\n" + + "col7 float default '3.0' not null,\n" + + "col8 date,\n" + + "col9 timestamp(6) default sysdate,\n" + + "col10 blob,\n" + + "col11 clob,\n" + + "col12 number(1,0),\n" + + "col13 timestamp with time zone not null,\n" + + "col14 number default (sysdate-to_date('1970-01-01 08:00:00', 'yyyy-mm-dd hh24:mi:ss'))*86400000,\n" + + "col15 timestamp(9) default to_timestamp('20190101 00:00:00.000000','yyyymmdd hh24:mi:ss.ff6') not null,\n" + + "col16 date default sysdate not null);"; + parser.parse(ddl, new Tables()); + List addEvent3 = parser.getAndClearParsedEvents(); + Assertions.assertEquals(16, addEvent3.size()); + // Special default values are handled for reference: + // io.debezium.connector.oracle.OracleDefaultValueConverter.castTemporalFunctionCall + testColumn( + addEvent3.get(0), + "col1".toUpperCase(), + "number(4, 2)", + "Decimal(4, 2)", + 4L, + 2, + true, + null); + testColumn( + addEvent3.get(1), + "col2".toUpperCase(), + "varchar2(255)", + "STRING", + 255 * 4L, + null, + false, + "'debezium'"); + testColumn( + addEvent3.get(2), + "col3".toUpperCase(), + "varchar2(255)", + "STRING", + 255 * 4L, + null, + false, + "sys_context('userenv','host')"); + testColumn( + addEvent3.get(3), + "col4".toUpperCase(), + "nvarchar2(255)", + "STRING", + 255 * 2L, + null, + false, + null); + testColumn( + addEvent3.get(4), + "col5".toUpperCase(), + "char(4)", + "STRING", + 4 * 4L, + null, + true, + null); + testColumn( + addEvent3.get(5), + "col6".toUpperCase(), + "nchar(4)", + "STRING", + 4 * 2L, + null, + true, + null); + testColumn( + addEvent3.get(6), + "col7".toUpperCase(), + "float", + "Decimal(38, 18)", + 38L, + 18, + false, + "'3.0'"); + testColumn( + addEvent3.get(7), + "col8".toUpperCase(), + "date", + "TIMESTAMP", + null, + null, + true, + null); + testColumn( + addEvent3.get(8), + "col9".toUpperCase(), + "timestamp(6)", + "TIMESTAMP", + null, + 6, + true, + "sysdate"); + testColumn( + addEvent3.get(9), + "col10".toUpperCase(), + "blob", + "BYTES", + OracleTypeConverter.BYTES_4GB - 1, + null, + true, + null); + testColumn( + addEvent3.get(10), + "col11".toUpperCase(), + "clob", + "STRING", + OracleTypeConverter.BYTES_4GB - 1, + null, + true, + null); + testColumn( + addEvent3.get(11), + "col12".toUpperCase(), + "number(1, 0)", + "Decimal(1, 0)", + 1L, + null, + true, + null); + testColumn( + addEvent3.get(12), + "col13".toUpperCase(), + "timestamp with time zone(6)", + "TIMESTAMP", + null, + 6, + false, + null); + testColumn( + addEvent3.get(13), + "col14".toUpperCase(), + "number", + "Decimal(38, 0)", + 38L, + null, + true, + "(sysdate-to_date('1970-01-01 08:00:00','yyyy-mm-dd hh24:mi:ss'))*86400000"); + testColumn( + addEvent3.get(14), + "col15".toUpperCase(), + "timestamp(9)", + "TIMESTAMP", + null, + 9, + false, + "to_timestamp('20190101 00:00:00.000000','yyyymmdd hh24:mi:ss.ff6')"); + testColumn( + addEvent3.get(15), + "col16".toUpperCase(), + "date", + "TIMESTAMP", + null, + null, + false, + "sysdate"); + + ddl = + "ALTER TABLE \"" + + SCHEMA_NAME + + "\".\"" + + TABLE_NAME + + "\" ADD \"ADD_COL2\" TIMESTAMP(6) DEFAULT current_timestamp(6) NOT NULL "; + parser.parse(ddl, new Tables()); + List addEvent4 = parser.getAndClearParsedEvents(); + Assertions.assertEquals(1, addEvent4.size()); + testColumn( + addEvent4.get(0), + "ADD_COL2", + "timestamp(6)", + "TIMESTAMP", + null, + 6, + false, + "current_timestamp(6)"); + } + + @Test + public void testParseDDLForDropColumn() { + String ddl = "ALTER TABLE \"" + SCHEMA_NAME + "\".\"" + TABLE_NAME + "\" DROP (T_VARCHAR2)"; + parser.parse(ddl, new Tables()); + List dropEvent1 = parser.getAndClearParsedEvents(); + Assertions.assertEquals(1, dropEvent1.size()); + Assertions.assertEquals( + "T_VARCHAR2", ((AlterTableDropColumnEvent) dropEvent1.get(0)).getColumn()); + + ddl = "alter table " + TABLE_NAME + " drop (col22, col23);"; + parser.parse(ddl, new Tables()); + List dropEvent2 = parser.getAndClearParsedEvents(); + Assertions.assertEquals(2, dropEvent2.size()); + Assertions.assertEquals( + "col22".toUpperCase(), ((AlterTableDropColumnEvent) dropEvent2.get(0)).getColumn()); + Assertions.assertEquals( + "col23".toUpperCase(), ((AlterTableDropColumnEvent) dropEvent2.get(1)).getColumn()); + + ddl = "alter table " + TABLE_NAME + " drop (\"col22\");"; + parser.parse(ddl, new Tables()); + List dropEvent3 = parser.getAndClearParsedEvents(); + Assertions.assertEquals(1, dropEvent3.size()); + Assertions.assertEquals( + "col22", ((AlterTableDropColumnEvent) dropEvent3.get(0)).getColumn()); + } + + @Test + public void testParseDDLForRenameColumn() { + String ddl = "alter table " + TABLE_NAME + " rename column STUDENT_NAME to STUDENT_NAME1"; + parser.parse(ddl, new Tables()); + List renameEvent1 = parser.getAndClearParsedEvents(); + Assertions.assertEquals(1, renameEvent1.size()); + Assertions.assertEquals( + "STUDENT_NAME", ((AlterTableChangeColumnEvent) renameEvent1.get(0)).getOldColumn()); + Assertions.assertEquals( + "STUDENT_NAME1", + ((AlterTableChangeColumnEvent) renameEvent1.get(0)).getColumn().getName()); + + ddl = + "alter table \"" + + TABLE_NAME + + "\" rename column STUDENT_ID to STUDENT_ID1;\n" + + "alter table \"" + + TABLE_NAME + + "\" rename column CLASS_ID to CLASS_ID1\n"; + + parser.parse(ddl, new Tables()); + List renameEvent2 = parser.getAndClearParsedEvents(); + Assertions.assertEquals(2, renameEvent2.size()); + Assertions.assertEquals( + "STUDENT_ID", ((AlterTableChangeColumnEvent) renameEvent2.get(0)).getOldColumn()); + Assertions.assertEquals( + "STUDENT_ID1", + ((AlterTableChangeColumnEvent) renameEvent2.get(0)).getColumn().getName()); + Assertions.assertEquals( + "CLASS_ID", ((AlterTableChangeColumnEvent) renameEvent2.get(1)).getOldColumn()); + Assertions.assertEquals( + "CLASS_ID1", + ((AlterTableChangeColumnEvent) renameEvent2.get(1)).getColumn().getName()); + } + + @Test + public void testParseDDLForModifyColumn() { + String ddl = "ALTER TABLE " + TABLE_NAME + " MODIFY COL1 varchar2(50) not null;"; + parser.parse(ddl, new Tables()); + List modifyEvent1 = parser.getAndClearParsedEvents(); + Assertions.assertEquals(1, modifyEvent1.size()); + testColumn( + modifyEvent1.get(0), "COL1", "varchar2(50)", "STRING", 50 * 4L, null, false, null); + + ddl = "alter table " + TABLE_NAME + " modify sex char(2) default 'M' not null ;"; + parser.parse(ddl, new Tables()); + List modifyEvent2 = parser.getAndClearParsedEvents(); + Assertions.assertEquals(1, modifyEvent2.size()); + testColumn( + modifyEvent2.get(0), + "sex".toUpperCase(), + "char(2)", + "STRING", + 2 * 4L, + null, + false, + "'M'"); + ddl = + "ALTER TABLE \"" + + SCHEMA_NAME + + "\".\"" + + TABLE_NAME + + "\" MODIFY (ID NUMBER(*,0) NULL);"; + parser.parse(ddl, new Tables()); + List modifyEvent3 = parser.getAndClearParsedEvents(); + Assertions.assertEquals(1, modifyEvent3.size()); + testColumn( + modifyEvent3.get(0), + "ID", + "number(38, 0)", + "Decimal(38, 0)", + 38L, + null, + true, + null); + } + + private void testColumn( + AlterTableColumnEvent alterTableColumnEvent, + String columnName, + String sourceType, + String dataType, + Long columnLength, + Integer scale, + boolean isNullable, + Object defaultValue) { + Column column; + switch (alterTableColumnEvent.getEventType()) { + case SCHEMA_CHANGE_ADD_COLUMN: + column = ((AlterTableAddColumnEvent) alterTableColumnEvent).getColumn(); + break; + case SCHEMA_CHANGE_MODIFY_COLUMN: + column = ((AlterTableModifyColumnEvent) alterTableColumnEvent).getColumn(); + break; + default: + throw new UnsupportedOperationException( + "Unsupported method named getColumn() for the AlterTableColumnEvent: " + + alterTableColumnEvent.getEventType().name()); + } + Assertions.assertEquals(columnName, column.getName()); + Assertions.assertEquals(sourceType.toUpperCase(), column.getSourceType()); + Assertions.assertEquals(dataType, column.getDataType().toString()); + Assertions.assertEquals(columnLength, column.getColumnLength()); + Assertions.assertEquals(scale, column.getScale()); + Assertions.assertEquals(isNullable, column.isNullable()); + Assertions.assertEquals(defaultValue, column.getDefaultValue()); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java index 2fc0fe8dca8..c1ba23b64e9 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java @@ -533,7 +533,7 @@ default void applySchemaChange( buildAlterTableSql( event.getSourceDialectName(), addColumn.getSourceType(), - AlterType.ADD.name(), + alterTableAddOperation(), addColumn, tableIdentifierWithQuoted, StringUtils.EMPTY); @@ -553,7 +553,7 @@ default void applySchemaChange( buildAlterTableSql( event.getSourceDialectName(), changeColumn.getSourceType(), - AlterType.CHANGE.name(), + alterTableChangeOperation(), changeColumn, tableIdentifierWithQuoted, oldColumnName); @@ -573,7 +573,7 @@ default void applySchemaChange( buildAlterTableSql( event.getSourceDialectName(), modifyColumn.getSourceType(), - AlterType.MODIFY.name(), + alterTableModifyOperation(), modifyColumn, tableIdentifierWithQuoted, StringUtils.EMPTY); @@ -593,7 +593,7 @@ default void applySchemaChange( buildAlterTableSql( event.getSourceDialectName(), null, - AlterType.DROP.name(), + alterTableDropOperation(), null, tableIdentifierWithQuoted, dropColumn); @@ -621,10 +621,11 @@ default String buildAlterTableSql( Column newColumn, String tableName, String oldColumnName) { - if (StringUtils.equals(alterOperation, AlterType.DROP.name())) { + if (StringUtils.equals(alterOperation, alterTableDropOperation())) { return String.format( "ALTER TABLE %s drop column %s", tableName, quoteIdentifier(oldColumnName)); } + TypeConverter typeConverter = ConverterLoader.loadTypeConverter(dialectName()); BasicTypeDefine typeBasicTypeDefine = (BasicTypeDefine) typeConverter.reconvert(newColumn); @@ -638,10 +639,14 @@ default String buildAlterTableSql( newColumn, oldColumnName, typeBasicTypeDefine.getColumnType()); + // Only decorate with default value when source dialect is same as sink dialect + // Todo Support for cross-database default values for ddl statements + if (sourceDialectName.equals(dialectName())) { + basicSql = decorateWithDefaultValue(basicSql, typeBasicTypeDefine); + } basicSql = decorateWithNullable(basicSql, typeBasicTypeDefine); - basicSql = decorateWithDefaultValue(basicSql, typeBasicTypeDefine); - basicSql = decorateWithComment(basicSql, typeBasicTypeDefine); - return basicSql + ";"; + basicSql = decorateWithComment(tableName, basicSql, typeBasicTypeDefine); + return dialectName().equals(DatabaseIdentifier.ORACLE) ? basicSql : basicSql + ";"; } /** @@ -685,7 +690,7 @@ default String decorateWithColumnNameAndType( StringBuilder sql = new StringBuilder(basicSql); String oldColumnNameWithQuoted = quoteIdentifier(oldColumnName); String newColumnNameWithQuoted = quoteIdentifier(newColumn.getName()); - if (alterOperation.equals(AlterType.CHANGE.name())) { + if (alterOperation.equals(alterTableChangeOperation())) { sql.append(oldColumnNameWithQuoted) .append(StringUtils.SPACE) .append(newColumnNameWithQuoted) @@ -711,7 +716,7 @@ default String decorateWithColumnNameAndType( */ default String decorateWithNullable(String basicSql, BasicTypeDefine typeBasicTypeDefine) { StringBuilder sql = new StringBuilder(basicSql); - if (typeBasicTypeDefine.isNullable()) { + if (typeBasicTypeDefine.isNullable() && !dialectName().equals(DatabaseIdentifier.ORACLE)) { sql.append("NULL "); } else { sql.append("NOT NULL "); @@ -743,11 +748,13 @@ && needsQuotesWithDefaultValue(typeBasicTypeDefine.getColumnType()) /** * decorate with comment * + * @param tableName table name with quoted * @param basicSql alter table sql for sink table * @param typeBasicTypeDefine type basic type define of new column * @return alter table sql with comment for sink table */ - default String decorateWithComment(String basicSql, BasicTypeDefine typeBasicTypeDefine) { + default String decorateWithComment( + String tableName, String basicSql, BasicTypeDefine typeBasicTypeDefine) { String comment = typeBasicTypeDefine.getComment(); StringBuilder sql = new StringBuilder(basicSql); if (StringUtils.isNotBlank(comment)) { @@ -792,4 +799,40 @@ enum AlterType { MODIFY, CHANGE } + + /** + * Get alter table add operation + * + * @return alter table add operation + */ + default String alterTableAddOperation() { + return AlterType.ADD.name(); + } + + /** + * Get alter table change operation + * + * @return alter table change operation + */ + default String alterTableChangeOperation() { + return AlterType.CHANGE.name(); + } + + /** + * Get alter table modify operation + * + * @return alter table modify operation + */ + default String alterTableModifyOperation() { + return AlterType.MODIFY.name(); + } + + /** + * Get alter table drop operation + * + * @return alter table drop operation + */ + default String alterTableDropOperation() { + return AlterType.DROP.name(); + } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlDialect.java index 22431b0d96f..163c0304459 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlDialect.java @@ -228,12 +228,13 @@ public Long approximateRowCntStatement(Connection connection, JdbcSourceTable ta } @Override - public String decorateWithComment(String basicSql, BasicTypeDefine typeBasicTypeDefine) { + public String decorateWithComment( + String tableName, String basicSql, BasicTypeDefine typeBasicTypeDefine) { MysqlType nativeType = (MysqlType) typeBasicTypeDefine.getNativeType(); if (NOT_SUPPORTED_DEFAULT_VALUES.contains(nativeType)) { return basicSql; } - return JdbcDialect.super.decorateWithComment(basicSql, typeBasicTypeDefine); + return JdbcDialect.super.decorateWithComment(tableName, basicSql, typeBasicTypeDefine); } @Override diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMysqlDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMysqlDialect.java index 1c5d7734fb2..1824d6c76ab 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMysqlDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMysqlDialect.java @@ -231,12 +231,13 @@ public Long approximateRowCntStatement(Connection connection, JdbcSourceTable ta } @Override - public String decorateWithComment(String basicSql, BasicTypeDefine typeBasicTypeDefine) { + public String decorateWithComment( + String tableName, String basicSql, BasicTypeDefine typeBasicTypeDefine) { OceanBaseMysqlType nativeType = (OceanBaseMysqlType) typeBasicTypeDefine.getNativeType(); if (NOT_SUPPORTED_DEFAULT_VALUES.contains(nativeType)) { return basicSql; } - return JdbcDialect.super.decorateWithComment(basicSql, typeBasicTypeDefine); + return JdbcDialect.super.decorateWithComment(tableName, basicSql, typeBasicTypeDefine); } @Override diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleDialect.java index b6a35dba0c1..6be021050d9 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleDialect.java @@ -17,7 +17,9 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oracle; +import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.converter.BasicTypeDefine; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.converter.JdbcRowConverter; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; @@ -326,4 +328,48 @@ public Object[] sampleDataFromColumn( } } } + + @Override + public String buildAlterTableSql( + String sourceDialectName, + String sourceColumnType, + String alterOperation, + Column newColumn, + String tableName, + String oldColumnName) { + // Because the oracle rename operation which no need to specify the column type, so we need + // to override the method to handle this case. + if (StringUtils.equals(alterOperation, alterTableChangeOperation())) { + return String.format( + "ALTER TABLE %s RENAME COLUMN %s TO %s", + tableName, oldColumnName, newColumn.getName()); + } + return JdbcDialect.super.buildAlterTableSql( + sourceDialectName, + sourceColumnType, + alterOperation, + newColumn, + tableName, + oldColumnName); + } + + @Override + public String alterTableChangeOperation() { + return "RENAME"; + } + + @Override + public String decorateWithComment( + String tableName, String basicSql, BasicTypeDefine typeBasicTypeDefine) { + String comment = typeBasicTypeDefine.getComment(); + StringBuilder sql = new StringBuilder(basicSql); + if (StringUtils.isNotBlank(comment)) { + String commentSql = + String.format( + "COMMENT ON COLUMN %s.%s IS '%s'", + tableName, quoteIdentifier(typeBasicTypeDefine.getName()), comment); + sql.append(";\n").append(commentSql); + } + return sql.toString(); + } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/AbstractJdbcSinkWriter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/AbstractJdbcSinkWriter.java index f894999a42f..f8f3fa6bbc5 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/AbstractJdbcSinkWriter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/AbstractJdbcSinkWriter.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.sink; +import org.apache.seatunnel.api.event.EventType; import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter; import org.apache.seatunnel.api.table.catalog.Column; @@ -96,14 +97,15 @@ protected void processSchemaChangeEvent(AlterTableColumnEvent event) throws IOEx break; case SCHEMA_CHANGE_MODIFY_COLUMN: Column modifyColumn = ((AlterTableModifyColumnEvent) event).getColumn(); - replaceColumnByIndex(columns, modifyColumn.getName(), modifyColumn); + replaceColumnByIndex( + event.getEventType(), columns, modifyColumn.getName(), modifyColumn); break; case SCHEMA_CHANGE_CHANGE_COLUMN: AlterTableChangeColumnEvent alterTableChangeColumnEvent = (AlterTableChangeColumnEvent) event; Column changeColumn = alterTableChangeColumnEvent.getColumn(); String oldColumnName = alterTableChangeColumnEvent.getOldColumn(); - replaceColumnByIndex(columns, oldColumnName, changeColumn); + replaceColumnByIndex(event.getEventType(), columns, oldColumnName, changeColumn); break; default: throw new SeaTunnelException( @@ -132,7 +134,7 @@ protected void reOpenOutputFormat(AlterTableColumnEvent event) throws IOExceptio } protected void replaceColumnByIndex( - List columns, String oldColumnName, Column newColumn) { + EventType eventType, List columns, String oldColumnName, Column newColumn) { for (int i = 0; i < columns.size(); i++) { if (columns.get(i).getName().equalsIgnoreCase(oldColumnName)) { columns.set(i, newColumn); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkWriter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkWriter.java index 3f43b2088d0..41dd41ff9e9 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkWriter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkWriter.java @@ -17,7 +17,9 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.sink; +import org.apache.seatunnel.api.event.EventType; import org.apache.seatunnel.api.sink.MultiTableResourceManager; +import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.catalog.TableSchema; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -28,6 +30,7 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.JdbcOutputFormatBuilder; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.SimpleJdbcConnectionPoolProviderProxy; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oracle.OracleDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.state.JdbcSinkState; import org.apache.seatunnel.connectors.seatunnel.jdbc.state.XidInfo; @@ -40,6 +43,8 @@ import java.util.List; import java.util.Optional; +import static org.apache.seatunnel.api.event.EventType.SCHEMA_CHANGE_CHANGE_COLUMN; + @Slf4j public class JdbcSinkWriter extends AbstractJdbcSinkWriter { private final Integer primaryKeyIndex; @@ -162,4 +167,22 @@ public void close() throws IOException { outputFormat.close(); } } + + @Override + protected void replaceColumnByIndex( + EventType eventType, List columns, String oldColumnName, Column newColumn) { + // The operation of renaming a column in Oracle is only supported to modify the column name, + // so we just modify the column name directly. + if (eventType.equals(SCHEMA_CHANGE_CHANGE_COLUMN) && dialect instanceof OracleDialect) { + for (int i = 0; i < columns.size(); i++) { + Column column = columns.get(i); + if (column.getName().equalsIgnoreCase(oldColumnName)) { + column = column.rename(newColumn.getName()); + columns.set(i, column); + } + } + return; + } + super.replaceColumnByIndex(eventType, columns, oldColumnName, newColumn); + } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/pom.xml index d3ee2be7415..01c9b2c7563 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/pom.xml @@ -30,6 +30,18 @@ connector-cdc-oracle-e2e SeaTunnel : E2E : Connector V2 : CDC Oracle + + + + org.apache.seatunnel + connector-jdbc + ${project.version} + pom + import + + + + @@ -71,6 +83,20 @@ ${project.version} test + + + org.apache.seatunnel + connector-cdc-mysql + ${project.version} + test-jar + test + + + + mysql + mysql-connector-java + test + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/AbstractOracleCDCIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/AbstractOracleCDCIT.java new file mode 100644 index 00000000000..95e42ad06d1 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/AbstractOracleCDCIT.java @@ -0,0 +1,154 @@ +/* + * 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.seatunnel.connectors.seatunnel.cdc.oracle; + +import org.apache.seatunnel.e2e.common.TestSuiteBase; + +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.utility.DockerLoggerFactory; + +import lombok.extern.slf4j.Slf4j; + +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.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertNotNull; + +@Slf4j +public class AbstractOracleCDCIT extends TestSuiteBase { + + protected static final String ORACLE_IMAGE = "goodboy008/oracle-19.3.0-ee:non-cdb"; + + protected static final String HOST = "oracle-host"; + + protected static final Integer ORACLE_PORT = 1521; + + protected static final String CONNECTOR_USER = "dbzuser"; + + protected static final String CONNECTOR_PWD = "dbz"; + + protected static final String SCHEMA_USER = "debezium"; + + protected static final String SCHEMA_PWD = "dbz"; + + public static final String ADMIN_USER = "sys as sysdba"; + + public static final String ADMIN_PWD = "top_secret"; + + protected static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)--.*$"); + + protected static final String SCEHMA_NAME = "DEBEZIUM"; + + protected static final String SOURCE_TABLE1 = "FULL_TYPES"; + + protected static final String SOURCE_TABLE2 = "FULL_TYPES2"; + + protected static final OracleContainer ORACLE_CONTAINER = + new OracleContainer(ORACLE_IMAGE) + .withUsername(CONNECTOR_USER) + .withPassword(CONNECTOR_PWD) + .withDatabaseName("ORCLCDB") + .withNetwork(NETWORK) + .withNetworkAliases(HOST) + .withExposedPorts(ORACLE_PORT) + .withLogConsumer( + new Slf4jLogConsumer( + DockerLoggerFactory.getLogger("oracle-docker-image"))); + + protected String oracleDriverUrl() { + return "https://repo1.maven.org/maven2/com/oracle/database/jdbc/ojdbc8/12.2.0.1/ojdbc8-12.2.0.1.jar"; + } + + static { + System.setProperty("oracle.jdbc.timezoneAsRegion", "false"); + } + + protected static void createAndInitialize(String sqlFile, String username, String password) + throws Exception { + final String ddlFile = String.format("ddl/%s.sql", sqlFile); + final URL ddlTestFile = OracleCDCIT.class.getClassLoader().getResource(ddlFile); + assertNotNull("Cannot locate " + ddlFile, ddlTestFile); + try (Connection connection = + getJdbcConnection(ORACLE_CONTAINER.getJdbcUrl(), username, password); + Statement statement = connection.createStatement()) { + + final List statements = + Arrays.stream( + Files.readAllLines(Paths.get(ddlTestFile.toURI())).stream() + .map(String::trim) + .filter(x -> !x.startsWith("--") && !x.isEmpty()) + .map( + x -> { + final Matcher m = + COMMENT_PATTERN.matcher(x); + return m.matches() ? m.group(1) : x; + }) + .collect(Collectors.joining("\n")) + .split(";")) + .collect(Collectors.toList()); + + for (String stmt : statements) { + statement.execute(stmt); + } + } + } + + protected static Connection getJdbcConnection(String jdbcUrl, String username, String password) + throws SQLException { + return DriverManager.getConnection(jdbcUrl, username, password); + } + + protected List> query(String jdbcUrl, String sql) { + try (Connection connection = getJdbcConnection(jdbcUrl, CONNECTOR_USER, CONNECTOR_PWD)) { + ResultSet resultSet = connection.createStatement().executeQuery(sql); + List> result = new ArrayList<>(); + int columnCount = resultSet.getMetaData().getColumnCount(); + while (resultSet.next()) { + ArrayList objects = new ArrayList<>(); + for (int i = 1; i <= columnCount; i++) { + objects.add(resultSet.getObject(i)); + } + log.debug(String.format("Print Oracle-CDC query, sql: %s, data: %s", sql, objects)); + result.add(objects); + } + return result; + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + protected static void dropTable(String jdbcUrl, String tableName) { + try (Connection connection = getJdbcConnection(jdbcUrl, CONNECTOR_USER, CONNECTOR_PWD)) { + connection.createStatement().execute(String.format("DROP TABLE %s", tableName)); + } catch (SQLException e) { + throw new RuntimeException(e); + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/OracleCDCIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/OracleCDCIT.java index 86282b7ff03..d09b502d8fa 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/OracleCDCIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/OracleCDCIT.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.connectors.seatunnel.cdc.oracle; import org.apache.seatunnel.e2e.common.TestResource; -import org.apache.seatunnel.e2e.common.TestSuiteBase; import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; import org.apache.seatunnel.e2e.common.container.EngineType; import org.apache.seatunnel.e2e.common.container.TestContainer; @@ -30,89 +29,38 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.TestTemplate; import org.testcontainers.containers.Container; -import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.lifecycle.Startables; -import org.testcontainers.utility.DockerLoggerFactory; import com.google.common.collect.Lists; import lombok.extern.slf4j.Slf4j; import java.io.IOException; -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.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import java.util.stream.Collectors; import java.util.stream.Stream; import static org.awaitility.Awaitility.await; -import static org.junit.Assert.assertNotNull; @Slf4j @DisabledOnContainer( value = {}, type = {EngineType.SPARK}, - disabledReason = - "Currently SPARK do not support cdc,Flink is prone to time out, temporarily disable") -public class OracleCDCIT extends TestSuiteBase implements TestResource { + disabledReason = "Currently SPARK do not support cdc") +public class OracleCDCIT extends AbstractOracleCDCIT implements TestResource { - private static final String ORACLE_IMAGE = "goodboy008/oracle-19.3.0-ee:non-cdb"; - - private static final String HOST = "oracle-host"; - - private static final Integer ORACLE_PORT = 1521; - - static { - System.setProperty("oracle.jdbc.timezoneAsRegion", "false"); - } - - public static final String CONNECTOR_USER = "dbzuser"; - - public static final String CONNECTOR_PWD = "dbz"; - - public static final String SCHEMA_USER = "debezium"; - - public static final String SCHEMA_PWD = "dbz"; - - public static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)--.*$"); - - private static final String DATABASE = "DEBEZIUM"; - private static final String SOURCE_TABLE1 = "FULL_TYPES"; - private static final String SOURCE_TABLE2 = "FULL_TYPES2"; private static final String SOURCE_TABLE_NO_PRIMARY_KEY = "FULL_TYPES_NO_PRIMARY_KEY"; private static final String SINK_TABLE1 = "SINK_FULL_TYPES"; private static final String SINK_TABLE2 = "SINK_FULL_TYPES2"; - private static final String SOURCE_SQL_TEMPLATE = "select * from %s.%s ORDER BY ID"; - public static final OracleContainer ORACLE_CONTAINER = - new OracleContainer(ORACLE_IMAGE) - .withUsername(CONNECTOR_USER) - .withPassword(CONNECTOR_PWD) - .withDatabaseName("ORCLCDB") - .withNetwork(NETWORK) - .withNetworkAliases(HOST) - .withExposedPorts(ORACLE_PORT) - .withLogConsumer( - new Slf4jLogConsumer( - DockerLoggerFactory.getLogger("oracle-docker-image"))); - - private String driverUrl() { - return "https://repo1.maven.org/maven2/com/oracle/database/jdbc/ojdbc8/12.2.0.1/ojdbc8-12.2.0.1.jar"; - } - @TestContainerExtension protected final ContainerExtendedFactory extendedFactory = container -> { @@ -121,7 +69,7 @@ private String driverUrl() { "bash", "-c", "mkdir -p /tmp/seatunnel/plugins/Oracle-CDC/lib && cd /tmp/seatunnel/plugins/Oracle-CDC/lib && wget " - + driverUrl()); + + oracleDriverUrl()); Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); }; @@ -130,10 +78,16 @@ private String driverUrl() { public void startUp() throws Exception { ORACLE_CONTAINER.setPortBindings( Lists.newArrayList(String.format("%s:%s", ORACLE_PORT, ORACLE_PORT))); - log.info("Starting containers..."); + log.info("Starting Oracle containers..."); Startables.deepStart(Stream.of(ORACLE_CONTAINER)).join(); - log.info("Containers are started."); - createAndInitialize(ORACLE_CONTAINER, "column_type_test"); + log.info("Oracle containers are started."); + createAndInitialize("column_type_test", ADMIN_USER, ADMIN_PWD); + } + + @AfterAll + @Override + public void tearDown() throws Exception { + ORACLE_CONTAINER.stop(); } @TestTemplate @@ -154,12 +108,12 @@ public void testOracleCdcCheckDataE2eForSkipAnalysis(TestContainer container) th private void checkDataForTheJob( TestContainer container, String jobConfPath, Boolean skipAnalysis) throws Exception { - clearTable(DATABASE, SOURCE_TABLE1); - clearTable(DATABASE, SOURCE_TABLE2); - clearTable(DATABASE, SINK_TABLE1); - clearTable(DATABASE, SINK_TABLE2); + clearTable(SCEHMA_NAME, SOURCE_TABLE1); + clearTable(SCEHMA_NAME, SOURCE_TABLE2); + clearTable(SCEHMA_NAME, SINK_TABLE1); + clearTable(SCEHMA_NAME, SINK_TABLE2); - insertSourceTable(DATABASE, SOURCE_TABLE1); + insertSourceTable(SCEHMA_NAME, SOURCE_TABLE1); if (skipAnalysis) { // analyzeTable before execute job @@ -169,7 +123,7 @@ private void checkDataForTheJob( + "\"DEBEZIUM\".\"FULL_TYPES\" " + "compute statistics for table"); log.info("analyze table {}", analyzeTable); - try (Connection connection = testConnection(ORACLE_CONTAINER); + try (Connection connection = getJdbcConnection(ORACLE_CONTAINER); Statement statement = connection.createStatement()) { statement.execute(analyzeTable); } @@ -191,30 +145,30 @@ private void checkDataForTheJob( .untilAsserted( () -> { Assertions.assertIterableEquals( - querySql(getSourceQuerySQL(DATABASE, SOURCE_TABLE1)), - querySql(getSourceQuerySQL(DATABASE, SINK_TABLE1))); + querySql(getSourceQuerySQL(SCEHMA_NAME, SOURCE_TABLE1)), + querySql(getSourceQuerySQL(SCEHMA_NAME, SINK_TABLE1))); }); // insert update delete - updateSourceTable(DATABASE, SOURCE_TABLE1); + updateSourceTable(SCEHMA_NAME, SOURCE_TABLE1); // stream stage await().atMost(600000, TimeUnit.MILLISECONDS) .untilAsserted( () -> { Assertions.assertIterableEquals( - querySql(getSourceQuerySQL(DATABASE, SOURCE_TABLE1)), - querySql(getSourceQuerySQL(DATABASE, SINK_TABLE1))); + querySql(getSourceQuerySQL(SCEHMA_NAME, SOURCE_TABLE1)), + querySql(getSourceQuerySQL(SCEHMA_NAME, SINK_TABLE1))); }); } @TestTemplate public void testOracleCdcCheckDataWithNoPrimaryKey(TestContainer container) throws Exception { - clearTable(DATABASE, SOURCE_TABLE_NO_PRIMARY_KEY); - clearTable(DATABASE, SINK_TABLE1); + clearTable(SCEHMA_NAME, SOURCE_TABLE_NO_PRIMARY_KEY); + clearTable(SCEHMA_NAME, SINK_TABLE1); - insertSourceTable(DATABASE, SOURCE_TABLE_NO_PRIMARY_KEY); + insertSourceTable(SCEHMA_NAME, SOURCE_TABLE_NO_PRIMARY_KEY); CompletableFuture.supplyAsync( () -> { @@ -234,12 +188,12 @@ public void testOracleCdcCheckDataWithNoPrimaryKey(TestContainer container) thro Assertions.assertIterableEquals( querySql( getSourceQuerySQL( - DATABASE, SOURCE_TABLE_NO_PRIMARY_KEY)), - querySql(getSourceQuerySQL(DATABASE, SINK_TABLE1))); + SCEHMA_NAME, SOURCE_TABLE_NO_PRIMARY_KEY)), + querySql(getSourceQuerySQL(SCEHMA_NAME, SINK_TABLE1))); }); // insert update delete - updateSourceTable(DATABASE, SOURCE_TABLE_NO_PRIMARY_KEY); + updateSourceTable(SCEHMA_NAME, SOURCE_TABLE_NO_PRIMARY_KEY); // stream stage await().atMost(600000, TimeUnit.MILLISECONDS) @@ -248,8 +202,8 @@ public void testOracleCdcCheckDataWithNoPrimaryKey(TestContainer container) thro Assertions.assertIterableEquals( querySql( getSourceQuerySQL( - DATABASE, SOURCE_TABLE_NO_PRIMARY_KEY)), - querySql(getSourceQuerySQL(DATABASE, SINK_TABLE1))); + SCEHMA_NAME, SOURCE_TABLE_NO_PRIMARY_KEY)), + querySql(getSourceQuerySQL(SCEHMA_NAME, SINK_TABLE1))); }); } @@ -257,10 +211,10 @@ public void testOracleCdcCheckDataWithNoPrimaryKey(TestContainer container) thro public void testOracleCdcCheckDataWithCustomPrimaryKey(TestContainer container) throws Exception { - clearTable(DATABASE, SOURCE_TABLE_NO_PRIMARY_KEY); - clearTable(DATABASE, SINK_TABLE1); + clearTable(SCEHMA_NAME, SOURCE_TABLE_NO_PRIMARY_KEY); + clearTable(SCEHMA_NAME, SINK_TABLE1); - insertSourceTable(DATABASE, SOURCE_TABLE_NO_PRIMARY_KEY); + insertSourceTable(SCEHMA_NAME, SOURCE_TABLE_NO_PRIMARY_KEY); CompletableFuture.supplyAsync( () -> { @@ -280,12 +234,12 @@ public void testOracleCdcCheckDataWithCustomPrimaryKey(TestContainer container) Assertions.assertIterableEquals( querySql( getSourceQuerySQL( - DATABASE, SOURCE_TABLE_NO_PRIMARY_KEY)), - querySql(getSourceQuerySQL(DATABASE, SINK_TABLE1))); + SCEHMA_NAME, SOURCE_TABLE_NO_PRIMARY_KEY)), + querySql(getSourceQuerySQL(SCEHMA_NAME, SINK_TABLE1))); }); // insert update delete - updateSourceTable(DATABASE, SOURCE_TABLE_NO_PRIMARY_KEY); + updateSourceTable(SCEHMA_NAME, SOURCE_TABLE_NO_PRIMARY_KEY); // stream stage await().atMost(600000, TimeUnit.MILLISECONDS) @@ -294,8 +248,8 @@ public void testOracleCdcCheckDataWithCustomPrimaryKey(TestContainer container) Assertions.assertIterableEquals( querySql( getSourceQuerySQL( - DATABASE, SOURCE_TABLE_NO_PRIMARY_KEY)), - querySql(getSourceQuerySQL(DATABASE, SINK_TABLE1))); + SCEHMA_NAME, SOURCE_TABLE_NO_PRIMARY_KEY)), + querySql(getSourceQuerySQL(SCEHMA_NAME, SINK_TABLE1))); }); } @@ -307,13 +261,13 @@ public void testOracleCdcCheckDataWithCustomPrimaryKey(TestContainer container) public void testOracleCdcMultiTableE2e(TestContainer container) throws IOException, InterruptedException { - clearTable(DATABASE, SOURCE_TABLE1); - clearTable(DATABASE, SOURCE_TABLE2); - clearTable(DATABASE, SINK_TABLE1); - clearTable(DATABASE, SINK_TABLE2); + clearTable(SCEHMA_NAME, SOURCE_TABLE1); + clearTable(SCEHMA_NAME, SOURCE_TABLE2); + clearTable(SCEHMA_NAME, SINK_TABLE1); + clearTable(SCEHMA_NAME, SINK_TABLE2); - insertSourceTable(DATABASE, SOURCE_TABLE1); - insertSourceTable(DATABASE, SOURCE_TABLE2); + insertSourceTable(SCEHMA_NAME, SOURCE_TABLE1); + insertSourceTable(SCEHMA_NAME, SOURCE_TABLE2); CompletableFuture.supplyAsync( () -> { @@ -336,21 +290,24 @@ public void testOracleCdcMultiTableE2e(TestContainer container) Assertions.assertIterableEquals( querySql( getSourceQuerySQL( - DATABASE, SOURCE_TABLE1)), + SCEHMA_NAME, + SOURCE_TABLE1)), querySql( getSourceQuerySQL( - DATABASE, SINK_TABLE1))), + SCEHMA_NAME, SINK_TABLE1))), () -> Assertions.assertIterableEquals( querySql( getSourceQuerySQL( - DATABASE, SOURCE_TABLE2)), + SCEHMA_NAME, + SOURCE_TABLE2)), querySql( getSourceQuerySQL( - DATABASE, SINK_TABLE2))))); + SCEHMA_NAME, + SINK_TABLE2))))); - updateSourceTable(DATABASE, SOURCE_TABLE1); - updateSourceTable(DATABASE, SOURCE_TABLE2); + updateSourceTable(SCEHMA_NAME, SOURCE_TABLE1); + updateSourceTable(SCEHMA_NAME, SOURCE_TABLE2); // stream stage await().atMost(600000, TimeUnit.MILLISECONDS) @@ -361,18 +318,21 @@ public void testOracleCdcMultiTableE2e(TestContainer container) Assertions.assertIterableEquals( querySql( getSourceQuerySQL( - DATABASE, SOURCE_TABLE1)), + SCEHMA_NAME, + SOURCE_TABLE1)), querySql( getSourceQuerySQL( - DATABASE, SINK_TABLE1))), + SCEHMA_NAME, SINK_TABLE1))), () -> Assertions.assertIterableEquals( querySql( getSourceQuerySQL( - DATABASE, SOURCE_TABLE2)), + SCEHMA_NAME, + SOURCE_TABLE2)), querySql( getSourceQuerySQL( - DATABASE, SINK_TABLE2))))); + SCEHMA_NAME, + SINK_TABLE2))))); } @TestTemplate @@ -383,13 +343,13 @@ public void testOracleCdcMultiTableE2e(TestContainer container) public void testMultiTableWithRestore(TestContainer container) throws IOException, InterruptedException { - clearTable(DATABASE, SOURCE_TABLE1); - clearTable(DATABASE, SOURCE_TABLE2); - clearTable(DATABASE, SINK_TABLE1); - clearTable(DATABASE, SINK_TABLE2); + clearTable(SCEHMA_NAME, SOURCE_TABLE1); + clearTable(SCEHMA_NAME, SOURCE_TABLE2); + clearTable(SCEHMA_NAME, SINK_TABLE1); + clearTable(SCEHMA_NAME, SINK_TABLE2); - insertSourceTable(DATABASE, SOURCE_TABLE1); - insertSourceTable(DATABASE, SOURCE_TABLE2); + insertSourceTable(SCEHMA_NAME, SOURCE_TABLE1); + insertSourceTable(SCEHMA_NAME, SOURCE_TABLE2); Long jobId = JobIdGenerator.newJobId(); CompletableFuture.supplyAsync( @@ -413,13 +373,15 @@ public void testMultiTableWithRestore(TestContainer container) Assertions.assertIterableEquals( querySql( getSourceQuerySQL( - DATABASE, SOURCE_TABLE1)), + SCEHMA_NAME, + SOURCE_TABLE1)), querySql( getSourceQuerySQL( - DATABASE, SINK_TABLE1))))); + SCEHMA_NAME, + SINK_TABLE1))))); // insert update delete - updateSourceTable(DATABASE, SOURCE_TABLE1); + updateSourceTable(SCEHMA_NAME, SOURCE_TABLE1); // stream stage await().atMost(600000, TimeUnit.MILLISECONDS) @@ -430,10 +392,12 @@ public void testMultiTableWithRestore(TestContainer container) Assertions.assertIterableEquals( querySql( getSourceQuerySQL( - DATABASE, SOURCE_TABLE1)), + SCEHMA_NAME, + SOURCE_TABLE1)), querySql( getSourceQuerySQL( - DATABASE, SINK_TABLE1))))); + SCEHMA_NAME, + SINK_TABLE1))))); Assertions.assertEquals(0, container.savepointJob(String.valueOf(jobId)).getExitCode()); @@ -460,20 +424,23 @@ public void testMultiTableWithRestore(TestContainer container) Assertions.assertIterableEquals( querySql( getSourceQuerySQL( - DATABASE, SOURCE_TABLE1)), + SCEHMA_NAME, + SOURCE_TABLE1)), querySql( getSourceQuerySQL( - DATABASE, SINK_TABLE1))), + SCEHMA_NAME, SINK_TABLE1))), () -> Assertions.assertIterableEquals( querySql( getSourceQuerySQL( - DATABASE, SOURCE_TABLE2)), + SCEHMA_NAME, + SOURCE_TABLE2)), querySql( getSourceQuerySQL( - DATABASE, SINK_TABLE2))))); + SCEHMA_NAME, + SINK_TABLE2))))); - updateSourceTable(DATABASE, SOURCE_TABLE2); + updateSourceTable(SCEHMA_NAME, SOURCE_TABLE2); // stream stage await().atMost(600000, TimeUnit.MILLISECONDS) @@ -484,18 +451,21 @@ public void testMultiTableWithRestore(TestContainer container) Assertions.assertIterableEquals( querySql( getSourceQuerySQL( - DATABASE, SOURCE_TABLE1)), + SCEHMA_NAME, + SOURCE_TABLE1)), querySql( getSourceQuerySQL( - DATABASE, SINK_TABLE1))), + SCEHMA_NAME, SINK_TABLE1))), () -> Assertions.assertIterableEquals( querySql( getSourceQuerySQL( - DATABASE, SOURCE_TABLE2)), + SCEHMA_NAME, + SOURCE_TABLE2)), querySql( getSourceQuerySQL( - DATABASE, SINK_TABLE2))))); + SCEHMA_NAME, + SINK_TABLE2))))); log.info("****************** container logs start ******************"); String containerLogs = container.getServerLogs(); @@ -503,43 +473,10 @@ public void testMultiTableWithRestore(TestContainer container) Assertions.assertFalse(containerLogs.contains("ERROR")); log.info("****************** container logs end ******************"); - clearTable(DATABASE, SOURCE_TABLE1); - clearTable(DATABASE, SOURCE_TABLE2); - clearTable(DATABASE, SINK_TABLE1); - clearTable(DATABASE, SINK_TABLE2); - } - - public static void createAndInitialize(OracleContainer oracleContainer, String sqlFile) - throws Exception { - final String ddlFile = String.format("ddl/%s.sql", sqlFile); - final URL ddlTestFile = OracleCDCIT.class.getClassLoader().getResource(ddlFile); - assertNotNull("Cannot locate " + ddlFile, ddlTestFile); - try (Connection connection = testConnection(oracleContainer); - Statement statement = connection.createStatement()) { - - final List statements = - Arrays.stream( - Files.readAllLines(Paths.get(ddlTestFile.toURI())).stream() - .map(String::trim) - .filter(x -> !x.startsWith("--") && !x.isEmpty()) - .map( - x -> { - final Matcher m = - COMMENT_PATTERN.matcher(x); - return m.matches() ? m.group(1) : x; - }) - .collect(Collectors.joining("\n")) - .split(";")) - .collect(Collectors.toList()); - - for (String stmt : statements) { - statement.execute(stmt); - } - } - } - - public static Connection testConnection(OracleContainer oracleContainer) throws SQLException { - return DriverManager.getConnection(oracleContainer.getJdbcUrl(), SCHEMA_USER, SCHEMA_PWD); + clearTable(SCEHMA_NAME, SOURCE_TABLE1); + clearTable(SCEHMA_NAME, SOURCE_TABLE2); + clearTable(SCEHMA_NAME, SINK_TABLE1); + clearTable(SCEHMA_NAME, SINK_TABLE2); } private List> querySql(String sql) { @@ -637,10 +574,4 @@ private void updateSourceTable(String database, String tableName) { private void clearTable(String database, String tableName) { executeSql("truncate table " + database + "." + tableName, SCHEMA_USER, SCHEMA_PWD); } - - @AfterAll - @Override - public void tearDown() throws Exception { - ORACLE_CONTAINER.stop(); - } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/OracleCDCWithSchemaChangeIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/OracleCDCWithSchemaChangeIT.java new file mode 100644 index 00000000000..201fa4e22ec --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/OracleCDCWithSchemaChangeIT.java @@ -0,0 +1,487 @@ +/* + * 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.seatunnel.connectors.seatunnel.cdc.oracle; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.converter.BasicTypeDefine; +import org.apache.seatunnel.common.utils.JdbcUrlUtil; +import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.testutils.MySqlContainer; +import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.testutils.MySqlVersion; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.mysql.MySqlCatalog; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleCatalog; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleURLParser; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.mysql.MySqlTypeConverter; +import org.apache.seatunnel.e2e.common.TestResource; +import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.container.EngineType; +import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; +import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.MethodOrderer; +import org.junit.jupiter.api.Order; +import org.junit.jupiter.api.TestMethodOrder; +import org.junit.jupiter.api.TestTemplate; +import org.junit.platform.commons.util.StringUtils; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.DockerLoggerFactory; + +import com.google.common.collect.Lists; +import com.mysql.cj.MysqlType; +import lombok.extern.slf4j.Slf4j; + +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.Statement; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; + +import static org.awaitility.Awaitility.await; + +@Slf4j +@TestMethodOrder(MethodOrderer.OrderAnnotation.class) +@DisabledOnContainer( + value = {}, + type = {EngineType.SPARK, EngineType.FLINK}, + disabledReason = + "Currently SPARK do not support cdc. In addition, currently only the zeta engine supports schema evolution for pr https://github.com/apache/seatunnel/pull/5125.") +public class OracleCDCWithSchemaChangeIT extends AbstractOracleCDCIT implements TestResource { + + private static final String BASIC_QUERY = "select * from %s.%s"; + + private static final String QUERY = BASIC_QUERY + " ORDER BY ID ASC"; + + private static final String ORACLE_DESC = + "SELECT COLUMN_NAME, DATA_TYPE, DATA_LENGTH, DATA_PRECISION, DATA_SCALE, NULLABLE, DATA_DEFAULT FROM all_tab_columns WHERE table_name = '%s' AND owner = '%s'"; + + private static final String PROJECTION_QUERY = + "select ID,VAL_VARCHAR,VAL_VARCHAR2,VAL_NVARCHAR2,VAL_CHAR,VAL_NCHAR,VAL_BF,VAL_BD,VAL_F,VAL_F_10,VAL_NUM,VAL_DP,VAL_R,VAL_DECIMAL,VAL_NUMERIC,VAL_NUM_VS,VAL_INT,VAL_INTEGER,VAL_SMALLINT,VAL_NUMBER_38_NO_SCALE,VAL_NUMBER_38_SCALE_0,VAL_NUMBER_1,VAL_NUMBER_2,VAL_NUMBER_4,VAL_NUMBER_9,VAL_NUMBER_18,VAL_NUMBER_2_NEGATIVE_SCALE,VAL_NUMBER_4_NEGATIVE_SCALE,VAL_NUMBER_9_NEGATIVE_SCALE,VAL_NUMBER_18_NEGATIVE_SCALE,VAL_NUMBER_36_NEGATIVE_SCALE,VAL_DATE,VAL_TS,VAL_TS_PRECISION2,VAL_TS_PRECISION4,VAL_TS_PRECISION9,VAL_TSLTZ,ADD_COLUMN1,ADD_COLUMN2,ADD_COLUMN3 from %s.%s ORDER BY ID ASC"; + + private static final String MYSQL_SINK = "oracle_cdc_2_mysql_sink_table"; + + private static final String MYSQL_DATABASE = "oracle_sink"; + private static final String MYSQL_USER_NAME = "mysqluser"; + private static final String MYSQL_USER_PASSWORD = "mysqlpw"; + + private static final String MYSQL_CONNECTOR_NAME = "st_user_sink"; + private static final String MYSQL_CONNECTOR_PASSWORD = "mysqlpw"; + + private static final MySqlContainer MYSQL_CONTAINER = createMySqlContainer(MySqlVersion.V8_0); + + private static MySqlContainer createMySqlContainer(MySqlVersion version) { + return new MySqlContainer(version) + .withConfigurationOverride("docker/server-gtids/my.cnf") + .withSetupSQL("docker/setup.sql") + .withNetwork(NETWORK) + .withNetworkAliases(HOST) + .withDatabaseName(MYSQL_DATABASE) + .withUsername(MYSQL_USER_NAME) + .withPassword(MYSQL_USER_PASSWORD) + .withLogConsumer( + new Slf4jLogConsumer(DockerLoggerFactory.getLogger("mysql-docker-image"))); + } + + private String mysqlDriverUrl() { + return "https://repo1.maven.org/maven2/com/mysql/mysql-connector-j/8.0.32/mysql-connector-j-8.0.32.jar"; + } + + @TestContainerExtension + protected final ContainerExtendedFactory extendedFactory = + container -> { + Container.ExecResult extraCommands = + container.execInContainer( + "bash", + "-c", + "mkdir -p /tmp/seatunnel/plugins/Oracle-CDC/lib && cd /tmp/seatunnel/plugins/Oracle-CDC/lib && wget " + + oracleDriverUrl() + + " && mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && wget " + + mysqlDriverUrl()); + Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); + }; + + @BeforeAll + @Override + public void startUp() throws Exception { + log.info("The second stage: Starting Mysql containers..."); + Startables.deepStart(Stream.of(MYSQL_CONTAINER)).join(); + log.info("Mysql Containers are started"); + + ORACLE_CONTAINER.setPortBindings( + Lists.newArrayList(String.format("%s:%s", ORACLE_PORT, ORACLE_PORT))); + log.info("Starting Oracle containers..."); + Startables.deepStart(Stream.of(ORACLE_CONTAINER)).join(); + log.info("Oracle containers are started."); + } + + @AfterAll + @Override + public void tearDown() throws Exception { + ORACLE_CONTAINER.stop(); + MYSQL_CONTAINER.stop(); + } + + @Order(1) + @TestTemplate + public void testOracleCdc2OracleWithSchemaEvolutionCase(TestContainer container) + throws Exception { + + createAndInitialize("full_types", ADMIN_USER, ADMIN_PWD); + CompletableFuture.runAsync( + () -> { + try { + container.executeJob("/oraclecdc_to_oracle_with_schema_change.conf"); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + }); + + // Waiting to job running for auto create sink table + Thread.sleep(10000L); + + assertSchemaEvolution( + ORACLE_CONTAINER.getJdbcUrl(), + ORACLE_CONTAINER.getJdbcUrl(), + SCEHMA_NAME, + SOURCE_TABLE1 + "_SINK", + false); + } + + @Order(2) + @TestTemplate + public void testOracleCdc2OracleWithSchemaEvolutionCaseExactlyOnce(TestContainer container) + throws Exception { + + dropTable(ORACLE_CONTAINER.getJdbcUrl(), SCEHMA_NAME + "." + SOURCE_TABLE1); + dropTable(ORACLE_CONTAINER.getJdbcUrl(), SCEHMA_NAME + "." + SOURCE_TABLE1 + "_SINK"); + createAndInitialize("full_types", ADMIN_USER, ADMIN_PWD); + CompletableFuture.runAsync( + () -> { + try { + container.executeJob( + "/oraclecdc_to_oracle_with_schema_change_exactly_once.conf"); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + }); + + Thread.sleep(10000L); + + assertSchemaEvolution( + ORACLE_CONTAINER.getJdbcUrl(), + ORACLE_CONTAINER.getJdbcUrl(), + SCEHMA_NAME, + SOURCE_TABLE1 + "_SINK", + false); + } + + @Order(3) + @TestTemplate + public void testOracleCdc2MysqlWithSchemaEvolutionCase(TestContainer container) + throws Exception { + dropTable(ORACLE_CONTAINER.getJdbcUrl(), SCEHMA_NAME + "." + SOURCE_TABLE1); + dropTable(ORACLE_CONTAINER.getJdbcUrl(), SCEHMA_NAME + "." + SOURCE_TABLE1 + "_SINK"); + createAndInitialize("full_types", ADMIN_USER, ADMIN_PWD); + CompletableFuture.runAsync( + () -> { + try { + container.executeJob("/oraclecdc_to_mysql_with_schema_change.conf"); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + }); + + Thread.sleep(10000L); + + assertSchemaEvolution( + ORACLE_CONTAINER.getJdbcUrl(), + MYSQL_CONTAINER.getJdbcUrl(), + MYSQL_DATABASE, + MYSQL_SINK, + true); + } + + private void assertSchemaEvolution( + String sourceJdbcUrl, + String sinkJdbcUrl, + String sinkSchemaName, + String sinkTableName, + boolean oracle2Mysql) + throws Exception { + await().atMost(300, TimeUnit.SECONDS) + .untilAsserted( + () -> + Assertions.assertIterableEquals( + query( + sourceJdbcUrl, + String.format(QUERY, SCEHMA_NAME, SOURCE_TABLE1)), + query( + sinkJdbcUrl, + String.format( + QUERY, sinkSchemaName, sinkTableName)))); + + // case1 add columns with cdc data at same time + createAndInitialize("add_columns", CONNECTOR_USER, CONNECTOR_PWD); + // verify the schema: oracle -> oracle + if (!oracle2Mysql) { + await().atMost(300, TimeUnit.SECONDS) + .untilAsserted( + () -> + Assertions.assertIterableEquals( + query( + sourceJdbcUrl, + String.format( + ORACLE_DESC, + SCEHMA_NAME, + SOURCE_TABLE1)), + query( + sinkJdbcUrl, + String.format( + ORACLE_DESC, + sinkSchemaName, + sinkTableName)))); + } else { + // verify the schema: oracle -> mysql + verifyOracle2MysqlSchema(); + } + // verify the data + await().atMost(300, TimeUnit.SECONDS) + .untilAsserted( + () -> { + if (!oracle2Mysql) { + Assertions.assertIterableEquals( + query( + sourceJdbcUrl, + String.format( + BASIC_QUERY, + SCEHMA_NAME, + SOURCE_TABLE1) + + " where id >= 7 ORDER BY ID ASC"), + query( + sinkJdbcUrl, + String.format( + BASIC_QUERY, + sinkSchemaName, + sinkTableName) + + " where id >= 7 ORDER BY ID ASC")); + + // The default value of add_column4 is current_timestamp(),so the + // history data of sink table with this column may be different from + // the source table because delay of apply schema change. + String query = + String.format( + "SELECT t1.id AS table1_id, t1.add_column4 AS table1_timestamp, " + + "t2.id AS table2_id, t2.add_column4 AS table2_timestamp, " + + "ABS(EXTRACT(SECOND FROM (t1.add_column4 - t2.add_column4))) AS time_diff " + + "FROM %s.%s t1 " + + "INNER JOIN %s.%s t2 ON t1.id = t2.id", + SCEHMA_NAME, + SOURCE_TABLE1, + SCEHMA_NAME, + sinkTableName); + try (Connection jdbcConnection = + getJdbcConnection( + ORACLE_CONTAINER.getJdbcUrl(), + CONNECTOR_USER, + CONNECTOR_PWD); + Statement statement = jdbcConnection.createStatement(); + ResultSet resultSet = statement.executeQuery(query); ) { + while (resultSet.next()) { + int timeDiff = resultSet.getInt("time_diff"); + Assertions.assertTrue( + timeDiff <= 30, + "Time difference exceeds 3 seconds: " + + timeDiff + + " seconds"); + } + } + } + + Assertions.assertIterableEquals( + query( + sourceJdbcUrl, + String.format( + PROJECTION_QUERY, SCEHMA_NAME, SOURCE_TABLE1)), + query( + sinkJdbcUrl, + String.format( + PROJECTION_QUERY, + sinkSchemaName, + sinkTableName))); + }); + + // case2 drop columns with cdc data at same time + assertCaseByDdlName( + sourceJdbcUrl, + sinkJdbcUrl, + "drop_columns", + sinkSchemaName, + sinkTableName, + oracle2Mysql); + + // case3 change column name with cdc data at same time + assertCaseByDdlName( + sourceJdbcUrl, + sinkJdbcUrl, + "rename_columns", + sinkSchemaName, + sinkTableName, + oracle2Mysql); + + // case4 modify column data type with cdc data at same time + assertCaseByDdlName( + sourceJdbcUrl, + sinkJdbcUrl, + "modify_columns", + sinkSchemaName, + sinkTableName, + oracle2Mysql); + } + + private void assertCaseByDdlName( + String sourceJdbcUrl, + String sinkJdbcUrl, + String ddlSqlName, + String sinkSchemaname, + String sinkTable, + boolean ignore) + throws Exception { + createAndInitialize(ddlSqlName, CONNECTOR_USER, CONNECTOR_PWD); + assertTableStructureAndData(sourceJdbcUrl, sinkJdbcUrl, sinkSchemaname, sinkTable, ignore); + } + + private void assertTableStructureAndData( + String sourceJdbcUrl, + String sinkJdbcUrl, + String sinkSchemaName, + String sinkTable, + boolean oracle2Mysql) { + // verify the schema: oracle -> oracle + if (!oracle2Mysql) { + await().atMost(300, TimeUnit.SECONDS) + .untilAsserted( + () -> + Assertions.assertIterableEquals( + query( + sourceJdbcUrl, + String.format( + ORACLE_DESC, + SCEHMA_NAME, + SOURCE_TABLE1)), + query( + sinkJdbcUrl, + String.format( + ORACLE_DESC, + sinkSchemaName, + sinkTable)))); + } else { + // verify the schema: oracle -> mysql + verifyOracle2MysqlSchema(); + } + + // verify the data + await().atMost(300, TimeUnit.SECONDS) + .untilAsserted( + () -> + Assertions.assertIterableEquals( + query( + sourceJdbcUrl, + String.format(QUERY, SCEHMA_NAME, SOURCE_TABLE1)), + query( + sinkJdbcUrl, + String.format(QUERY, sinkSchemaName, sinkTable)))); + } + + private static void verifyOracle2MysqlSchema() { + try (MySqlCatalog mySqlCatalog = + new MySqlCatalog( + "mysql", + MYSQL_CONNECTOR_NAME, + MYSQL_CONNECTOR_PASSWORD, + JdbcUrlUtil.getUrlInfo(MYSQL_CONTAINER.getJdbcUrl())); + OracleCatalog oracleCatalog = + new OracleCatalog( + "oracle", + CONNECTOR_USER, + CONNECTOR_PWD, + OracleURLParser.parse(ORACLE_CONTAINER.getJdbcUrl()), + null)) { + mySqlCatalog.open(); + oracleCatalog.open(); + + CatalogTable mySqlCatalogTable = + mySqlCatalog.getTable(TablePath.of(MYSQL_DATABASE, MYSQL_SINK)); + TableSchema sinkTableSchemaInMysql = mySqlCatalogTable.getTableSchema(); + List sinkColumnsInMysql = sinkTableSchemaInMysql.getColumns(); + + CatalogTable oracleCatalogTable = + oracleCatalog.getTable(TablePath.of("ORCLCDB", SCEHMA_NAME, SOURCE_TABLE1)); + TableSchema sourceTableSchemaInOracle = oracleCatalogTable.getTableSchema(); + List sourceColumnsInOracle = sourceTableSchemaInOracle.getColumns(); + + MySqlTypeConverter mySqlTypeConverter = + new MySqlTypeConverter( + org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.mysql + .MySqlVersion.V_8); + Assertions.assertEquals(sourceColumnsInOracle.size(), sinkColumnsInMysql.size()); + for (int i = 0; i < sourceColumnsInOracle.size(); i++) { + Column sourceColumn = sourceColumnsInOracle.get(i); + BasicTypeDefine typeBasicTypeDefine = + mySqlTypeConverter.reconvert(sourceColumn); + Column sinkColumn = sinkColumnsInMysql.get(i); + BasicTypeDefine typeBasicTypeDefine1 = + mySqlTypeConverter.reconvert(sinkColumn); + Assertions.assertEquals( + typeBasicTypeDefine.getName(), typeBasicTypeDefine1.getName()); + Assertions.assertEquals( + typeBasicTypeDefine.getDataType(), typeBasicTypeDefine1.getDataType()); + Assertions.assertEquals( + typeBasicTypeDefine.getNativeType(), typeBasicTypeDefine1.getNativeType()); + Assertions.assertEquals( + typeBasicTypeDefine.getLength(), typeBasicTypeDefine1.getLength()); + Assertions.assertEquals( + typeBasicTypeDefine.getPrecision(), typeBasicTypeDefine1.getPrecision()); + Assertions.assertEquals( + typeBasicTypeDefine.getScale(), typeBasicTypeDefine1.getScale()); + Assertions.assertEquals( + typeBasicTypeDefine.isNullable(), typeBasicTypeDefine1.isNullable()); + Assertions.assertEquals( + typeBasicTypeDefine.getDefaultValue(), + typeBasicTypeDefine1.getDefaultValue()); + if (StringUtils.isNotBlank(typeBasicTypeDefine.getComment())) { + Assertions.assertTrue( + typeBasicTypeDefine1 + .getComment() + .equalsIgnoreCase(typeBasicTypeDefine.getComment())); + } + } + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/ddl/add_columns.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/ddl/add_columns.sql new file mode 100644 index 00000000000..54a5dc982f1 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/ddl/add_columns.sql @@ -0,0 +1,77 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. + +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: column_type_test +-- ---------------------------------------------------------------------------------------------------------------- +-- Set session timezone to fixed Asia/Shanghai for checking TIMESTAMP_LTZ type +-- ALTER SESSION SET TIME_ZONE='Asia/Shanghai'; + +INSERT INTO DEBEZIUM.FULL_TYPES +VALUES (2, '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', 'yyyy-mm-dd HH24:MI:SS.FF5') +); +INSERT INTO DEBEZIUM.FULL_TYPES +VALUES (3, 'vc3', 'vc3', 'nvc3', '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-31', 'yyyy-mm-dd'),TO_TIMESTAMP('2022-10-31 12:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-10-31 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'), + TO_TIMESTAMP('2022-10-31 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-10-31 12:34:56.125456789', 'yyyy-mm-dd HH24:MI:SS.FF9'),TO_TIMESTAMP_TZ('2022-10-31 01:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5') + ); +INSERT INTO DEBEZIUM.FULL_TYPES +VALUES (4, 'vc4', 'vc4', 'nvc4', '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-11-01', 'yyyy-mm-dd'),TO_TIMESTAMP('2022-11-01 12:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-01 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'), + TO_TIMESTAMP('2022-11-01 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-01 12:34:56.125456789', 'yyyy-mm-dd HH24:MI:SS.FF9'),TO_TIMESTAMP_TZ('2022-11-01 01:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5') + ); + + +update DEBEZIUM.FULL_TYPES set VAL_VARCHAR = 'dailai' where ID = 3; +delete from DEBEZIUM.FULL_TYPES where ID = 2; + +alter table DEBEZIUM.FULL_TYPES ADD (ADD_COLUMN1 VARCHAR2(64) default 'yy' not null,ADD_COLUMN2 int default 1 not null); + +update DEBEZIUM.FULL_TYPES set VAL_VARCHAR2 = 'dailai' where ID = 3; +INSERT INTO DEBEZIUM.FULL_TYPES +VALUES (5, 'vc5', 'vc5', 'nvc5', '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-11-02', 'yyyy-mm-dd'),TO_TIMESTAMP('2022-11-02 12:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-02 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'), + TO_TIMESTAMP('2022-11-02 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-02 12:34:56.125456789', 'yyyy-mm-dd HH24:MI:SS.FF9'),TO_TIMESTAMP_TZ('2022-11-02 01:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'), + 'yy5', 1 + ); +INSERT INTO DEBEZIUM.FULL_TYPES +VALUES (6, 'vc6', 'vc6', 'nvc6', '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-11-03', 'yyyy-mm-dd'),TO_TIMESTAMP('2022-11-03 12:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-03 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'), + TO_TIMESTAMP('2022-11-03 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-03 12:34:56.125456789', 'yyyy-mm-dd HH24:MI:SS.FF9'),TO_TIMESTAMP_TZ('2022-11-03 01:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'), + 'yy6', 1 + ); +delete from DEBEZIUM.FULL_TYPES where ID = 5; + +alter table DEBEZIUM.FULL_TYPES ADD ADD_COLUMN3 float default 1.1 not null ; +alter table DEBEZIUM.FULL_TYPES ADD ADD_COLUMN4 timestamp default current_timestamp not null ; + +delete from DEBEZIUM.FULL_TYPES where ID = 3; +INSERT INTO DEBEZIUM.FULL_TYPES +VALUES (7, 'vc7', 'vc7', 'nvc7', '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-11-02', 'yyyy-mm-dd'),TO_TIMESTAMP('2022-11-02 12:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-02 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'), + TO_TIMESTAMP('2022-11-02 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-02 12:34:56.125456789', 'yyyy-mm-dd HH24:MI:SS.FF9'),TO_TIMESTAMP_TZ('2022-11-02 01:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'), + 'yy7', 1, 1.1, TO_TIMESTAMP('2022-11-02 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5') + ); +INSERT INTO DEBEZIUM.FULL_TYPES +VALUES (8, 'vc8', 'vc8', 'nvc8', '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-11-03', 'yyyy-mm-dd'),TO_TIMESTAMP('2022-11-03 12:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-03 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'), + TO_TIMESTAMP('2022-11-03 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-03 12:34:56.125456789', 'yyyy-mm-dd HH24:MI:SS.FF9'),TO_TIMESTAMP_TZ('2022-11-03 01:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'), + 'yy8', 1, 1.1, TO_TIMESTAMP('2022-11-03 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5') + ); + +update DEBEZIUM.FULL_TYPES set VAL_VARCHAR = 'dailai' where ID = 7; diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/ddl/drop_columns.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/ddl/drop_columns.sql new file mode 100644 index 00000000000..87cd58bbe10 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/ddl/drop_columns.sql @@ -0,0 +1,54 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. + +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: column_type_test +-- ---------------------------------------------------------------------------------------------------------------- +-- Set session timezone to fixed Asia/Shanghai for checking TIMESTAMP_LTZ type +-- ALTER SESSION SET TIME_ZONE='Asia/Shanghai'; + +alter table DEBEZIUM.FULL_TYPES drop (ADD_COLUMN4); +INSERT INTO DEBEZIUM.FULL_TYPES +VALUES (9, 'vc7', 'vc7', 'nvc7', '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-11-02', 'yyyy-mm-dd'),TO_TIMESTAMP('2022-11-02 12:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-02 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'), + TO_TIMESTAMP('2022-11-02 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-02 12:34:56.125456789', 'yyyy-mm-dd HH24:MI:SS.FF9'),TO_TIMESTAMP_TZ('2022-11-02 01:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'), + 'yy7', 1, 1.1 + ); +INSERT INTO DEBEZIUM.FULL_TYPES +VALUES (10, 'vc8', 'vc8', 'nvc8', '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-11-03', 'yyyy-mm-dd'),TO_TIMESTAMP('2022-11-03 12:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-03 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'), + TO_TIMESTAMP('2022-11-03 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-03 12:34:56.125456789', 'yyyy-mm-dd HH24:MI:SS.FF9'),TO_TIMESTAMP_TZ('2022-11-03 01:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'), + 'yy8', 1, 1.1 + ); + + +update DEBEZIUM.FULL_TYPES set VAL_VARCHAR2 = 'dailai' where ID = 7; +delete from DEBEZIUM.FULL_TYPES where ID < 7; + +alter table DEBEZIUM.FULL_TYPES drop (ADD_COLUMN1, ADD_COLUMN3); + +INSERT INTO DEBEZIUM.FULL_TYPES +VALUES (11, 'vc7', 'vc7', 'nvc7', '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-11-02', 'yyyy-mm-dd'),TO_TIMESTAMP('2022-11-02 12:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-02 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'), + TO_TIMESTAMP('2022-11-02 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-02 12:34:56.125456789', 'yyyy-mm-dd HH24:MI:SS.FF9'),TO_TIMESTAMP_TZ('2022-11-02 01:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'), + 1 + ); +INSERT INTO DEBEZIUM.FULL_TYPES +VALUES (12, 'vc8', 'vc8', 'nvc8', '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-11-03', 'yyyy-mm-dd'),TO_TIMESTAMP('2022-11-03 12:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-03 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'), + TO_TIMESTAMP('2022-11-03 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-03 12:34:56.125456789', 'yyyy-mm-dd HH24:MI:SS.FF9'),TO_TIMESTAMP_TZ('2022-11-03 01:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'), + 1 + ); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/ddl/full_types.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/ddl/full_types.sql new file mode 100644 index 00000000000..c40fae207a8 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/ddl/full_types.sql @@ -0,0 +1,78 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. + +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: column_type_test +-- ---------------------------------------------------------------------------------------------------------------- +-- Set session timezone to fixed Asia/Shanghai for checking TIMESTAMP_LTZ type +-- ALTER SESSION SET TIME_ZONE='Asia/Shanghai'; +GRANT ALL PRIVILEGES TO dbzuser; + +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_TSLTZ TIMESTAMP(6) WITH LOCAL TIME ZONE, + 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', 'yyyy-mm-dd HH24:MI:SS.FF5') + ); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/ddl/modify_columns.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/ddl/modify_columns.sql new file mode 100644 index 00000000000..f29a27b9219 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/ddl/modify_columns.sql @@ -0,0 +1,37 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. + +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: column_type_test +-- ---------------------------------------------------------------------------------------------------------------- +-- Set session timezone to fixed Asia/Shanghai for checking TIMESTAMP_LTZ type +-- ALTER SESSION SET TIME_ZONE='Asia/Shanghai'; + +alter table DEBEZIUM.FULL_TYPES modify VAL_VARCHAR VARCHAR2(2048); + +delete from DEBEZIUM.FULL_TYPES where ID < 13; +INSERT INTO DEBEZIUM.FULL_TYPES +VALUES (16, 'vc7', 'vc7', 'nvc7', '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-11-02', 'yyyy-mm-dd'),TO_TIMESTAMP('2022-11-02 12:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-02 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'), + TO_TIMESTAMP('2022-11-02 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-02 12:34:56.125456789', 'yyyy-mm-dd HH24:MI:SS.FF9'),TO_TIMESTAMP_TZ('2022-11-02 01:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'), + 1 + ); +INSERT INTO DEBEZIUM.FULL_TYPES +VALUES (17, 'vc8', 'vc8', 'nvc8', '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-11-03', 'yyyy-mm-dd'),TO_TIMESTAMP('2022-11-03 12:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-03 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'), + TO_TIMESTAMP('2022-11-03 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-03 12:34:56.125456789', 'yyyy-mm-dd HH24:MI:SS.FF9'),TO_TIMESTAMP_TZ('2022-11-03 01:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'), + 1 + ); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/ddl/rename_columns.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/ddl/rename_columns.sql new file mode 100644 index 00000000000..ee9d92a210a --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/ddl/rename_columns.sql @@ -0,0 +1,37 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. + +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: column_type_test +-- ---------------------------------------------------------------------------------------------------------------- +-- Set session timezone to fixed Asia/Shanghai for checking TIMESTAMP_LTZ type +-- ALTER SESSION SET TIME_ZONE='Asia/Shanghai'; + +alter table DEBEZIUM.FULL_TYPES rename column VAL_VARCHAR2 to VAL_VARCHAR2_RENAMED; + +delete from DEBEZIUM.FULL_TYPES where ID < 10; +INSERT INTO DEBEZIUM.FULL_TYPES +VALUES (13, 'vc7', 'vc7', 'nvc7', '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-11-02', 'yyyy-mm-dd'),TO_TIMESTAMP('2022-11-02 12:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-02 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'), + TO_TIMESTAMP('2022-11-02 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-02 12:34:56.125456789', 'yyyy-mm-dd HH24:MI:SS.FF9'),TO_TIMESTAMP_TZ('2022-11-02 01:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'), + 1 + ); +INSERT INTO DEBEZIUM.FULL_TYPES +VALUES (14, 'vc8', 'vc8', 'nvc8', '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-11-03', 'yyyy-mm-dd'),TO_TIMESTAMP('2022-11-03 12:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-03 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'), + TO_TIMESTAMP('2022-11-03 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'),TO_TIMESTAMP('2022-11-03 12:34:56.125456789', 'yyyy-mm-dd HH24:MI:SS.FF9'),TO_TIMESTAMP_TZ('2022-11-03 01:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'), + 1 + ); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/docker/server-gtids/my.cnf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/docker/server-gtids/my.cnf new file mode 100644 index 00000000000..a390897885d --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/docker/server-gtids/my.cnf @@ -0,0 +1,65 @@ +# +# 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. +# + +# For advice on how to change settings please see +# http://dev.mysql.com/doc/refman/5.7/en/server-configuration-defaults.html + +[mysqld] +# +# Remove leading # and set to the amount of RAM for the most important data +# cache in MySQL. Start at 70% of total RAM for dedicated server, else 10%. +# innodb_buffer_pool_size = 128M +# +# Remove leading # to turn on a very important data integrity option: logging +# changes to the binary log between backups. +# log_bin +# +# Remove leading # to set options mainly useful for reporting servers. +# The server defaults are faster for transactions and fast SELECTs. +# Adjust sizes as needed, experiment to find the optimal values. +# join_buffer_size = 128M +# sort_buffer_size = 2M +# read_rnd_buffer_size = 2M +skip-host-cache +skip-name-resolve +#datadir=/var/lib/mysql +#socket=/var/lib/mysql/mysql.sock +secure-file-priv=/var/lib/mysql +user=mysql + +# Disabling symbolic-links is recommended to prevent assorted security risks +symbolic-links=0 + +#log-error=/var/log/mysqld.log +#pid-file=/var/run/mysqld/mysqld.pid + +# ---------------------------------------------- +# Enable the binlog for replication & CDC +# ---------------------------------------------- + +# Enable binary replication log and set the prefix, expiration, and log format. +# The prefix is arbitrary, expiration can be short for integration tests but would +# be longer on a production system. Row-level info is required for ingest to work. +# Server ID is required, but this will vary on production systems +server-id = 223344 +log_bin = mysql-bin +expire_logs_days = 1 +binlog_format = row + +# enable gtid mode +gtid_mode = on +enforce_gtid_consistency = on \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/docker/setup.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/docker/setup.sql new file mode 100644 index 00000000000..726ca738899 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/docker/setup.sql @@ -0,0 +1,35 @@ +-- +-- 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. +-- + +-- In production you would almost certainly limit the replication user must be on the follower (slave) machine, +-- to prevent other clients accessing the log from other machines. For example, 'replicator'@'follower.acme.com'. +-- However, in this database we'll grant 2 users different privileges: +-- +-- 1) 'mysqluser' - all privileges +-- 2) 'st_user_source' - all privileges required by the snapshot reader AND binlog reader (used for testing) +-- 3) 'st_user_sink' - all privileges required by the write data (used for testing) +-- +GRANT ALL PRIVILEGES ON *.* TO 'mysqluser'@'%'; + +CREATE USER 'st_user_source' IDENTIFIED BY 'mysqlpw'; +GRANT SELECT, RELOAD, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT, DROP, LOCK TABLES ON *.* TO 'st_user_source'@'%'; +CREATE USER 'st_user_sink' IDENTIFIED BY 'mysqlpw'; +GRANT SELECT, INSERT, UPDATE, DELETE, CREATE, DROP, INDEX, ALTER ON *.* TO 'st_user_sink'@'%'; +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: oracle_sink +-- ---------------------------------------------------------------------------------------------------------------- +CREATE DATABASE if not exists oracle_sink; diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_mysql_with_schema_change.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_mysql_with_schema_change.conf new file mode 100644 index 00000000000..5beddd504fa --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_mysql_with_schema_change.conf @@ -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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + # You can set engine configuration here + parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Oracle-CDC { + result_table_name = "customers" + username = "dbzuser" + password = "dbz" + database-names = ["ORCLCDB"] + schema-names = ["DEBEZIUM"] + table-names = ["ORCLCDB.DEBEZIUM.FULL_TYPES"] + base-url = "jdbc:oracle:thin:@oracle-host:1521/ORCLCDB" + source.reader.close.timeout = 120000 + connection.pool.size = 1 + debezium { + database.oracle.jdbc.timezoneAsRegion = false + include.schema.changes = true + } + } +} + +transform { +} + +sink { + jdbc { + source_table_name = "customers" + url = "jdbc:mysql://oracle-host:3306/oracle_sink" + driver = "com.mysql.cj.jdbc.Driver" + user = "st_user_sink" + password = "mysqlpw" + generate_sink_sql = true + # You need to configure both database and table + database = oracle_sink + table = oracle_cdc_2_mysql_sink_table + primary_keys = ["id"] + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_with_schema_change.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_with_schema_change.conf new file mode 100644 index 00000000000..96c6c67dc89 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_with_schema_change.conf @@ -0,0 +1,64 @@ +# +# 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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + # You can set engine configuration here + parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Oracle-CDC { + result_table_name = "customers" + username = "dbzuser" + password = "dbz" + database-names = ["ORCLCDB"] + schema-names = ["DEBEZIUM"] + table-names = ["ORCLCDB.DEBEZIUM.FULL_TYPES"] + base-url = "jdbc:oracle:thin:@oracle-host:1521/ORCLCDB" + source.reader.close.timeout = 120000 + connection.pool.size = 1 + debezium { + database.oracle.jdbc.timezoneAsRegion = false + include.schema.changes = true + } + } +} + +transform { +} + +sink { + Jdbc { + source_table_name = "customers" + driver = "oracle.jdbc.driver.OracleDriver" + url = "jdbc:oracle:thin:@oracle-host:1521/ORCLCDB" + user = "dbzuser" + password = "dbz" + generate_sink_sql = true + database = "ORCLCDB" + table = "DEBEZIUM.FULL_TYPES_SINK" + batch_size = 1 + primary_keys = ["ID"] + connection.pool.size = 1 + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_with_schema_change_exactly_once.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_with_schema_change_exactly_once.conf new file mode 100644 index 00000000000..4f09d40a1f7 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_with_schema_change_exactly_once.conf @@ -0,0 +1,67 @@ +# +# 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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + # You can set engine configuration here + parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Oracle-CDC { + result_table_name = "customers" + username = "dbzuser" + password = "dbz" + database-names = ["ORCLCDB"] + schema-names = ["DEBEZIUM"] + table-names = ["ORCLCDB.DEBEZIUM.FULL_TYPES"] + base-url = "jdbc:oracle:thin:@oracle-host:1521/ORCLCDB" + source.reader.close.timeout = 120000 + connection.pool.size = 1 + debezium { + database.oracle.jdbc.timezoneAsRegion = false + include.schema.changes = true + } + } +} + +transform { +} + +sink { +Jdbc { + source_table_name = "customers" + driver = "oracle.jdbc.driver.OracleDriver" + url = "jdbc:oracle:thin:@oracle-host:1521/ORCLCDB" + user = "dbzuser" + password = "dbz" + generate_sink_sql = true + schema_save_mode = RECREATE_SCHEMA + database = "ORCLCDB" + table = "DEBEZIUM.FULL_TYPES_SINK" + batch_size = 1 + primary_keys = ["ID"] + connection.pool.size = 1 + is_exactly_once = true + xa_data_source_class_name = "oracle.jdbc.xa.OracleXADataSource" +} +}