lvyanquan commented on code in PR #3916:
URL: https://github.com/apache/flink-cdc/pull/3916#discussion_r1977066005


##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-jdbc/src/main/java/org/apache/flink/cdc/connectors/jdbc/catalog/JdbcCatalog.java:
##########
@@ -0,0 +1,289 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.jdbc.catalog;
+
+import org.apache.flink.cdc.common.event.AddColumnEvent;
+import org.apache.flink.cdc.common.event.AlterColumnTypeEvent;
+import org.apache.flink.cdc.common.event.CreateTableEvent;
+import org.apache.flink.cdc.common.event.DropColumnEvent;
+import org.apache.flink.cdc.common.event.DropTableEvent;
+import org.apache.flink.cdc.common.event.RenameColumnEvent;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.event.TruncateTableEvent;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.common.types.DataType;
+import org.apache.flink.cdc.common.utils.Preconditions;
+import org.apache.flink.cdc.common.utils.StringUtils;
+import org.apache.flink.cdc.connectors.jdbc.config.JdbcSinkConfig;
+import org.apache.flink.cdc.connectors.jdbc.conn.JdbcConnectionFactory;
+import org.apache.flink.cdc.connectors.jdbc.conn.JdbcConnectionPoolFactory;
+import org.apache.flink.table.catalog.exceptions.CatalogException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.List;
+import java.util.Map;
+
+/** An abstract base class for various JDBC-like sinks. */
+public abstract class JdbcCatalog implements Serializable {
+    private static final Logger LOG = 
LoggerFactory.getLogger(JdbcCatalog.class);
+
+    private final String catalogName;
+
+    protected final JdbcConnectionFactory connectionFactory;
+
+    public JdbcCatalog(
+            String catalogName, JdbcSinkConfig sinkConfig, 
JdbcConnectionPoolFactory poolFactory) {
+        this.catalogName = catalogName;
+        this.connectionFactory = new JdbcConnectionFactory(sinkConfig, 
poolFactory);
+    }
+
+    public String getCatalogName() {
+        return catalogName;
+    }
+
+    public abstract void close() throws CatalogException;
+
+    public void createTable(CreateTableEvent createTableEvent, boolean 
ignoreIfExists)
+            throws CatalogException {
+        TableId tableId = createTableEvent.tableId();
+        checkTableIdArguments(tableId);
+
+        Schema schema = createTableEvent.getSchema();
+        String createTableSql = buildCreateTableSql(tableId, schema, 
ignoreIfExists);
+
+        try {
+            executeUpdate(createTableSql);
+            LOG.info(
+                    "Successfully created table `{}`.`{}`. Raw SQL: {}",
+                    tableId.getSchemaName(),
+                    tableId.getTableName(),
+                    createTableSql);
+        } catch (SQLException e) {
+            throw new CatalogException(
+                    String.format(
+                            "Failed to create table `%s`.`%s`. Raw SQL: %s",
+                            tableId.getSchemaName(), tableId.getTableName(), 
createTableSql),
+                    e);
+        }
+    }
+
+    public void addColumn(AddColumnEvent addColumnEvent) throws 
CatalogException {
+        TableId tableId = addColumnEvent.tableId();
+        checkTableIdArguments(tableId);
+
+        Preconditions.checkArgument(
+                !addColumnEvent.getAddedColumns().isEmpty(), "Added columns 
should not be empty.");
+
+        String addColumnSql = buildAlterAddColumnsSql(tableId, 
addColumnEvent.getAddedColumns());
+        try {
+            executeUpdate(addColumnSql);
+            LOG.info(
+                    "Successfully added column in table `{}`.`{}`. Raw SQL: 
{}",
+                    tableId.getSchemaName(),
+                    tableId.getTableName(),
+                    addColumnSql);
+        } catch (Exception e) {
+            throw new CatalogException(
+                    String.format(
+                            "Failed to add column in table `%s`.`%s`. Raw SQL: 
%s",
+                            tableId.getSchemaName(), tableId.getTableName(), 
addColumnSql),
+                    e);
+        }
+    }
+
+    public void dropColumn(DropColumnEvent dropColumnEvent) throws 
CatalogException {
+        TableId tableId = dropColumnEvent.tableId();
+        checkTableIdArguments(tableId);
+
+        List<String> droppedColumnNames = 
dropColumnEvent.getDroppedColumnNames();
+        Preconditions.checkArgument(
+                !droppedColumnNames.isEmpty(), "Dropped columns should not be 
empty.");
+
+        droppedColumnNames.forEach(
+                column -> {
+                    String dropColumnSql = buildDropColumnSql(tableId, column);
+                    try {
+                        executeUpdate(dropColumnSql);
+                        LOG.info(
+                                "Successfully dropped column in table 
`{}`.`{}`. Raw SQL: {}",
+                                tableId.getSchemaName(),
+                                tableId.getTableName(),
+                                dropColumnSql);
+                    } catch (Exception e) {
+                        throw new CatalogException(
+                                String.format(
+                                        "Failed to drop column in table 
`%s`.`%s`. Raw SQL: %s",
+                                        tableId.getSchemaName(),
+                                        tableId.getTableName(),
+                                        dropColumnSql),
+                                e);
+                    }
+                });
+    }
+
+    public void renameColumn(RenameColumnEvent renameColumnEvent) throws 
CatalogException {
+        TableId tableId = renameColumnEvent.tableId();
+        checkTableIdArguments(tableId);
+
+        Map<String, String> nameMapping = renameColumnEvent.getNameMapping();
+        for (Map.Entry<String, String> entry : nameMapping.entrySet()) {
+            String renameColumnSql =
+                    buildRenameColumnSql(tableId, entry.getKey(), 
entry.getValue());
+            try {
+                executeUpdate(renameColumnSql);
+                LOG.info(
+                        "Successfully renamed column in table `{}`.`{}`. Raw 
SQL: {}",
+                        tableId.getSchemaName(),
+                        tableId.getTableName(),
+                        renameColumnSql);
+            } catch (Exception e) {
+                throw new CatalogException(
+                        String.format(
+                                "Failed to rename column in table `%s`.`%s`. 
Raw SQL: %s",
+                                tableId.getSchemaName(), 
tableId.getTableName(), renameColumnSql),
+                        e);
+            }
+        }
+    }
+
+    public void alterColumnType(AlterColumnTypeEvent alterColumnTypeEvent) 
throws CatalogException {
+        TableId tableId = alterColumnTypeEvent.tableId();
+        checkTableIdArguments(tableId);
+
+        Map<String, DataType> typeMapping = 
alterColumnTypeEvent.getTypeMapping();
+        for (Map.Entry<String, DataType> entry : typeMapping.entrySet()) {
+            String alterColumnTypeSql =
+                    buildAlterColumnTypeSql(tableId, entry.getKey(), 
entry.getValue());
+
+            try {
+                executeUpdate(alterColumnTypeSql);
+                LOG.info(
+                        "Successfully altered column type in table `{}`.`{}`. 
Raw SQL: {}",
+                        tableId.getSchemaName(),
+                        tableId.getTableName(),
+                        alterColumnTypeSql);
+            } catch (Exception e) {
+                throw new CatalogException(
+                        String.format(
+                                "Failed to alter column type in table 
`%s`.`%s`. Raw SQL: %s",
+                                tableId.getSchemaName(),
+                                tableId.getTableName(),
+                                alterColumnTypeSql),
+                        e);
+            }
+        }
+    }
+
+    public void truncateTable(TruncateTableEvent truncateTableEvent) throws 
CatalogException {
+        TableId tableId = truncateTableEvent.tableId();
+        checkTableIdArguments(tableId);
+
+        String truncateTableSql = buildTruncateTableSql(tableId);
+
+        try {
+            executeUpdate(truncateTableSql);
+            LOG.info(
+                    "Successfully truncated table `{}`.`{}`. Raw SQL: {}",
+                    tableId.getSchemaName(),
+                    tableId.getTableName(),
+                    truncateTableSql);
+        } catch (SQLException e) {
+            throw new CatalogException(
+                    String.format(
+                            "Failed to truncate table `%s`.`%s`. Raw SQL: %s",
+                            tableId.getSchemaName(), tableId.getTableName(), 
truncateTableSql),
+                    e);
+        }
+    }
+
+    public void dropTable(DropTableEvent dropTableEvent, boolean 
ignoreIfNotExist)
+            throws CatalogException {
+        TableId tableId = dropTableEvent.tableId();
+        checkTableIdArguments(tableId);
+
+        String truncateTableSql = buildDropTableSql(tableId, ignoreIfNotExist);

Review Comment:
   dropTableSql



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

Review Comment:
   org.apache.flink.cdc.connectors.jdbc.connection 



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-jdbc/src/main/java/org/apache/flink/cdc/connectors/jdbc/sink/v2/JdbcWriter.java:
##########
@@ -0,0 +1,248 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.jdbc.sink.v2;
+
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.StatefulSink;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.schema.Column;
+import org.apache.flink.cdc.connectors.jdbc.catalog.JdbcCatalog;
+import org.apache.flink.cdc.connectors.jdbc.sink.utils.JsonWrapper;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.JdbcStatementBuilder;
+import 
org.apache.flink.connector.jdbc.datasource.connections.JdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.JdbcOutputFormat;
+import org.apache.flink.connector.jdbc.internal.JdbcOutputSerializer;
+import 
org.apache.flink.connector.jdbc.internal.executor.JdbcBatchStatementExecutor;
+import org.apache.flink.connector.jdbc.sink.writer.JdbcWriterState;
+
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.type.TypeReference;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Implementation class of the {@link StatefulSink.StatefulSinkWriter} 
interface. */
+public class JdbcWriter<IN> implements StatefulSink.StatefulSinkWriter<IN, 
JdbcWriterState> {
+    private static final Logger LOG = 
LoggerFactory.getLogger(JdbcWriter.class);
+
+    private final JdbcExecutionOptions executionOptions;
+
+    private final JdbcConnectionProvider connectionProvider;
+
+    private final JdbcOutputSerializer<Object> outputSerializer;
+    private final RecordSerializationSchema<IN> serializationSchema;
+
+    private final JsonWrapper jsonWrapper;
+
+    private final JdbcCatalog catalog;
+    private final Map<
+                    TableId,
+                    JdbcOutputFormat<Object, JdbcRowData, 
JdbcBatchStatementExecutor<JdbcRowData>>>
+            jdbcUpsertOutputs;
+
+    private final Map<
+                    TableId,
+                    JdbcOutputFormat<Object, JdbcRowData, 
JdbcBatchStatementExecutor<JdbcRowData>>>
+            jdbcDeleteOutputs;
+
+    private JdbcOutputFormat<IN, IN, JdbcBatchStatementExecutor<IN>> 
jdbcOutput;
+
+    public JdbcWriter(
+            Sink.InitContext initContext,
+            JdbcExecutionOptions executionOptions,
+            JdbcConnectionProvider connectionProvider,
+            JdbcOutputSerializer<Object> outputSerializer,
+            RecordSerializationSchema<IN> serializationSchema,
+            JdbcCatalog catalog) {
+
+        checkNotNull(initContext, "initContext must be defined");
+        checkNotNull(executionOptions, "executionOptions must be defined");
+        checkNotNull(connectionProvider, "connectionProvider must be defined");
+        checkNotNull(outputSerializer, "outputSerializer must be defined");
+        checkNotNull(serializationSchema, "serializationSchema must be 
defined");
+
+        this.jsonWrapper = new JsonWrapper();
+        this.executionOptions = executionOptions;
+        this.connectionProvider = connectionProvider;
+        this.outputSerializer = outputSerializer;
+        this.serializationSchema = serializationSchema;
+        this.catalog = catalog;
+        this.jdbcUpsertOutputs = new ConcurrentHashMap<>();
+        this.jdbcDeleteOutputs = new ConcurrentHashMap<>();
+    }
+
+    @Override
+    public List<JdbcWriterState> snapshotState(long l) throws IOException {
+        return Collections.emptyList();
+    }
+
+    @Override
+    public void write(IN event, Context context) throws IOException {
+        JdbcRowData rowData = serializationSchema.serialize(event);
+        if (rowData == null) {
+            return;
+        }
+
+        if (RowKind.SCHEMA_CHANGE.is(rowData.getRowKind())) {
+            TableId tableId = rowData.getTableId();
+
+            // Close and remove from jdbcUpsertOutputs
+            JdbcOutputFormat<Object, JdbcRowData, 
JdbcBatchStatementExecutor<JdbcRowData>>
+                    upsertOutput = jdbcUpsertOutputs.get(tableId);
+            if (upsertOutput != null) {
+                upsertOutput.close();
+                jdbcUpsertOutputs.remove(tableId);
+            }
+
+            // Close and remove from jdbcDeleteOutputs
+            JdbcOutputFormat<Object, JdbcRowData, 
JdbcBatchStatementExecutor<JdbcRowData>>
+                    deleteOutput = jdbcDeleteOutputs.get(tableId);
+            if (deleteOutput != null) {
+                deleteOutput.close();
+                jdbcDeleteOutputs.remove(tableId);
+            }
+
+            return;
+        }
+
+        // insert event
+        if (RowKind.INSERT.is(rowData.getRowKind())) {
+            String upsertStmt =
+                    catalog.getUpsertStatement(rowData.getTableId(), 
rowData.getSchema());
+            JdbcStatementBuilder<JdbcRowData> upsertStmtBuilder =
+                    
getUpsertStatementBuilder(rowData.getSchema().getColumns());
+
+            JdbcOutputFormat<Object, JdbcRowData, 
JdbcBatchStatementExecutor<JdbcRowData>>
+                    outputFormat =
+                            jdbcUpsertOutputs.computeIfAbsent(
+                                    rowData.getTableId(),
+                                    id -> getJdbcOutputFormat(upsertStmt, 
upsertStmtBuilder));
+
+            outputFormat.writeRecord(rowData);
+            outputFormat.flush();
+            LOG.debug(
+                    "Successfully upserted into table `{}`.`{}`. Raw SQL: {}",
+                    rowData.getTableId().getSchemaName(),
+                    rowData.getTableId().getTableName(),
+                    upsertStmt);
+        }
+
+        // delete event
+        if (RowKind.DELETE.is(rowData.getRowKind())) {
+            String deleteStmt =
+                    catalog.getDeleteStatement(
+                            rowData.getTableId(), 
rowData.getSchema().primaryKeys());
+            JdbcStatementBuilder<JdbcRowData> delStmtBuilder =
+                    
getDeleteStatementBuilder(rowData.getSchema().primaryKeys());
+
+            JdbcOutputFormat<Object, JdbcRowData, 
JdbcBatchStatementExecutor<JdbcRowData>>
+                    outputFormat =
+                            jdbcDeleteOutputs.computeIfAbsent(
+                                    rowData.getTableId(),
+                                    id -> getJdbcOutputFormat(deleteStmt, 
delStmtBuilder));
+
+            outputFormat.writeRecord(rowData);
+            outputFormat.flush();
+            LOG.debug(
+                    "Successfully deleted rows in table `{}`.`{}`. Raw SQL: 
{}",
+                    rowData.getTableId().getSchemaName(),
+                    rowData.getTableId().getTableName(),
+                    deleteStmt);
+        }
+    }
+
+    private JdbcOutputFormat<Object, JdbcRowData, 
JdbcBatchStatementExecutor<JdbcRowData>>
+            getJdbcOutputFormat(String sql, JdbcStatementBuilder<JdbcRowData> 
builder) {
+        JdbcOutputFormat<Object, JdbcRowData, 
JdbcBatchStatementExecutor<JdbcRowData>>
+                jdbcOutputFormat =
+                        new JdbcOutputFormat<>(
+                                connectionProvider,
+                                executionOptions,
+                                () -> JdbcBatchStatementExecutor.simple(sql, 
builder));
+        try {
+            jdbcOutputFormat.open(outputSerializer);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+        return jdbcOutputFormat;
+    }
+
+    @Override
+    public void flush(boolean b) throws IOException, RuntimeException {
+        // To ensure the sequential execution of insert and delete,
+        // PrepareStatements cannot perform insert and delete batch operations 
at the same time,
+        // so flush operations are not used separately.

Review Comment:
   I think there may be performance issues. 
   However, this depends on us emitting information on whether it is in the 
snapshot reading stage, so we can handle it uniformly in the future.



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-jdbc/pom.xml:
##########
@@ -0,0 +1,42 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<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";>
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.flink</groupId>
+        <artifactId>flink-cdc-pipeline-connectors</artifactId>
+        <version>${revision}</version>
+    </parent>
+
+    <artifactId>flink-cdc-pipeline-connector-jdbc</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-connector-jdbc</artifactId>
+            <version>3.2.0-${flink.major.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.zaxxer</groupId>
+            <artifactId>HikariCP</artifactId>
+            <version>4.0.3</version>

Review Comment:
   We can extract the version of HikariCP to a common properties to avoid 
conflict.



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

Review Comment:
   Well, we have too many `ConnectionPools`.  
   But this is not suitable for being placed in the common module, so for now 
we can only let them exist first.



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/factory/MySqlDataSinkFactory.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.mysql.factory;
+
+import org.apache.flink.cdc.common.configuration.ConfigOption;
+import org.apache.flink.cdc.common.configuration.Configuration;
+import org.apache.flink.cdc.common.factories.DataSinkFactory;
+import org.apache.flink.cdc.common.factories.Factory;
+import org.apache.flink.cdc.common.factories.FactoryHelper;
+import org.apache.flink.cdc.common.sink.DataSink;
+import org.apache.flink.cdc.connectors.jdbc.options.JdbcSinkOptions;
+import org.apache.flink.cdc.connectors.jdbc.sink.JdbcDataSink;
+import org.apache.flink.cdc.connectors.mysql.sink.MySqlDataSinkConfig;
+import org.apache.flink.cdc.connectors.mysql.sink.MySqlPooledDataSinkFactory;
+import org.apache.flink.cdc.connectors.mysql.sink.catalog.MySqlCatalog;
+import org.apache.flink.cdc.connectors.mysql.sink.catalog.MySqlCatalogFactory;
+import org.apache.flink.cdc.connectors.mysql.utils.OptionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import static 
org.apache.flink.cdc.connectors.jdbc.options.JdbcSinkOptions.JDBC_PROPERTIES_PROP_PREFIX;
+import static 
org.apache.flink.cdc.connectors.mysql.sink.MySqlDataSinkOptions.CONNECTION_POOL_SIZE;
+import static 
org.apache.flink.cdc.connectors.mysql.sink.MySqlDataSinkOptions.CONNECT_MAX_RETRIES;
+import static 
org.apache.flink.cdc.connectors.mysql.sink.MySqlDataSinkOptions.CONNECT_TIMEOUT;
+import static 
org.apache.flink.cdc.connectors.mysql.sink.MySqlDataSinkOptions.DRIVER_CLASS_NAME;
+import static 
org.apache.flink.cdc.connectors.mysql.sink.MySqlDataSinkOptions.HOSTNAME;
+import static 
org.apache.flink.cdc.connectors.mysql.sink.MySqlDataSinkOptions.PASSWORD;
+import static 
org.apache.flink.cdc.connectors.mysql.sink.MySqlDataSinkOptions.PORT;
+import static 
org.apache.flink.cdc.connectors.mysql.sink.MySqlDataSinkOptions.SERVER_TIME_ZONE;
+import static 
org.apache.flink.cdc.connectors.mysql.sink.MySqlDataSinkOptions.USERNAME;
+
+/** A {@link Factory} to create MySQL Data Sink. */
+public class MySqlDataSinkFactory implements DataSinkFactory {
+    private static final Logger LOG = 
LoggerFactory.getLogger(MySqlDataSinkFactory.class);
+
+    public static final String IDENTIFIER = "mysql";
+
+    @Override
+    public DataSink createDataSink(Context context) {
+        FactoryHelper.createFactoryHelper(this, context)
+                .validateExcept(JDBC_PROPERTIES_PROP_PREFIX);
+
+        final Configuration config = context.getFactoryConfiguration();
+        MySqlDataSinkConfig.Builder builder = new 
MySqlDataSinkConfig.Builder();
+
+        config.getOptional(HOSTNAME).ifPresent(builder::hostname);
+        config.getOptional(PORT).ifPresent(builder::port);
+        config.getOptional(USERNAME).ifPresent(builder::username);
+        config.getOptional(PASSWORD).ifPresent(builder::password);
+
+        
builder.serverTimeZone(config.getOptional(SERVER_TIME_ZONE).orElse("UTC"));
+        builder.connectTimeout(config.get(CONNECT_TIMEOUT));
+        builder.connectionPoolSize(config.get(CONNECTION_POOL_SIZE));
+        builder.connectMaxRetries(config.get(CONNECT_MAX_RETRIES));
+        builder.driverClassName(config.get(DRIVER_CLASS_NAME));
+
+        Properties properties = new Properties();
+        Map<String, String> jdbcProperties =
+                JdbcSinkOptions.getPropertiesByPrefix(config, 
JDBC_PROPERTIES_PROP_PREFIX);
+        properties.putAll(jdbcProperties);
+        builder.jdbcProperties(properties);
+
+        String jdbcUrl = 
MySqlPooledDataSinkFactory.INSTANCE.getJdbcUrl(builder.build());
+        builder.connUrl(jdbcUrl);
+
+        if (LOG.isInfoEnabled()) {
+            OptionUtils.printOptions(IDENTIFIER, config.toMap());

Review Comment:
   We could use `ConfigurationUtils.hideSensitiveValues` to avoid exposing 
sensitive message.



-- 
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

Reply via email to