linjianchang commented on code in PR #3995: URL: https://github.com/apache/flink-cdc/pull/3995#discussion_r2052006751
########## flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/pom.xml: ########## @@ -0,0 +1,263 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ Copyright 2023 Ververica Inc. + ~ + ~ Licensed under the Apache License, Version 2.0 (the "License"); + ~ you may not use this file except in compliance with the License. + ~ You may obtain a copy of the License at + ~ + ~ http://www.apache.org/licenses/LICENSE-2.0 + ~ + ~ Unless required by applicable law or agreed to in writing, software + ~ distributed under the License is distributed on an "AS IS" BASIS, + ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + ~ See the License for the specific language governing permissions and + ~ limitations under the License. + --> + +<project xmlns="http://maven.apache.org/POM/4.0.0" + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <parent> + <groupId>org.apache.flink</groupId> + <artifactId>flink-cdc-pipeline-connectors</artifactId> + <version>${revision}</version> + </parent> + <modelVersion>4.0.0</modelVersion> + + <artifactId>flink-cdc-pipeline-connector-oracle</artifactId> + + <properties> + </properties> + + <dependencies> + <dependency> + <groupId>io.debezium</groupId> + <artifactId>debezium-core</artifactId> + <version>1.9.8.Final</version> + </dependency> + <dependency> + <groupId>io.debezium</groupId> + <artifactId>debezium-ddl-parser</artifactId> + <version>${debezium.version}</version> + <exclusions> + <exclusion> + <artifactId>debezium-core</artifactId> + <groupId>io.debezium</groupId> + </exclusion> + </exclusions> + </dependency> + + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-table-planner_${scala.binary.version}</artifactId> + <version>${flink.version}</version> + <scope>test</scope> + </dependency> + + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-table-runtime</artifactId> + <version>${flink.version}</version> + <scope>test</scope> + </dependency> + + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-test-utils</artifactId> + <version>${flink.version}</version> + <scope>test</scope> + </dependency> + + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-connector-test-utils</artifactId> + <version>${flink.version}</version> + <scope>test</scope> + </dependency> + + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-core</artifactId> + <version>${flink.version}</version> + <type>test-jar</type> + <scope>test</scope> + </dependency> + + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-streaming-java</artifactId> + <version>${flink.version}</version> + <type>test-jar</type> + <scope>test</scope> + </dependency> + + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-table-common</artifactId> + <version>${flink.version}</version> + <type>test-jar</type> + <scope>test</scope> + </dependency> + + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-tests</artifactId> + <version>${flink.version}</version> + <type>test-jar</type> + <scope>test</scope> + </dependency> + + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-table-planner_${scala.binary.version}</artifactId> + <version>${flink.version}</version> + <type>test-jar</type> + <scope>test</scope> + </dependency> + + <!-- test dependencies on TestContainers --> + + <dependency> + <groupId>org.testcontainers</groupId> + <artifactId>mysql</artifactId> + <version>${testcontainers.version}</version> + <scope>test</scope> + </dependency> + <!-- geometry dependencies --> + <dependency> + <groupId>com.esri.geometry</groupId> + <artifactId>esri-geometry-api</artifactId> + <version>${geometry.version}</version> + <exclusions> + <exclusion> + <groupId>com.fasterxml.jackson.core</groupId> + <artifactId>jackson-core</artifactId> + </exclusion> + </exclusions> + </dependency> + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-connector-oracle-cdc</artifactId> + <version>${parent.version}</version> + </dependency> + + <dependency> + <groupId>com.jayway.jsonpath</groupId> + <artifactId>json-path</artifactId> + <version>${json-path.version}</version> + <scope>test</scope> + </dependency> + <dependency> + <groupId>com.ververica</groupId> + <artifactId>flink-cdc-source-e2e-tests</artifactId> + <version>cty-3.0-2.2-SNAPSHOT</version> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-connector-test-util</artifactId> + <version>3.4-SNAPSHOT</version> + <scope>test</scope> + </dependency> + </dependencies> + + <build> + <plugins> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-shade-plugin</artifactId> + <version>3.2.4</version> + <executions> + <execution> + <id>shade-flink</id> + <phase>package</phase> + <goals> + <goal>shade</goal> + </goals> + <configuration> + <shadeTestJar>false</shadeTestJar> + <artifactSet> + <includes> + <include>io.debezium:debezium-api</include> + <include>io.debezium:debezium-embedded</include> + <include>io.debezium:debezium-core</include> + <include>io.debezium:debezium-ddl-parser</include> + <include>io.debezium:debezium-connector-oracle</include> + <include>io.debezium:debezium-connector-mysql</include> + <include>com.ververica:flink-connector-debezium</include> + <include>com.ververica:flink-connector-mysql-cdc</include> + <include>com.ververica:flink-connector-oracle-cdc</include> + <include>com.ververica:flink-cdc-base</include> + <include>com.oracle.ojdbc:ojdbc8</include> + <include>cn.easyproject:orai18n</include> + <include>org.antlr:antlr4-runtime</include> + <include>org.apache.kafka:*</include> + <include>mysql:mysql-connector-java</include> + <include>com.zendesk:mysql-binlog-connector-java</include> Review Comment: Have been removed ########## flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/com/apache/flink/cdc/connectors/oracle/source/OracleEventDeserializer.java: ########## @@ -0,0 +1,189 @@ +/* + * 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 com.apache.flink.cdc.connectors.oracle.source; + +import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.cdc.common.data.binary.BinaryStringData; +import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.connectors.oracle.table.OracleReadableMetaData; +import org.apache.flink.cdc.debezium.event.DebeziumEventDeserializationSchema; +import org.apache.flink.cdc.debezium.table.DebeziumChangelogMode; +import org.apache.flink.table.data.TimestampData; + +import com.apache.flink.cdc.connectors.oracle.source.parser.OracleAntlrDdlParser; +import com.esri.core.geometry.ogc.OGCGeometry; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.debezium.data.Envelope; +import io.debezium.data.geometry.Geometry; +import io.debezium.data.geometry.Point; +import io.debezium.relational.Tables; +import io.debezium.relational.history.HistoryRecord; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; + +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.apache.flink.cdc.connectors.base.utils.SourceRecordUtils.getHistoryRecord; + +/** Event deserializer for {@link OracleDataSource}. */ +@Internal +public class OracleEventDeserializer extends DebeziumEventDeserializationSchema { + + private static final long serialVersionUID = 1L; + + public static final String SCHEMA_CHANGE_EVENT_KEY_NAME = + "io.debezium.connector.oracle.SchemaChangeKey"; + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private final boolean includeSchemaChanges; + + private transient Tables tables; + + private transient OracleAntlrDdlParser customParser; + + private String hostname; + + private String port; + + private boolean isAddMeta; + + List<OracleReadableMetaData> readableMetadataList; + + public OracleEventDeserializer( + DebeziumChangelogMode changelogMode, + boolean includeSchemaChanges, + String hostname, + String port, + List<OracleReadableMetaData> readableMetadataList) { + super(new OracleSchemaDataTypeInference(), changelogMode); + this.includeSchemaChanges = includeSchemaChanges; + this.hostname = hostname; + this.port = port; + this.readableMetadataList = readableMetadataList; + } + + @Override + protected List<SchemaChangeEvent> deserializeSchemaChangeRecord(SourceRecord record) { + if (includeSchemaChanges) { + try { + HistoryRecord historyRecord = getHistoryRecord(record); + + String databaseName = + historyRecord.document().getString(HistoryRecord.Fields.DATABASE_NAME); + String schemaName = + historyRecord.document().getString(HistoryRecord.Fields.SCHEMA_NAME); + if (customParser == null) { + customParser = new OracleAntlrDdlParser(databaseName, schemaName); + tables = new Tables(); + } + String ddl = + historyRecord.document().getString(HistoryRecord.Fields.DDL_STATEMENTS); + customParser.setCurrentDatabase(databaseName); + customParser.parse(ddl, tables); + return customParser.getAndClearParsedEvents(); + } catch (Exception e) { + throw new IllegalStateException("Failed to parse the schema change : " + record, e); + } + } + return Collections.emptyList(); + } + + @Override + protected boolean isDataChangeRecord(SourceRecord record) { + Schema valueSchema = record.valueSchema(); + Struct value = (Struct) record.value(); + return value != null + && valueSchema != null + && valueSchema.field(Envelope.FieldName.OPERATION) != null + && value.getString(Envelope.FieldName.OPERATION) != null; + } + + @Override + protected boolean isSchemaChangeRecord(SourceRecord record) { + Schema keySchema = record.keySchema(); + return keySchema != null && SCHEMA_CHANGE_EVENT_KEY_NAME.equalsIgnoreCase(keySchema.name()); + } + + @Override + protected TableId getTableId(SourceRecord record) { + String[] parts = record.topic().split("\\."); + return TableId.tableId(parts[1], parts[2]); + } + + @Override + protected Map<String, String> getMetadata(SourceRecord record) { + Map<String, String> map = new HashMap<>(); + if (isAddMeta) { + map.put(OracleDataSourceOptions.HOSTNAME.key(), hostname); + map.put(OracleDataSourceOptions.PORT.key(), port); + } Review Comment: Have been removed -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org