aiwenmo commented on code in PR #3995:
URL: https://github.com/apache/flink-cdc/pull/3995#discussion_r2193803732


##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/com/apache/flink/cdc/connectors/oracle/dto/ColumnInfo.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.apache.flink.cdc.connectors.oracle.dto;
+

Review Comment:
   Please use org.apache.



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/pom.xml:
##########
@@ -0,0 +1,253 @@
+<?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>${debezium.version}</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>org.apache.flink</groupId>
+            <artifactId>flink-connector-test-util</artifactId>
+            <version>${parent.version}</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>org.apache.flink:flink-connector-debezium</include>
+                                    
<include>org.apache.flink:flink-connector-oracle-cdc</include>
+                                    
<include>org.apache.flink: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>com.fasterxml.*:*</include>
+                                    <include>com.google.guava:*</include>
+                                    
<include>com.esri.geometry:esri-geometry-api</include>
+                                    <include>com.zaxxer:HikariCP</include>
+                                    <!--  Include fixed version 
30.1.1-jre-14.0 of flink shaded guava  -->
+                                    
<include>org.apache.flink:flink-shaded-guava</include>
+                                </includes>
+                            </artifactSet>
+                            <filters>
+                                <filter>
+                                    <artifact>org.apache.kafka:*</artifact>
+                                    <excludes>
+                                        
<exclude>kafka/kafka-version.properties</exclude>
+                                        <exclude>LICENSE</exclude>
+                                        <!-- Does not contain anything 
relevant.
+                                            Cites a binary dependency on 
jersey, but this is neither reflected in the
+                                            dependency graph, nor are any 
jersey files bundled. -->
+                                        <exclude>NOTICE</exclude>
+                                        <exclude>common/**</exclude>
+                                    </excludes>
+                                </filter>
+                            </filters>
+                            <relocations>
+                                <relocation>
+                                    <pattern>org.apache.kafka</pattern>
+                                    <shadedPattern>
+                                        
com.ververica.cdc.connectors.shaded.org.apache.kafka
+                                    </shadedPattern>
+                                </relocation>
+                                <relocation>
+                                    <pattern>org.antlr</pattern>
+                                    <shadedPattern>
+                                        
com.ververica.cdc.connectors.shaded.org.antlr
+                                    </shadedPattern>
+                                </relocation>
+                                <relocation>
+                                    <pattern>com.fasterxml</pattern>
+                                    <shadedPattern>
+                                        
com.ververica.cdc.connectors.shaded.com.fasterxml
+                                    </shadedPattern>
+                                </relocation>
+                                <relocation>
+                                    <pattern>com.google</pattern>
+                                    <shadedPattern>
+                                        
com.ververica.cdc.connectors.shaded.com.google
+                                    </shadedPattern>
+                                </relocation>
+                                <relocation>
+                                    <pattern>com.esri.geometry</pattern>
+                                    
<shadedPattern>com.ververica.cdc.connectors.shaded.com.esri.geometry</shadedPattern>
+                                </relocation>
+                                <relocation>
+                                    <pattern>com.zaxxer</pattern>
+                                    <shadedPattern>
+                                        
com.ververica.cdc.connectors.shaded.com.zaxxer
+                                    </shadedPattern>

Review Comment:
   Why relocate com.ververica.cdc.connectors? 
   Please use the correct package path.



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/com/apache/flink/cdc/connectors/oracle/factory/OracleDataSourceFactory.java:
##########
@@ -0,0 +1,245 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.apache.flink.cdc.connectors.oracle.factory;
+
+import org.apache.flink.cdc.common.annotation.Internal;
+import org.apache.flink.cdc.common.configuration.ConfigOption;
+import org.apache.flink.cdc.common.configuration.Configuration;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.factories.DataSourceFactory;
+import org.apache.flink.cdc.common.schema.Selectors;
+import org.apache.flink.cdc.common.source.DataSource;
+import org.apache.flink.cdc.common.utils.StringUtils;
+import org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfig;
+import 
org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfigFactory;
+import org.apache.flink.cdc.connectors.oracle.table.OracleReadableMetaData;
+
+import com.apache.flink.cdc.connectors.oracle.source.OracleDataSource;
+import com.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions;
+import com.apache.flink.cdc.connectors.oracle.utils.OracleSchemaUtils;

Review Comment:
   Please use org.apache.



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/com/apache/flink/cdc/connectors/oracle/factory/OracleDataSourceFactory.java:
##########
@@ -0,0 +1,245 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.apache.flink.cdc.connectors.oracle.factory;
+
+import org.apache.flink.cdc.common.annotation.Internal;
+import org.apache.flink.cdc.common.configuration.ConfigOption;
+import org.apache.flink.cdc.common.configuration.Configuration;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.factories.DataSourceFactory;
+import org.apache.flink.cdc.common.schema.Selectors;
+import org.apache.flink.cdc.common.source.DataSource;
+import org.apache.flink.cdc.common.utils.StringUtils;
+import org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfig;
+import 
org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfigFactory;
+import org.apache.flink.cdc.connectors.oracle.table.OracleReadableMetaData;
+
+import com.apache.flink.cdc.connectors.oracle.source.OracleDataSource;
+import com.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions;
+import com.apache.flink.cdc.connectors.oracle.utils.OracleSchemaUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.time.ZoneId;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static 
com.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.METADATA_LIST;

Review Comment:
   Please use org.apache.



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/com/apache/flink/cdc/connectors/oracle/factory/OracleDataSourceFactory.java:
##########
@@ -0,0 +1,245 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.apache.flink.cdc.connectors.oracle.factory;
+
+import org.apache.flink.cdc.common.annotation.Internal;
+import org.apache.flink.cdc.common.configuration.ConfigOption;
+import org.apache.flink.cdc.common.configuration.Configuration;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.factories.DataSourceFactory;
+import org.apache.flink.cdc.common.schema.Selectors;
+import org.apache.flink.cdc.common.source.DataSource;
+import org.apache.flink.cdc.common.utils.StringUtils;
+import org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfig;
+import 
org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfigFactory;
+import org.apache.flink.cdc.connectors.oracle.table.OracleReadableMetaData;
+
+import com.apache.flink.cdc.connectors.oracle.source.OracleDataSource;
+import com.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions;
+import com.apache.flink.cdc.connectors.oracle.utils.OracleSchemaUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.time.ZoneId;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static 
com.apache.flink.cdc.connectors.oracle.source.OracleDataSourceOptions.METADATA_LIST;
+import static 
org.apache.flink.cdc.connectors.base.utils.ObjectUtils.doubleCompare;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** A {@link Factory} to create {@link OracleDataSource}. */
+@Internal
+public class OracleDataSourceFactory implements DataSourceFactory {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(OracleDataSourceFactory.class);
+
+    public static final String IDENTIFIER = "oracle";
+    private String[] capturedTables;
+
+    @Override
+    public DataSource createDataSource(Context context) {
+        final Configuration config = context.getFactoryConfiguration();
+        int fetchSize = 
config.get(OracleDataSourceOptions.SCAN_SNAPSHOT_FETCH_SIZE);
+        int splitSize = 
config.get(OracleDataSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE);
+        int splitMetaGroupSize = 
config.get(OracleDataSourceOptions.CHUNK_META_GROUP_SIZE);
+
+        double distributionFactorUpper =
+                
config.get(OracleDataSourceOptions.CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND);
+        double distributionFactorLower =
+                
config.get(OracleDataSourceOptions.CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND);
+        int connectMaxRetries = 
config.get(OracleDataSourceOptions.CONNECT_MAX_RETRIES);
+        int connectionPoolSize = 
config.get(OracleDataSourceOptions.CONNECTION_POOL_SIZE);
+        String tables = config.get(OracleDataSourceOptions.TABLES);
+        validateIntegerOption(
+                OracleDataSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE, 
splitSize, 1);
+        validateIntegerOption(OracleDataSourceOptions.CHUNK_META_GROUP_SIZE, 
splitMetaGroupSize, 1);
+        
validateIntegerOption(OracleDataSourceOptions.SCAN_SNAPSHOT_FETCH_SIZE, 
fetchSize, 1);
+        validateIntegerOption(OracleDataSourceOptions.CONNECTION_POOL_SIZE, 
connectionPoolSize, 1);
+        validateIntegerOption(OracleDataSourceOptions.CONNECT_MAX_RETRIES, 
connectMaxRetries, 0);
+        validateDistributionFactorUpper(distributionFactorUpper);
+        validateDistributionFactorLower(distributionFactorLower);
+        OracleSourceConfigFactory configFactory =
+                (OracleSourceConfigFactory)
+                        new OracleSourceConfigFactory()
+                                
.hostname(config.get(OracleDataSourceOptions.HOSTNAME))
+                                .port(config.get(OracleDataSourceOptions.PORT))
+                                .databaseList(
+                                        config.get(
+                                                OracleDataSourceOptions
+                                                        .DATABASE)) // monitor 
oracledatabase
+                                .tableList(
+                                        config.get(
+                                                OracleDataSourceOptions
+                                                        .TABLES)) // monitor 
productstable
+                                
.username(config.get(OracleDataSourceOptions.USERNAME))
+                                
.password(config.get(OracleDataSourceOptions.PASSWORD))
+                                .includeSchemaChanges(true);
+        Selectors selectors = new 
Selectors.SelectorsBuilder().includeTables(tables).build();
+        String[] capturedTables = getTableList(configFactory.create(0), 
selectors);
+        if (capturedTables.length == 0) {
+            throw new IllegalArgumentException(
+                    "Cannot find any table by the option 'tables' = " + 
tables);
+        }
+        configFactory.tableList(capturedTables);
+        
configFactory.databaseList(config.get(OracleDataSourceOptions.DATABASE));
+        configFactory.schemaList(new String[] 
{config.get(OracleDataSourceOptions.SCHEMALIST)});
+        String metadataList = config.get(METADATA_LIST);
+        List<OracleReadableMetaData> readableMetadataList = 
listReadableMetadata(metadataList);
+        return new OracleDataSource(configFactory, config, capturedTables, 
readableMetadataList);
+    }
+
+    public static List<OracleReadableMetaData> listReadableMetadata(String 
metadataList) {
+        if (StringUtils.isNullOrWhitespaceOnly(metadataList)) {
+            return new ArrayList<>();
+        }
+        Set<String> readableMetadataList =
+                Arrays.stream(metadataList.split(","))
+                        .map(String::trim)
+                        .collect(Collectors.toSet());
+        List<OracleReadableMetaData> foundMetadata = new ArrayList<>();
+        for (OracleReadableMetaData metadata : 
OracleReadableMetaData.values()) {
+            if (readableMetadataList.contains(metadata.getKey())) {
+                foundMetadata.add(metadata);
+                readableMetadataList.remove(metadata.getKey());
+            }
+        }
+        if (readableMetadataList.isEmpty()) {
+            return foundMetadata;
+        }
+        throw new IllegalArgumentException(
+                String.format(
+                        "[%s] cannot be found in mysql metadata.",

Review Comment:
   oracle metadata



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to